diff --git a/build.sbt b/build.sbt index 35d014948..c09112ec7 100644 --- a/build.sbt +++ b/build.sbt @@ -301,6 +301,21 @@ lazy val sqs = (project in file("modules/sqs")) ).dependsOn(core % "compile->compile;test->test") .settings(name := "sqs") +lazy val r53 = (project in file("modules/r53")) + .enablePlugins(AutomateHeaderPlugin) + .configs(IntegrationTest) + .settings(sharedSettings) + .settings(headerSettings(IntegrationTest)) + .settings(inConfig(IntegrationTest)(scalafmtConfigSettings)) + .settings(corePublishSettings) + .settings(testSettings) + .settings(Defaults.itSettings) + .settings(libraryDependencies ++= r53Dependencies ++ commonTestDependencies.map(_ % "test, it")) + .settings( + organization := "io.vinyldns", + ).dependsOn(core % "compile->compile;test->test") + .settings(name := "r53") + val preparePortal = TaskKey[Unit]("preparePortal", "Runs NPM to prepare portal for start") val checkJsHeaders = TaskKey[Unit]("checkJsHeaders", "Runs script to check for APL 2.0 license headers") val createJsHeaders = TaskKey[Unit]("createJsHeaders", "Runs script to prepend APL 2.0 license headers to files") @@ -446,6 +461,7 @@ addCommandAlias("validate", "; root/clean; " + "api/headerCheck api/test:headerCheck api/it:headerCheck " + "dynamodb/headerCheck dynamodb/test:headerCheck dynamodb/it:headerCheck " + "mysql/headerCheck mysql/test:headerCheck mysql/it:headerCheck " + + "r53/headerCheck r53/test:headerCheck r53/it:headerCheck " + "sqs/headerCheck sqs/test:headerCheck sqs/it:headerCheck " + "portal/headerCheck portal/test:headerCheck; " + "portal/createJsHeaders;portal/checkJsHeaders;" + diff --git a/docker/docker-compose.yml b/docker/docker-compose.yml index ba71c010b..4dde336a1 100644 --- a/docker/docker-compose.yml +++ b/docker/docker-compose.yml @@ -31,8 +31,9 @@ services: ports: - "19006:19006" - "19007:19007" + - "19009:19009" environment: - - SERVICES=sns:19006,sqs:19007 + - SERVICES=sns:19006,sqs:19007,route53:19009 - START_WEB=0 mail: diff --git a/modules/api/functional_test/live_tests/batch/create_batch_change_test.py b/modules/api/functional_test/live_tests/batch/create_batch_change_test.py index 8246fe69c..cc470a17b 100644 --- a/modules/api/functional_test/live_tests/batch/create_batch_change_test.py +++ b/modules/api/functional_test/live_tests/batch/create_batch_change_test.py @@ -3894,7 +3894,7 @@ def test_create_batch_delete_record_for_invalid_record_data_fails(shared_zone_te assert_failed_change_in_error_response(errors[0], input_name="delete-non-existent-record.ok.", record_data="1.1.1.1", change_type="DeleteRecordSet", error_messages=['Record "delete-non-existent-record.ok." Does Not Exist: cannot delete a record that does not exist.']) assert_failed_change_in_error_response(errors[1], input_name=a_delete_fqdn, record_data="4.5.6.7", change_type="DeleteRecordSet", - error_messages=['Record data AData(4.5.6.7) does not exist for "' + a_delete_fqdn + '".']) + error_messages=['Record data 4.5.6.7 does not exist for "' + a_delete_fqdn + '".']) finally: clear_recordset_list(to_delete, client) diff --git a/modules/api/src/it/scala/vinyldns/api/domain/dns/DnsConversionsIntegrationSpec.scala b/modules/api/src/it/scala/vinyldns/api/domain/dns/DnsConversionsIntegrationSpec.scala deleted file mode 100644 index f37351d1d..000000000 --- a/modules/api/src/it/scala/vinyldns/api/domain/dns/DnsConversionsIntegrationSpec.scala +++ /dev/null @@ -1,98 +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.domain.dns - -import org.scalatest.matchers.should.Matchers -import org.scalatest.wordspec.AnyWordSpec -import org.xbill.DNS -import vinyldns.api.domain.dns.DnsProtocol.{DnsResponse, NoError} -import vinyldns.api.domain.record.RecordSetChangeGenerator -import vinyldns.core.domain.zone.{Zone, ZoneConnection, ZoneStatus} -import vinyldns.api.ResultHelpers -import vinyldns.core.TestRecordSetData.{aaaa, ds} -import vinyldns.core.domain.record.{RecordSet, RecordType} - -class DnsConversionsIntegrationSpec extends AnyWordSpec with Matchers with ResultHelpers { - - private val zoneName = "example.com." - private val testZone = Zone( - zoneName, - "test@test.com", - ZoneStatus.Active, - connection = - Some(ZoneConnection("vinyldns.", "vinyldns.", "nzisn+4G2ldMn0q1CV3vsg==", "127.0.0.1:19001")), - transferConnection = - Some(ZoneConnection("vinyldns.", "vinyldns.", "nzisn+4G2ldMn0q1CV3vsg==", "127.0.0.1:19001")) - ) - - "Interacting with the DNS backend" should { - "remove the tsig key value during an update" in { - val testRecord = aaaa.copy(zoneId = testZone.id) - val conn = DnsConnection(testZone.connection.get) - val result: DnsResponse = - rightResultOf(conn.addRecord(RecordSetChangeGenerator.forAdd(testRecord, testZone)).value) - - result shouldBe a[NoError] - val resultingMessage = result.asInstanceOf[NoError].message - resultingMessage.getSectionArray(DNS.Section.ADDITIONAL) shouldBe empty - - val resultingMessageString = resultingMessage.toString - - resultingMessageString should not contain "TSIG" - - val queryResult: List[RecordSet] = - rightResultOf(conn.resolve(testRecord.name, testZone.name, RecordType.AAAA).value) - - val recordOut = queryResult.head - recordOut.records should contain theSameElementsAs testRecord.records - recordOut.name shouldBe testRecord.name - recordOut.ttl shouldBe testRecord.ttl - recordOut.typ shouldBe testRecord.typ - } - "Successfully add and remove DS record type" in { - val testRecord = ds.copy(zoneId = testZone.id) - - val conn = DnsConnection(testZone.connection.get) - val result: DnsResponse = - rightResultOf(conn.addRecord(RecordSetChangeGenerator.forAdd(testRecord, testZone)).value) - - result shouldBe a[NoError] - - val queryResult: List[RecordSet] = - rightResultOf(conn.resolve(testRecord.name, testZone.name, RecordType.DS).value) - - val recordOut = queryResult.head - recordOut.records should contain theSameElementsAs testRecord.records - recordOut.name shouldBe testRecord.name - recordOut.ttl shouldBe testRecord.ttl - recordOut.typ shouldBe testRecord.typ - - // deleting the record just added - val deleteResult: DnsResponse = - rightResultOf( - conn.deleteRecord(RecordSetChangeGenerator.forAdd(testRecord, testZone)).value - ) - - deleteResult shouldBe a[NoError] - - val deleteQuery: List[RecordSet] = - rightResultOf(conn.resolve(testRecord.name, testZone.name, RecordType.DS).value) - - deleteQuery shouldBe List.empty - } - } -} diff --git a/modules/api/src/it/scala/vinyldns/api/domain/record/RecordSetServiceIntegrationSpec.scala b/modules/api/src/it/scala/vinyldns/api/domain/record/RecordSetServiceIntegrationSpec.scala index c4f34f157..3b3eec11f 100644 --- a/modules/api/src/it/scala/vinyldns/api/domain/record/RecordSetServiceIntegrationSpec.scala +++ b/modules/api/src/it/scala/vinyldns/api/domain/record/RecordSetServiceIntegrationSpec.scala @@ -24,16 +24,15 @@ import org.scalatest.matchers.should.Matchers import org.scalatest.concurrent.PatienceConfiguration import org.scalatestplus.mockito.MockitoSugar import org.scalatest.time.{Seconds, Span} -import vinyldns.api.Interfaces._ import vinyldns.api._ import vinyldns.api.domain.access.AccessValidations -import vinyldns.api.domain.dns.DnsConnection import vinyldns.api.domain.zone._ import vinyldns.api.engine.TestMessageQueue import vinyldns.core.TestMembershipData._ import vinyldns.core.TestZoneData.testConnection import vinyldns.core.domain.{Fqdn, HighValueDomainError} import vinyldns.core.domain.auth.AuthPrincipal +import vinyldns.core.domain.backend.{Backend, BackendResolver} import vinyldns.core.domain.membership.{Group, GroupRepository, User, UserRepository} import vinyldns.core.domain.record.RecordType._ import vinyldns.core.domain.record._ @@ -224,13 +223,8 @@ class RecordSetServiceIntegrationSpec ownerGroupId = Some(sharedGroup.id) ) - private val zoneConnection = - ZoneConnection("vinyldns.", "vinyldns.", "nzisn+4G2ldMn0q1CV3vsg==", "10.1.1.1") - - private val configuredConnections = - ConfiguredDnsConnections(zoneConnection, zoneConnection, List()) - - private val mockDnsConnection = mock[DnsConnection] + private val mockBackendResolver = mock[BackendResolver] + private val mockBackend = mock[Backend] def setup(): Unit = { recordSetRepo = @@ -266,8 +260,7 @@ class RecordSetServiceIntegrationSpec mock[UserRepository], TestMessageQueue, new AccessValidations(), - (_, _) => mockDnsConnection, - configuredConnections, + mockBackendResolver, false ) } @@ -383,8 +376,8 @@ class RecordSetServiceIntegrationSpec "fail to add relative record if apex record with same name already exists" in { val newRecord = apexTestRecordNameConflict.copy(name = "zone-test-name-conflicts") - doReturn(IO(List(newRecord)).toResult) - .when(mockDnsConnection) + doReturn(IO(List(newRecord))) + .when(mockBackend) .resolve( zoneTestNameConflicts.name, zoneTestNameConflicts.name, @@ -405,8 +398,8 @@ class RecordSetServiceIntegrationSpec "fail to add apex record if relative record with same name already exists" in { val newRecord = subTestRecordNameConflict.copy(name = "relative-name-conflict.") - doReturn(IO(List(newRecord)).toResult) - .when(mockDnsConnection) + doReturn(IO(List(newRecord))) + .when(mockBackend) .resolve(newRecord.name, zoneTestNameConflicts.name, newRecord.typ) val result = diff --git a/modules/api/src/it/scala/vinyldns/api/domain/zone/ZoneConnectionValidatorIntegrationSpec.scala b/modules/api/src/it/scala/vinyldns/api/domain/zone/ZoneConnectionValidatorIntegrationSpec.scala deleted file mode 100644 index 991862d00..000000000 --- a/modules/api/src/it/scala/vinyldns/api/domain/zone/ZoneConnectionValidatorIntegrationSpec.scala +++ /dev/null @@ -1,50 +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.domain.zone -import cats.scalatest.EitherMatchers -import org.scalatest.matchers.should.Matchers -import org.scalatest.wordspec.AnyWordSpec -import vinyldns.api.VinylDNSConfig -import vinyldns.core.domain.zone.ConfiguredDnsConnections -import vinyldns.core.health.HealthCheck.HealthCheckError - -class ZoneConnectionValidatorIntegrationSpec extends AnyWordSpec with Matchers with EitherMatchers { - "ZoneConnectionValidatorIntegrationSpec" should { - "have a valid health check if we can connect to DNS backend" in { - val check = new ZoneConnectionValidator(VinylDNSConfig.configuredDnsConnections) - .healthCheck(10000) - .unsafeRunSync() - check should beRight(()) - } - - "respond with a failure if health check fails" in { - val connections = VinylDNSConfig.configuredDnsConnections - val badConn = connections.defaultZoneConnection.copy(primaryServer = "localhost:1234") - val toTest = ConfiguredDnsConnections(badConn, badConn, List()) - val result = - new ZoneConnectionValidator(toTest) - .healthCheck(10000) - .unsafeRunSync() - result should beLeft( - HealthCheckError( - "vinyldns.api.domain.zone.ZoneConnectionValidator health " + - "check failed with msg='Connection refused (Connection refused)'" - ) - ) - } - } -} diff --git a/modules/api/src/it/scala/vinyldns/api/domain/zone/ZoneServiceIntegrationSpec.scala b/modules/api/src/it/scala/vinyldns/api/domain/zone/ZoneServiceIntegrationSpec.scala index 779684210..ebd631259 100644 --- a/modules/api/src/it/scala/vinyldns/api/domain/zone/ZoneServiceIntegrationSpec.scala +++ b/modules/api/src/it/scala/vinyldns/api/domain/zone/ZoneServiceIntegrationSpec.scala @@ -16,8 +16,10 @@ package vinyldns.api.domain.zone +import cats.data.NonEmptyList import cats.effect._ import org.joda.time.DateTime +import org.mockito.Mockito.doReturn import org.scalatest._ import org.scalatest.matchers.should.Matchers import org.scalatest.wordspec.AnyWordSpec @@ -33,6 +35,7 @@ import vinyldns.core.TestMembershipData.{okAuth, okUser} import vinyldns.core.TestZoneData.okZone import vinyldns.core.domain.Fqdn import vinyldns.core.domain.auth.AuthPrincipal +import vinyldns.core.domain.backend.BackendResolver import vinyldns.core.domain.membership.{GroupRepository, UserRepository} import vinyldns.core.domain.record._ import vinyldns.core.domain.zone._ @@ -93,6 +96,8 @@ class ZoneServiceIntegrationSpec private val changeSetNS = ChangeSet(RecordSetChangeGenerator.forAdd(testRecordNS, okZone)) private val changeSetA = ChangeSet(RecordSetChangeGenerator.forAdd(testRecordA, okZone)) + private val mockBackendResolver = mock[BackendResolver] + def clearRecordSetRepo(): Unit = DB.localTx { s => s.executeUpdate("DELETE FROM recordset") @@ -113,6 +118,8 @@ class ZoneServiceIntegrationSpec waitForSuccess(recordSetRepo.apply(changeSetNS)) waitForSuccess(recordSetRepo.apply(changeSetA)) + doReturn(NonEmptyList.one("func-test-backend")).when(mockBackendResolver).ids + testZoneService = new ZoneService( zoneRepo, mock[GroupRepository], @@ -121,7 +128,8 @@ class ZoneServiceIntegrationSpec mock[ZoneConnectionValidator], TestMessageQueue, new ZoneValidations(1000), - new AccessValidations() + new AccessValidations(), + mockBackendResolver ) } diff --git a/modules/api/src/it/scala/vinyldns/api/domain/zone/ZoneViewLoaderIntegrationSpec.scala b/modules/api/src/it/scala/vinyldns/api/domain/zone/ZoneViewLoaderIntegrationSpec.scala index 7ab875962..8dcf272af 100644 --- a/modules/api/src/it/scala/vinyldns/api/domain/zone/ZoneViewLoaderIntegrationSpec.scala +++ b/modules/api/src/it/scala/vinyldns/api/domain/zone/ZoneViewLoaderIntegrationSpec.scala @@ -16,46 +16,84 @@ package vinyldns.api.domain.zone +import cats.effect.{ContextShift, IO} import org.scalatest.matchers.should.Matchers import org.scalatest.wordspec.AnyWordSpec import org.xbill.DNS.ZoneTransferException +import vinyldns.api.VinylDNSConfig +import vinyldns.api.backend.dns.DnsBackend +import vinyldns.core.domain.backend.{BackendConfigs, BackendResolver} import vinyldns.core.domain.zone.{Zone, ZoneConnection} +import scala.concurrent.ExecutionContext + class ZoneViewLoaderIntegrationSpec extends AnyWordSpec with Matchers { + private implicit val ec: ExecutionContext = scala.concurrent.ExecutionContext.global + private implicit val cs: ContextShift[IO] = IO.contextShift(ec) + private val backendResolver = + BackendResolver + .apply(BackendConfigs.load(VinylDNSConfig.apiBackend).unsafeRunSync()) + .unsafeRunSync() + "ZoneViewLoader" should { "return a ZoneView upon success" in { - DnsZoneViewLoader(Zone("vinyldns.", "test@test.com")) + val zone = Zone("vinyldns.", "test@test.com") + DnsZoneViewLoader(zone, backendResolver.resolve(zone)) .load() .unsafeRunSync() shouldBe a[ZoneView] } "return a failure if the transfer connection is bad" in { - assertThrows[IllegalArgumentException]( - DnsZoneViewLoader( - Zone("vinyldns.", "bad@transfer.connection") - .copy( - transferConnection = - Some(ZoneConnection("invalid-connection.", "bad-key", "invalid-key", "10.1.1.1")) + assertThrows[IllegalArgumentException] { + val zone = Zone( + "vinyldns.", + "bad@transfer.connection", + connection = Some( + ZoneConnection( + "vinyldns.", + "vinyldns.", + "nzisn+4G2ldMn0q1CV3vsg==", + "127.0.0.1:19001" ) - ).load() + ), + transferConnection = + Some(ZoneConnection("invalid-connection.", "bad-key", "invalid-key", "10.1.1.1")) + ) + val backend = backendResolver.resolve(zone).asInstanceOf[DnsBackend] + println(s"${backend.id}, ${backend.xfrInfo}, ${backend.resolver.getAddress}") + DnsZoneViewLoader(zone, backendResolver.resolve(zone)) + .load() .unsafeRunSync() - ) + } } "return a failure if the zone doesn't exist in the DNS backend" in { - assertThrows[ZoneTransferException]( - DnsZoneViewLoader(Zone("non-existent-zone", "bad@zone.test")) + assertThrows[ZoneTransferException] { + val zone = Zone("non-existent-zone", "bad@zone.test") + DnsZoneViewLoader(zone, backendResolver.resolve(zone)) .load() .unsafeRunSync() - ) + } } "return a failure if the zone is larger than the max zone size" in { - assertThrows[ZoneTooLargeError]( - DnsZoneViewLoader(Zone("vinyldns.", "test@test.com"), DnsZoneViewLoader.dnsZoneTransfer, 1) + assertThrows[ZoneTooLargeError] { + val zone = Zone( + "vinyldns.", + "test@test.com", + connection = Some( + ZoneConnection( + "vinyldns.", + "vinyldns.", + "nzisn+4G2ldMn0q1CV3vsg==", + "127.0.0.1:19001" + ) + ) + ) + DnsZoneViewLoader(zone, backendResolver.resolve(zone), 1) .load() .unsafeRunSync() - ) + } } } } diff --git a/modules/api/src/main/resources/reference.conf b/modules/api/src/main/resources/reference.conf index 3546f9d81..a28e05760 100644 --- a/modules/api/src/main/resources/reference.conf +++ b/modules/api/src/main/resources/reference.conf @@ -5,6 +5,59 @@ ################################################################################################################ vinyldns { + # configured backend providers + backend { + # Use "default" when dns backend legacy = true + # otherwise, use the id of one of the connections in any of your backends + default-backend-id = "default" + + # this is where we can save additional backends + backend-providers = [ + { + class-name = "vinyldns.api.backend.dns.DnsBackendProviderLoader" + settings = { + legacy = true # set this to true to attempt to load legacy config YAML + backends = [] + + # if not legacy then this... + # legacy = false + # backends = [ + # { + # id = "default" + # zone-connection = { + # name = "vinyldns." + # keyName = "vinyldns." + # key = "nzisn+4G2ldMn0q1CV3vsg==" + # primaryServer = "127.0.0.1:19001" + # } + # transfer-connection = { + # name = "vinyldns." + # keyName = "vinyldns." + # key = "nzisn+4G2ldMn0q1CV3vsg==" + # primaryServer = "127.0.0.1:19001" + # } + # }, + # { + # id = "func-test-backend" + # zone-connection = { + # name = "vinyldns." + # keyName = "vinyldns." + # key = "nzisn+4G2ldMn0q1CV3vsg==" + # primaryServer = "127.0.0.1:19001" + # } + # transfer-connection = { + # name = "vinyldns." + # keyName = "vinyldns." + # key = "nzisn+4G2ldMn0q1CV3vsg==" + # primaryServer = "127.0.0.1:19001" + # } + # } + #] + } + } + ] + } + # if we should start up polling for change requests, set this to false for the inactive cluster processing-disabled = false diff --git a/modules/api/src/main/scala/vinyldns/api/Boot.scala b/modules/api/src/main/scala/vinyldns/api/Boot.scala index f2d939484..1a1a9df5c 100644 --- a/modules/api/src/main/scala/vinyldns/api/Boot.scala +++ b/modules/api/src/main/scala/vinyldns/api/Boot.scala @@ -30,7 +30,6 @@ import vinyldns.api.crypto.Crypto import vinyldns.api.domain.access.AccessValidations import vinyldns.api.domain.auth.MembershipAuthPrincipalProvider import vinyldns.api.domain.batch.{BatchChangeConverter, BatchChangeService, BatchChangeValidations} -import vinyldns.api.domain.dns.DnsConnection import vinyldns.api.domain.membership._ import vinyldns.api.domain.record.RecordSetService import vinyldns.api.domain.zone._ @@ -38,6 +37,7 @@ import vinyldns.api.metrics.APIMetrics import vinyldns.api.repository.{ApiDataAccessor, ApiDataAccessorProvider, TestDataLoader} import vinyldns.api.route.VinylDNSService import vinyldns.core.VinylDNSMetrics +import vinyldns.core.domain.backend.{BackendConfigs, BackendResolver} import vinyldns.core.health.HealthService import vinyldns.core.queue.{MessageCount, MessageQueueLoader} import vinyldns.core.repository.DataStoreLoader @@ -74,7 +74,8 @@ object Boot extends App { loaderResponse <- DataStoreLoader .loadAll[ApiDataAccessor](repoConfigs, crypto, ApiDataAccessorProvider) repositories = loaderResponse.accessor - connections = VinylDNSConfig.configuredDnsConnections + backendConfigs <- BackendConfigs.load(VinylDNSConfig.apiBackend) + backendResolver <- BackendResolver.apply(backendConfigs) _ <- TestDataLoader .loadTestData( repositories.userRepository, @@ -109,7 +110,7 @@ object Boot extends App { repositories.recordChangeRepository, repositories.batchChangeRepository, notifiers, - connections + backendResolver ) .start } yield { @@ -123,15 +124,13 @@ object Boot extends App { ) val membershipService = MembershipService(repositories) val connectionValidator = - new ZoneConnectionValidator(connections) + new ZoneConnectionValidator(backendResolver) val recordSetService = RecordSetService( repositories, messageQueue, recordAccessValidations, - (zone, connections) => - DnsConnection(ZoneConnectionValidator.getZoneConnection(zone, connections)), - connections, + backendResolver, VinylDNSConfig.validateRecordLookupAgainstDnsBackend ) val zoneService = ZoneService( @@ -139,10 +138,11 @@ object Boot extends App { connectionValidator, messageQueue, zoneValidations, - recordAccessValidations + recordAccessValidations, + backendResolver ) val healthService = new HealthService( - messageQueue.healthCheck :: connectionValidator.healthCheck(healthCheckTimeout) :: + messageQueue.healthCheck :: backendResolver.healthCheck(healthCheckTimeout) :: loaderResponse.healthChecks ) val batchChangeConverter = diff --git a/modules/api/src/main/scala/vinyldns/api/VinylDNSConfig.scala b/modules/api/src/main/scala/vinyldns/api/VinylDNSConfig.scala index 1ddb91347..1c9d9a04b 100644 --- a/modules/api/src/main/scala/vinyldns/api/VinylDNSConfig.scala +++ b/modules/api/src/main/scala/vinyldns/api/VinylDNSConfig.scala @@ -35,7 +35,7 @@ import vinyldns.core.domain.record.RecordType import scala.collection.JavaConverters._ import scala.util.matching.Regex -import vinyldns.core.domain.zone.{ConfiguredDnsConnections, DnsBackend, ZoneConnection} +import vinyldns.core.domain.zone.{ConfiguredDnsConnections, LegacyDnsBackend, ZoneConnection} import vinyldns.core.queue.MessageQueueConfig import vinyldns.core.repository.DataStoreConfig import vinyldns.core.notifier.NotifierConfig @@ -47,6 +47,9 @@ object VinylDNSConfig { lazy val config: Config = ConfigFactory.load() lazy val vinyldnsConfig: Config = config.getConfig("vinyldns") + lazy val apiBackend: Config = + vinyldnsConfig.getConfig("backend") + lazy val dataStoreConfigs: IO[List[DataStoreConfig]] = vinyldnsConfig .getStringList("data-stores") @@ -119,7 +122,7 @@ object VinylDNSConfig { .getConfigList("backends") .asScala .map { - ConfigSource.fromConfig(_).loadOrThrow[DnsBackend] + ConfigSource.fromConfig(_).loadOrThrow[LegacyDnsBackend] } .toList .map(_.encrypted(Crypto.instance)) diff --git a/modules/api/src/main/scala/vinyldns/api/backend/CommandHandler.scala b/modules/api/src/main/scala/vinyldns/api/backend/CommandHandler.scala index 2a8e36dd2..774d467c7 100644 --- a/modules/api/src/main/scala/vinyldns/api/backend/CommandHandler.scala +++ b/modules/api/src/main/scala/vinyldns/api/backend/CommandHandler.scala @@ -20,14 +20,13 @@ import cats.effect.{ContextShift, IO, Timer} import fs2._ import fs2.concurrent.SignallingRef import org.slf4j.LoggerFactory -import vinyldns.api.domain.dns.DnsConnection -import vinyldns.api.domain.zone.ZoneConnectionValidator import vinyldns.api.engine.{ BatchChangeHandler, RecordSetChangeHandler, ZoneChangeHandler, ZoneSyncHandler } +import vinyldns.core.domain.backend.{Backend, BackendResolver} import vinyldns.core.domain.batch.{BatchChange, BatchChangeCommand, BatchChangeRepository} import vinyldns.core.domain.record.{RecordChangeRepository, RecordSetChange, RecordSetRepository} import vinyldns.core.domain.zone._ @@ -51,14 +50,14 @@ object CommandHandler { def mainFlow( zoneChangeHandler: ZoneChange => IO[ZoneChange], - recordChangeHandler: (DnsConnection, RecordSetChange) => IO[RecordSetChange], + recordChangeHandler: (Backend, RecordSetChange) => IO[RecordSetChange], zoneSyncHandler: ZoneChange => IO[ZoneChange], batchChangeHandler: BatchChangeCommand => IO[Option[BatchChange]], mq: MessageQueue, count: MessageCount, pollingInterval: FiniteDuration, pauseSignal: SignallingRef[IO, Boolean], - connections: ConfiguredDnsConnections, + backendResolver: BackendResolver, maxOpen: Int = 4 )(implicit timer: Timer[IO]): Stream[IO, Unit] = { @@ -74,7 +73,7 @@ object CommandHandler { recordChangeHandler, zoneSyncHandler, batchChangeHandler, - connections + backendResolver ) // Delete messages from message queue when complete @@ -146,10 +145,10 @@ object CommandHandler { /* Actually processes a change request */ def processChangeRequests( zoneChangeProcessor: ZoneChange => IO[ZoneChange], - recordChangeProcessor: (DnsConnection, RecordSetChange) => IO[RecordSetChange], + recordChangeProcessor: (Backend, RecordSetChange) => IO[RecordSetChange], zoneSyncProcessor: ZoneChange => IO[ZoneChange], batchChangeProcessor: BatchChangeCommand => IO[Option[BatchChange]], - connections: ConfiguredDnsConnections + backendResolver: BackendResolver ): Pipe[IO, CommandMessage, MessageOutcome] = _.evalMap[IO, MessageOutcome] { message => message.command match { @@ -161,9 +160,7 @@ object CommandHandler { outcomeOf(message)(zoneChangeProcessor(zoneChange)) case rcr: RecordSetChange => - val dnsConn = - DnsConnection(ZoneConnectionValidator.getZoneConnection(rcr.zone, connections)) - outcomeOf(message)(recordChangeProcessor(dnsConn, rcr)) + outcomeOf(message)(recordChangeProcessor(backendResolver.resolve(rcr.zone), rcr)) case bcc: BatchChangeCommand => outcomeOf(message)(batchChangeProcessor(bcc)) @@ -207,7 +204,7 @@ object CommandHandler { recordChangeRepo: RecordChangeRepository, batchChangeRepo: BatchChangeRepository, notifiers: AllNotifiers, - connections: ConfiguredDnsConnections + backendResolver: BackendResolver )(implicit timer: Timer[IO]): IO[Unit] = { // Handlers for each type of change request val zoneChangeHandler = @@ -215,7 +212,7 @@ object CommandHandler { val recordChangeHandler = RecordSetChangeHandler(recordSetRepo, recordChangeRepo, batchChangeRepo) val zoneSyncHandler = - ZoneSyncHandler(recordSetRepo, recordChangeRepo, zoneChangeRepo, zoneRepo) + ZoneSyncHandler(recordSetRepo, recordChangeRepo, zoneChangeRepo, zoneRepo, backendResolver) val batchChangeHandler = BatchChangeHandler(batchChangeRepo, notifiers) @@ -229,7 +226,7 @@ object CommandHandler { msgsPerPoll, pollingInterval, processingSignal, - connections + backendResolver ) .compile .drain diff --git a/modules/api/src/main/scala/vinyldns/api/domain/dns/DnsConnection.scala b/modules/api/src/main/scala/vinyldns/api/backend/dns/DnsBackend.scala similarity index 69% rename from modules/api/src/main/scala/vinyldns/api/domain/dns/DnsConnection.scala rename to modules/api/src/main/scala/vinyldns/api/backend/dns/DnsBackend.scala index 3a25cf8f3..7fd2fc70f 100644 --- a/modules/api/src/main/scala/vinyldns/api/domain/dns/DnsConnection.scala +++ b/modules/api/src/main/scala/vinyldns/api/backend/dns/DnsBackend.scala @@ -14,24 +14,28 @@ * limitations under the License. */ -package vinyldns.api.domain.dns +package vinyldns.api.backend.dns + +import java.net.SocketAddress import cats.effect._ import cats.syntax.all._ import org.slf4j.{Logger, LoggerFactory} import org.xbill.DNS -import vinyldns.api.Interfaces.{result, _} -import vinyldns.api.crypto.Crypto +import vinyldns.api.domain.zone.ZoneTooLargeError +import vinyldns.core.crypto.CryptoAlgebra +import vinyldns.core.domain.backend.{Backend, BackendResponse} import vinyldns.core.domain.record.RecordType.RecordType -import vinyldns.core.domain.record.{RecordSet, RecordSetChange, RecordSetChangeType} +import vinyldns.core.domain.record.{RecordSet, RecordSetChange, RecordSetChangeType, RecordType} import vinyldns.core.domain.zone.{Zone, ZoneConnection} +import scala.collection.JavaConverters._ + object DnsProtocol { sealed trait DnsRequest final case class Apply(change: RecordSetChange) extends DnsRequest - // TODO: Remove origin once we change to using Zone Activation case class Resolve(name: String, zone: Zone, typ: RecordType) case class UpdateConnection(zoneConnection: ZoneConnection) @@ -84,25 +88,70 @@ class DnsQuery(val lookup: DNS.Lookup, val zoneName: DNS.Name) { def error: String = lookup.getErrorString } -class DnsConnection(val resolver: DNS.SimpleResolver) extends DnsConversions { +final case class TransferInfo(address: SocketAddress, tsig: DNS.TSIG) + +class DnsBackend(val id: String, val resolver: DNS.SimpleResolver, val xfrInfo: TransferInfo) + extends Backend + with DnsConversions { import DnsProtocol._ - val logger: Logger = LoggerFactory.getLogger(classOf[DnsConnection]) + val logger: Logger = LoggerFactory.getLogger(classOf[DnsBackend]) - def applyChange(change: RecordSetChange): Result[DnsResponse] = change.changeType match { - case RecordSetChangeType.Create => addRecord(change) - case RecordSetChangeType.Update => updateRecord(change) - case RecordSetChangeType.Delete => deleteRecord(change) + def applyChange(change: RecordSetChange): IO[BackendResponse] = { + change.changeType match { + case RecordSetChangeType.Create => addRecord(change) + case RecordSetChangeType.Update => updateRecord(change) + case RecordSetChangeType.Delete => deleteRecord(change) + } + }.attempt.flatMap { + case Left(DnsProtocol.Refused(msg)) => IO(BackendResponse.Retry(msg)) + case Right(DnsProtocol.NoError(msg)) => IO(BackendResponse.NoError(msg.toString)) + case Left(otherFailure) => IO.raiseError(otherFailure) } - def resolve(name: String, zoneName: String, typ: RecordType): Result[List[RecordSet]] = - IO { + def resolve(name: String, zoneName: String, typ: RecordType): IO[List[RecordSet]] = + IO.fromEither { for { query <- toQuery(name, zoneName, typ) records <- runQuery(query) } yield records - }.toResult + } + + def loadZone(zone: Zone, maxZoneSize: Int): IO[List[RecordSet]] = { + val dnsZoneName = zoneDnsName(zone.name) + val zti = DNS.ZoneTransferIn.newAXFR(dnsZoneName, xfrInfo.address, xfrInfo.tsig) + + for { + zoneXfr <- IO { + zti.run() + zti.getAXFR.asScala.map(_.asInstanceOf[DNS.Record]).toList.distinct + } + rawDnsRecords = zoneXfr.filter( + record => fromDnsRecordType(record.getType) != RecordType.UNKNOWN + ) + _ <- if (rawDnsRecords.length > maxZoneSize) { + IO.raiseError( + ZoneTooLargeError( + s"Zone too large ${zone.name}, ${rawDnsRecords.length} records exceeded max $maxZoneSize" + ) + ) + } else { + IO.pure(Unit) + } + dnsZoneName <- IO(zoneDnsName(zone.name)) + recordSets <- IO(rawDnsRecords.map(toRecordSet(_, dnsZoneName, zone.id))) + } yield recordSets + } + + /** + * Indicates if the zone is present in the backend + * + * @param zone The zone to check if exists + * @return true if it exists; false otherwise + */ + def zoneExists(zone: Zone): IO[Boolean] = + resolve(zone.name, zone.name, RecordType.SOA).map(_.nonEmpty) private[dns] def toQuery( name: String, @@ -125,7 +174,7 @@ class DnsConnection(val resolver: DNS.SimpleResolver) extends DnsConversions { case _ => Right(change) } - private[dns] def addRecord(change: RecordSetChange): Result[DnsResponse] = result { + private[dns] def addRecord(change: RecordSetChange): IO[DnsResponse] = IO.fromEither { for { change <- recordsArePresent(change) addRecord <- toDnsRRset(change.recordSet, change.zone.name) @@ -134,7 +183,7 @@ class DnsConnection(val resolver: DNS.SimpleResolver) extends DnsConversions { } yield response } - private[dns] def updateRecord(change: RecordSetChange): Result[DnsResponse] = result { + private[dns] def updateRecord(change: RecordSetChange): IO[DnsResponse] = IO.fromEither { for { change <- recordsArePresent(change) dnsRecord <- toDnsRRset(change.recordSet, change.zone.name) @@ -144,7 +193,7 @@ class DnsConnection(val resolver: DNS.SimpleResolver) extends DnsConversions { } yield response } - private[dns] def deleteRecord(change: RecordSetChange): Result[DnsResponse] = result { + private[dns] def deleteRecord(change: RecordSetChange): IO[DnsResponse] = IO.fromEither { for { change <- recordsArePresent(change) dnsRecord <- toDnsRRset(change.recordSet, change.zone.name) @@ -198,21 +247,41 @@ class DnsConnection(val resolver: DNS.SimpleResolver) extends DnsConversions { } } -object DnsConnection { +object DnsBackend { - def apply(conn: ZoneConnection): DnsConnection = new DnsConnection(createResolver(conn)) + def apply( + id: String, + conn: ZoneConnection, + xfrConn: Option[ZoneConnection], + crypto: CryptoAlgebra + ): DnsBackend = { + val tsig = createTsig(conn, crypto) + val resolver = createResolver(conn, tsig) - def createResolver(conn: ZoneConnection): DNS.SimpleResolver = { - // IMPORTANT! Make sure we decrypt the zone connection before creating the resolver - val decryptedConnection = conn.decrypted(Crypto.instance) - val (host, port) = parseHostAndPort(decryptedConnection.primaryServer) + val xfrInfo = xfrConn + .map { xc => + val xt = createTsig(xc, crypto) + val xr = createResolver(xc, xt) + TransferInfo(xr.getAddress, xt) + } + .getOrElse(TransferInfo(resolver.getAddress, tsig)) + new DnsBackend(id, resolver, xfrInfo) + } + + def createResolver(conn: ZoneConnection, tsig: DNS.TSIG): DNS.SimpleResolver = { + val (host, port) = parseHostAndPort(conn.primaryServer) val resolver = new DNS.SimpleResolver(host) resolver.setPort(port) - resolver.setTSIGKey(new DNS.TSIG(decryptedConnection.keyName, decryptedConnection.key)) + resolver.setTSIGKey(tsig) resolver } + def createTsig(conn: ZoneConnection, crypto: CryptoAlgebra): DNS.TSIG = { + val decryptedConnection = conn.decrypted(crypto) + new DNS.TSIG(decryptedConnection.keyName, decryptedConnection.key) + } + def parseHostAndPort(primaryServer: String): (String, Int) = { val parts = primaryServer.trim().split(':') if (parts.length < 2) diff --git a/modules/api/src/main/scala/vinyldns/api/backend/dns/DnsBackendConfig.scala b/modules/api/src/main/scala/vinyldns/api/backend/dns/DnsBackendConfig.scala new file mode 100644 index 000000000..ea220751d --- /dev/null +++ b/modules/api/src/main/scala/vinyldns/api/backend/dns/DnsBackendConfig.scala @@ -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.backend.dns + +import vinyldns.core.crypto.CryptoAlgebra +import vinyldns.core.domain.zone.ZoneConnection + +final case class DnsBackendConfig( + id: String, + zoneConnection: ZoneConnection, + transferConnection: Option[ZoneConnection] +) { + def toDnsConnection(crypto: CryptoAlgebra): DnsBackend = + DnsBackend.apply(id, zoneConnection, transferConnection, crypto) +} diff --git a/modules/api/src/main/scala/vinyldns/api/backend/dns/DnsBackendProvider.scala b/modules/api/src/main/scala/vinyldns/api/backend/dns/DnsBackendProvider.scala new file mode 100644 index 000000000..b61cf2c73 --- /dev/null +++ b/modules/api/src/main/scala/vinyldns/api/backend/dns/DnsBackendProvider.scala @@ -0,0 +1,59 @@ +/* + * 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.backend.dns + +import vinyldns.core.crypto.CryptoAlgebra +import vinyldns.core.domain.backend.{Backend, BackendProvider} +import vinyldns.core.domain.zone.Zone + +class DnsBackendProvider(connections: List[DnsBackend], crypto: CryptoAlgebra) + extends BackendProvider { + + private val connMap: Map[String, DnsBackend] = + connections.map { c => + c.id -> c + }.toMap + + /** + * Given a zone, returns a connection to the zone, returns None if cannot connect + * + * @param zone The zone to attempt to connect to + * @return A backend that is usable, or None if it could not connect + */ + def connect(zone: Zone): Option[Backend] = + // Use the connection info on the zone if present + zone.connection + .map { conn => + DnsBackend.apply("unknown", conn, zone.transferConnection, crypto) + } + .orElse { + zone.backendId.flatMap(connectById) + } + + /** + * Given a backend id, looks up the backend for this provider if it exists + * + * @return A backend that is usable, or None if could not connect + */ + def connectById(backendId: String): Option[Backend] = + connMap.get(backendId) + + /** + * @return The backend ids loaded with this provider + */ + def ids: List[String] = connMap.keys.toList +} diff --git a/modules/api/src/main/scala/vinyldns/api/backend/dns/DnsBackendProviderConfig.scala b/modules/api/src/main/scala/vinyldns/api/backend/dns/DnsBackendProviderConfig.scala new file mode 100644 index 000000000..0b9b03824 --- /dev/null +++ b/modules/api/src/main/scala/vinyldns/api/backend/dns/DnsBackendProviderConfig.scala @@ -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.backend.dns + +import cats.effect.{Blocker, ContextShift, IO} +import com.typesafe.config.Config +import pureconfig.ConfigSource +import pureconfig.generic.auto._ +import pureconfig.module.catseffect.syntax.CatsEffectConfigSource + +final case class DnsBackendProviderConfig(legacy: Boolean, backends: List[DnsBackendConfig]) + +object DnsBackendProviderConfig { + def load(config: Config)(implicit cs: ContextShift[IO]): IO[DnsBackendProviderConfig] = + Blocker[IO].use( + ConfigSource.fromConfig(config).loadF[IO, DnsBackendProviderConfig](_) + ) +} diff --git a/modules/api/src/main/scala/vinyldns/api/backend/dns/DnsBackendProviderLoader.scala b/modules/api/src/main/scala/vinyldns/api/backend/dns/DnsBackendProviderLoader.scala new file mode 100644 index 000000000..60002392c --- /dev/null +++ b/modules/api/src/main/scala/vinyldns/api/backend/dns/DnsBackendProviderLoader.scala @@ -0,0 +1,63 @@ +/* + * 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.backend.dns + +import cats.effect.{ContextShift, IO} +import vinyldns.api.VinylDNSConfig +import vinyldns.api.crypto.Crypto +import vinyldns.core.domain.backend.{BackendProvider, BackendProviderConfig, BackendProviderLoader} + +class DnsBackendProviderLoader extends BackendProviderLoader { + + private implicit val cs: ContextShift[IO] = + IO.contextShift(scala.concurrent.ExecutionContext.global) + + /** + * Loads a backend based on the provided config so that it is ready to use + * This is internally used typically during startup + * + * @param config The BackendConfig, has settings that are specific to this backend + * @return A ready-to-use Backend instance, or does an IO.raiseError if something bad occurred. + */ + def load(config: BackendProviderConfig): IO[BackendProvider] = + // if legacy = true, load from the old configured dns connections + // otherwise, load new stuff + DnsBackendProviderConfig.load(config.settings).map { bec => + if (bec.legacy) { + // legacy adds a backend id named "default" with the default configuration + // and loads the backend connections from the legacy YAML config + val conns = VinylDNSConfig.configuredDnsConnections.dnsBackends.map { be => + DnsBackend + .apply(be.id, be.zoneConnection, Some(be.transferConnection), Crypto.instance) + } + val defaultConn = + DnsBackend.apply( + "default", + VinylDNSConfig.configuredDnsConnections.defaultZoneConnection, + Some(VinylDNSConfig.configuredDnsConnections.defaultTransferConnection), + Crypto.instance + ) + new DnsBackendProvider(defaultConn :: conns, Crypto.instance) + } else { + // Assumes the "new" YAML config + new DnsBackendProvider( + bec.backends.map(_.toDnsConnection(Crypto.instance)), + Crypto.instance + ) + } + } +} diff --git a/modules/api/src/main/scala/vinyldns/api/domain/dns/DnsConversions.scala b/modules/api/src/main/scala/vinyldns/api/backend/dns/DnsConversions.scala similarity index 99% rename from modules/api/src/main/scala/vinyldns/api/domain/dns/DnsConversions.scala rename to modules/api/src/main/scala/vinyldns/api/backend/dns/DnsConversions.scala index 3a6182446..efe9647ac 100644 --- a/modules/api/src/main/scala/vinyldns/api/domain/dns/DnsConversions.scala +++ b/modules/api/src/main/scala/vinyldns/api/backend/dns/DnsConversions.scala @@ -14,7 +14,7 @@ * limitations under the License. */ -package vinyldns.api.domain.dns +package vinyldns.api.backend.dns import java.net.InetAddress @@ -22,10 +22,10 @@ import cats.syntax.either._ import org.joda.time.DateTime import org.xbill.DNS import scodec.bits.ByteVector -import vinyldns.api.domain.dns.DnsProtocol._ -import vinyldns.core.domain.{DomainHelpers, Fqdn, record} +import vinyldns.api.backend.dns.DnsProtocol._ import vinyldns.core.domain.record.RecordType._ import vinyldns.core.domain.record._ +import vinyldns.core.domain.{DomainHelpers, Fqdn, record} import scala.collection.JavaConverters._ import scala.util.Try diff --git a/modules/api/src/main/scala/vinyldns/api/domain/ReverseZoneHelpers.scala b/modules/api/src/main/scala/vinyldns/api/domain/ReverseZoneHelpers.scala index 101bac316..a8e9050eb 100644 --- a/modules/api/src/main/scala/vinyldns/api/domain/ReverseZoneHelpers.scala +++ b/modules/api/src/main/scala/vinyldns/api/domain/ReverseZoneHelpers.scala @@ -20,7 +20,7 @@ import cats.implicits._ import com.aaronbedra.orchard.CIDR import vinyldns.api.domain.zone.InvalidRequest import vinyldns.core.domain.zone.Zone -import vinyldns.api.domain.dns.DnsConversions._ +import vinyldns.api.backend.dns.DnsConversions._ import scala.util.Try diff --git a/modules/api/src/main/scala/vinyldns/api/domain/batch/BatchChangeService.scala b/modules/api/src/main/scala/vinyldns/api/domain/batch/BatchChangeService.scala index dfb7ff43d..2341f83bb 100644 --- a/modules/api/src/main/scala/vinyldns/api/domain/batch/BatchChangeService.scala +++ b/modules/api/src/main/scala/vinyldns/api/domain/batch/BatchChangeService.scala @@ -26,7 +26,7 @@ import vinyldns.api.domain.DomainValidations._ import vinyldns.api.domain.auth.AuthPrincipalProvider import vinyldns.api.domain.batch.BatchChangeInterfaces._ import vinyldns.api.domain.batch.BatchTransformations._ -import vinyldns.api.domain.dns.DnsConversions._ +import vinyldns.api.backend.dns.DnsConversions._ import vinyldns.api.repository.ApiDataAccessor import vinyldns.core.domain.auth.AuthPrincipal import vinyldns.core.domain.batch.BatchChangeApprovalStatus.BatchChangeApprovalStatus diff --git a/modules/api/src/main/scala/vinyldns/api/domain/batch/BatchTransformations.scala b/modules/api/src/main/scala/vinyldns/api/domain/batch/BatchTransformations.scala index b5fa5edf3..25600bda8 100644 --- a/modules/api/src/main/scala/vinyldns/api/domain/batch/BatchTransformations.scala +++ b/modules/api/src/main/scala/vinyldns/api/domain/batch/BatchTransformations.scala @@ -23,7 +23,7 @@ import vinyldns.api.VinylDNSConfig import vinyldns.api.domain.ReverseZoneHelpers import vinyldns.api.domain.batch.BatchChangeInterfaces.ValidatedBatch import vinyldns.api.domain.batch.BatchTransformations.LogicalChangeType.LogicalChangeType -import vinyldns.api.domain.dns.DnsConversions.getIPv6FullReverseName +import vinyldns.api.backend.dns.DnsConversions.getIPv6FullReverseName import vinyldns.core.domain.batch._ import vinyldns.core.domain.record.{AAAAData, RecordData, RecordSet, RecordSetChange} import vinyldns.core.domain.record.RecordType._ diff --git a/modules/api/src/main/scala/vinyldns/api/domain/record/RecordSetChangeGenerator.scala b/modules/api/src/main/scala/vinyldns/api/domain/record/RecordSetChangeGenerator.scala index e94615f03..864f7c2d0 100644 --- a/modules/api/src/main/scala/vinyldns/api/domain/record/RecordSetChangeGenerator.scala +++ b/modules/api/src/main/scala/vinyldns/api/domain/record/RecordSetChangeGenerator.scala @@ -19,8 +19,8 @@ package vinyldns.api.domain.record import java.util.UUID import org.joda.time.DateTime +import vinyldns.api.backend.dns.DnsConversions import vinyldns.core.domain.auth.AuthPrincipal -import vinyldns.api.domain.dns.DnsConversions import vinyldns.core.domain.zone.Zone import vinyldns.core.domain.record._ diff --git a/modules/api/src/main/scala/vinyldns/api/domain/record/RecordSetHelpers.scala b/modules/api/src/main/scala/vinyldns/api/domain/record/RecordSetHelpers.scala index 22102cc8e..88f0b2901 100644 --- a/modules/api/src/main/scala/vinyldns/api/domain/record/RecordSetHelpers.scala +++ b/modules/api/src/main/scala/vinyldns/api/domain/record/RecordSetHelpers.scala @@ -16,7 +16,7 @@ package vinyldns.api.domain.record -import vinyldns.api.domain.dns.DnsConversions +import vinyldns.api.backend.dns.DnsConversions import vinyldns.core.domain.record.RecordSet object RecordSetHelpers { diff --git a/modules/api/src/main/scala/vinyldns/api/domain/record/RecordSetService.scala b/modules/api/src/main/scala/vinyldns/api/domain/record/RecordSetService.scala index f5e1b2891..e6d156ade 100644 --- a/modules/api/src/main/scala/vinyldns/api/domain/record/RecordSetService.scala +++ b/modules/api/src/main/scala/vinyldns/api/domain/record/RecordSetService.scala @@ -23,25 +23,24 @@ import vinyldns.api.domain.zone._ import vinyldns.api.repository.ApiDataAccessor import vinyldns.api.route.{ListGlobalRecordSetsResponse, ListRecordSetsByZoneResponse} import vinyldns.core.domain.record._ -import vinyldns.core.domain.zone.{ConfiguredDnsConnections, Zone, ZoneCommandResult, ZoneRepository} +import vinyldns.core.domain.zone.{Zone, ZoneCommandResult, ZoneRepository} import vinyldns.core.queue.MessageQueue import cats.data._ import cats.effect.IO import org.xbill.DNS.ReverseMap import vinyldns.api.domain.DomainValidations.{validateIpv4Address, validateIpv6Address} import vinyldns.api.domain.access.AccessValidationsAlgebra -import vinyldns.api.domain.dns.DnsConnection import vinyldns.core.domain.record.NameSort.NameSort import vinyldns.core.domain.record.RecordType.RecordType import vinyldns.core.domain.DomainHelpers.ensureTrailingDot +import vinyldns.core.domain.backend.{Backend, BackendResolver} object RecordSetService { def apply( dataAccessor: ApiDataAccessor, messageQueue: MessageQueue, accessValidation: AccessValidationsAlgebra, - dnsConnection: (Zone, ConfiguredDnsConnections) => DnsConnection, - configuredDnsConnections: ConfiguredDnsConnections, + backendResolver: BackendResolver, validateRecordLookupAgainstDnsBackend: Boolean ): RecordSetService = new RecordSetService( @@ -52,8 +51,7 @@ object RecordSetService { dataAccessor.userRepository, messageQueue, accessValidation, - dnsConnection, - configuredDnsConnections, + backendResolver, validateRecordLookupAgainstDnsBackend ) } @@ -66,8 +64,7 @@ class RecordSetService( userRepository: UserRepository, messageQueue: MessageQueue, accessValidation: AccessValidationsAlgebra, - dnsConnection: (Zone, ConfiguredDnsConnections) => DnsConnection, - configuredDnsConnections: ConfiguredDnsConnections, + backendResolver: BackendResolver, validateRecordLookupAgainstDnsBackend: Boolean ) extends RecordSetServiceAlgebra { @@ -82,8 +79,7 @@ class RecordSetService( rsForValidations = change.recordSet _ <- isNotHighValueDomain(recordSet, zone).toResult _ <- recordSetDoesNotExist( - dnsConnection, - configuredDnsConnections, + backendResolver.resolve, zone, rsForValidations, validateRecordLookupAgainstDnsBackend @@ -120,8 +116,7 @@ class RecordSetService( .getRecordSetsByName(zone.id, rsForValidations.name) .toResult[List[RecordSet]] _ <- isUniqueUpdate( - dnsConnection, - configuredDnsConnections, + backendResolver.resolve, rsForValidations, existingRecordsWithName, zone, @@ -375,8 +370,7 @@ class RecordSetService( } def recordSetDoesNotExist( - dnsConnection: (Zone, ConfiguredDnsConnections) => DnsConnection, - configuredDnsConnections: ConfiguredDnsConnections, + backendConnection: Zone => Backend, zone: Zone, recordSet: RecordSet, validateRecordLookupAgainstDnsBackend: Boolean @@ -384,9 +378,9 @@ class RecordSetService( recordSetDoesNotExistInDatabase(recordSet, zone).value.flatMap { case Left(recordSetAlreadyExists: RecordSetAlreadyExists) if validateRecordLookupAgainstDnsBackend => - dnsConnection(zone, configuredDnsConnections) + backendConnection(zone) .resolve(recordSet.name, zone.name, recordSet.typ) - .value + .attempt .map { case Right(existingRecords) => if (existingRecords.isEmpty) Right(()) @@ -397,8 +391,7 @@ class RecordSetService( }.toResult def isUniqueUpdate( - dnsConnection: (Zone, ConfiguredDnsConnections) => DnsConnection, - configuredDnsConnections: ConfiguredDnsConnections, + backendConnection: Zone => Backend, newRecordSet: RecordSet, existingRecordsWithName: List[RecordSet], zone: Zone, @@ -408,9 +401,9 @@ class RecordSetService( .recordSetDoesNotExist(newRecordSet, existingRecordsWithName, zone) match { case Left(recordSetAlreadyExists: RecordSetAlreadyExists) if validateRecordLookupAgainstDnsBackend => - dnsConnection(zone, configuredDnsConnections) + backendConnection(zone) .resolve(newRecordSet.name, zone.name, newRecordSet.typ) - .value + .attempt .map { case Right(existingRecords) => if (existingRecords.isEmpty) Right(()) diff --git a/modules/api/src/main/scala/vinyldns/api/domain/record/RecordSetValidations.scala b/modules/api/src/main/scala/vinyldns/api/domain/record/RecordSetValidations.scala index ecd52cdc0..0d0ebf3b4 100644 --- a/modules/api/src/main/scala/vinyldns/api/domain/record/RecordSetValidations.scala +++ b/modules/api/src/main/scala/vinyldns/api/domain/record/RecordSetValidations.scala @@ -19,8 +19,8 @@ package vinyldns.api.domain.record import cats.syntax.either._ import vinyldns.api.Interfaces._ import vinyldns.api.VinylDNSConfig +import vinyldns.api.backend.dns.DnsConversions import vinyldns.api.domain._ -import vinyldns.api.domain.dns.DnsConversions import vinyldns.core.domain.DomainHelpers._ import vinyldns.core.domain.record.RecordType._ import vinyldns.api.domain.zone._ @@ -28,6 +28,7 @@ import vinyldns.core.domain.auth.AuthPrincipal import vinyldns.core.domain.membership.Group import vinyldns.core.domain.record.{RecordSet, RecordType} import vinyldns.core.domain.zone.Zone + import scala.util.matching.Regex object RecordSetValidations { diff --git a/modules/api/src/main/scala/vinyldns/api/domain/zone/ZoneConnectionValidator.scala b/modules/api/src/main/scala/vinyldns/api/domain/zone/ZoneConnectionValidator.scala index d5c49c643..52c2ea2ff 100644 --- a/modules/api/src/main/scala/vinyldns/api/domain/zone/ZoneConnectionValidator.scala +++ b/modules/api/src/main/scala/vinyldns/api/domain/zone/ZoneConnectionValidator.scala @@ -16,76 +16,31 @@ package vinyldns.api.domain.zone -import java.net.{InetSocketAddress, Socket} - import cats.effect._ import cats.syntax.all._ -import org.slf4j.{Logger, LoggerFactory} import vinyldns.api.Interfaces._ import vinyldns.api.VinylDNSConfig -import vinyldns.api.domain.dns.DnsConnection -import vinyldns.core.domain.record.{RecordSet, RecordType} -import vinyldns.core.domain.zone.{ConfiguredDnsConnections, DnsBackend, Zone, ZoneConnection} -import vinyldns.core.health.HealthCheck._ +import vinyldns.core.domain.backend.{Backend, BackendResolver} +import vinyldns.core.domain.record.RecordType +import vinyldns.core.domain.zone.Zone import scala.concurrent.duration._ trait ZoneConnectionValidatorAlgebra { - def validateZoneConnections(zone: Zone): Result[Unit] def isValidBackendId(backendId: Option[String]): Either[Throwable, Unit] - } -object ZoneConnectionValidator { - - val logger: Logger = LoggerFactory.getLogger(classOf[ZoneConnectionValidator]) - - def getZoneConnection( - zone: Zone, - configuredDnsConnections: ConfiguredDnsConnections - ): ZoneConnection = - zone.connection - .orElse(getDnsBackend(zone, configuredDnsConnections).map(_.zoneConnection)) - .getOrElse(configuredDnsConnections.defaultZoneConnection) - - def getTransferConnection( - zone: Zone, - configuredDnsConnections: ConfiguredDnsConnections - ): ZoneConnection = - zone.transferConnection - .orElse(getDnsBackend(zone, configuredDnsConnections).map(_.transferConnection)) - .getOrElse(configuredDnsConnections.defaultTransferConnection) - - def getDnsBackend( - zone: Zone, - configuredDnsConnections: ConfiguredDnsConnections - ): Option[DnsBackend] = - zone.backendId - .flatMap { bid => - val backend = configuredDnsConnections.dnsBackends.find(_.id == bid) - if (backend.isEmpty) { - logger.error( - s"BackendId [$bid] for zone [${zone.id}: ${zone.name}] is not defined in config" - ) - } - backend - } -} - -class ZoneConnectionValidator(connections: ConfiguredDnsConnections) +class ZoneConnectionValidator(backendResolver: BackendResolver) extends ZoneConnectionValidatorAlgebra { - import ZoneConnectionValidator._ import ZoneRecordValidations._ // Takes a long time to load large zones val opTimeout: FiniteDuration = 60.seconds - val (healthCheckAddress, healthCheckPort) = - DnsConnection.parseHostAndPort(connections.defaultZoneConnection.primaryServer) - - def loadDns(zone: Zone): IO[ZoneView] = DnsZoneViewLoader(zone).load() + def loadDns(zone: Zone): IO[ZoneView] = + DnsZoneViewLoader(zone, backendResolver.resolve(zone)).load() def hasApexNS(zoneView: ZoneView): Result[Unit] = { val apexRecord = zoneView.recordSetsMap.get(zoneView.zone.name, RecordType.NS) match { @@ -107,8 +62,8 @@ class ZoneConnectionValidator(connections: ConfiguredDnsConnections) .toResult } - def getDnsConnection(zone: Zone): Result[DnsConnection] = - Either.catchNonFatal(dnsConnection(getZoneConnection(zone, connections))).toResult + def getBackendConnection(zone: Zone): Result[Backend] = + backendResolver.resolve(zone).toResult def loadZone(zone: Zone): Result[ZoneView] = withTimeout( @@ -117,22 +72,22 @@ class ZoneConnectionValidator(connections: ConfiguredDnsConnections) ConnectionFailed(zone, "Unable to connect to zone: Transfer connection invalid") ) - def hasSOA(records: List[RecordSet], zone: Zone): Result[Unit] = { - if (records.isEmpty) { - ConnectionFailed(zone, "SOA Record for zone not found").asLeft[Unit] - } else { - ().asRight[Throwable] - } - }.toResult + def zoneExists(zone: Zone, backend: Backend): Result[Unit] = + backend + .zoneExists(zone) + .ifM( + IO(Right(())), + IO(Left(ConnectionFailed(zone, s"Unable to find zone ${zone.name} in backend ${backend.id}"))) + ) + .toResult def validateZoneConnections(zone: Zone): Result[Unit] = { val result = for { - connection <- getDnsConnection(zone) - resp <- connection.resolve(zone.name, zone.name, RecordType.SOA) + connection <- getBackendConnection(zone) + _ <- zoneExists(zone, connection) view <- loadZone(zone) _ <- hasApexNS(view) - _ <- hasSOA(resp, zone) } yield () result.leftMap { @@ -142,20 +97,8 @@ class ZoneConnectionValidator(connections: ConfiguredDnsConnections) } } - def healthCheck(timeout: Int): HealthCheck = - Resource - .fromAutoCloseable(IO(new Socket())) - .use( - socket => - IO(socket.connect(new InetSocketAddress(healthCheckAddress, healthCheckPort), timeout)) - ) - .attempt - .asHealthCheck(classOf[ZoneConnectionValidator]) - def isValidBackendId(backendId: Option[String]): Either[Throwable, Unit] = ensuring(InvalidRequest(s"Invalid backendId: [$backendId]; please check system configuration")) { - backendId.forall(id => connections.dnsBackends.exists(_.id == id)) + backendId.forall(id => backendResolver.isRegistered(id)) } - - private[domain] def dnsConnection(conn: ZoneConnection): DnsConnection = DnsConnection(conn) } diff --git a/modules/api/src/main/scala/vinyldns/api/domain/zone/ZoneService.scala b/modules/api/src/main/scala/vinyldns/api/domain/zone/ZoneService.scala index 1272fdb30..36956b3e5 100644 --- a/modules/api/src/main/scala/vinyldns/api/domain/zone/ZoneService.scala +++ b/modules/api/src/main/scala/vinyldns/api/domain/zone/ZoneService.scala @@ -18,13 +18,14 @@ package vinyldns.api.domain.zone import cats.implicits._ import vinyldns.api.domain.access.AccessValidationsAlgebra -import vinyldns.api.{Interfaces, VinylDNSConfig} +import vinyldns.api.Interfaces import vinyldns.core.domain.auth.AuthPrincipal import vinyldns.api.repository.ApiDataAccessor import vinyldns.core.domain.membership.{Group, GroupRepository, User, UserRepository} import vinyldns.core.domain.zone._ import vinyldns.core.queue.MessageQueue import vinyldns.core.domain.DomainHelpers.ensureTrailingDot +import vinyldns.core.domain.backend.BackendResolver object ZoneService { def apply( @@ -32,7 +33,8 @@ object ZoneService { connectionValidator: ZoneConnectionValidatorAlgebra, messageQueue: MessageQueue, zoneValidations: ZoneValidations, - accessValidation: AccessValidationsAlgebra + accessValidation: AccessValidationsAlgebra, + backendResolver: BackendResolver ): ZoneService = new ZoneService( dataAccessor.zoneRepository, @@ -42,7 +44,8 @@ object ZoneService { connectionValidator, messageQueue, zoneValidations, - accessValidation + accessValidation, + backendResolver ) } @@ -54,7 +57,8 @@ class ZoneService( connectionValidator: ZoneConnectionValidatorAlgebra, messageQueue: MessageQueue, zoneValidations: ZoneValidations, - accessValidation: AccessValidationsAlgebra + accessValidation: AccessValidationsAlgebra, + backendResolver: BackendResolver ) extends ZoneServiceAlgebra { import accessValidation._ @@ -233,7 +237,7 @@ class ZoneService( } def getBackendIds(): Result[List[String]] = - VinylDNSConfig.configuredDnsConnections.dnsBackends.map(_.id).toResult + backendResolver.ids.toList.toResult def zoneDoesNotExist(zoneName: String): Result[Unit] = zoneRepository diff --git a/modules/api/src/main/scala/vinyldns/api/domain/zone/ZoneView.scala b/modules/api/src/main/scala/vinyldns/api/domain/zone/ZoneView.scala index d8ab0c0b3..3c6531952 100644 --- a/modules/api/src/main/scala/vinyldns/api/domain/zone/ZoneView.scala +++ b/modules/api/src/main/scala/vinyldns/api/domain/zone/ZoneView.scala @@ -16,7 +16,7 @@ package vinyldns.api.domain.zone -import vinyldns.api.domain.dns.DnsConversions._ +import vinyldns.api.backend.dns.DnsConversions._ import vinyldns.api.domain import vinyldns.api.domain.record.RecordSetChangeGenerator import vinyldns.core.domain.record.{RecordSet, RecordSetChange} diff --git a/modules/api/src/main/scala/vinyldns/api/domain/zone/ZoneViewLoader.scala b/modules/api/src/main/scala/vinyldns/api/domain/zone/ZoneViewLoader.scala index 6b52ba424..66a2307c0 100644 --- a/modules/api/src/main/scala/vinyldns/api/domain/zone/ZoneViewLoader.scala +++ b/modules/api/src/main/scala/vinyldns/api/domain/zone/ZoneViewLoader.scala @@ -18,49 +18,24 @@ package vinyldns.api.domain.zone import cats.effect._ import org.slf4j.LoggerFactory -import org.xbill.DNS -import org.xbill.DNS.{TSIG, ZoneTransferIn} import vinyldns.api.VinylDNSConfig -import vinyldns.api.crypto.Crypto -import vinyldns.api.domain.dns.DnsConversions -import vinyldns.core.route.Monitored - -import scala.collection.JavaConverters._ -import vinyldns.core.domain.record.{NameSort, RecordSetRepository, RecordType} +import vinyldns.api.backend.dns.DnsConversions +import vinyldns.core.domain.backend.Backend +import vinyldns.core.domain.record.{NameSort, RecordSetRepository} import vinyldns.core.domain.zone.Zone +import vinyldns.core.route.Monitored trait ZoneViewLoader { def load: () => IO[ZoneView] } object DnsZoneViewLoader extends DnsConversions { - val logger = LoggerFactory.getLogger("DnsZoneViewLoader") - - def dnsZoneTransfer(zone: Zone): ZoneTransferIn = { - val conn = - ZoneConnectionValidator - .getTransferConnection(zone, VinylDNSConfig.configuredDnsConnections) - .decrypted(Crypto.instance) - val TSIGKey = new TSIG(conn.keyName, conn.key) - val parts = conn.primaryServer.trim().split(':') - val (hostName, port) = - if (parts.length < 2) - (conn.primaryServer, 53) - else - (parts(0), parts(1).toInt) - - val dnsZoneName = zoneDnsName(zone.name) - ZoneTransferIn.newAXFR(dnsZoneName, hostName, port, TSIGKey) - } - - def apply(zone: Zone): DnsZoneViewLoader = - DnsZoneViewLoader(zone, dnsZoneTransfer) } case class DnsZoneViewLoader( zone: Zone, - zoneTransfer: Zone => ZoneTransferIn, + backendConnection: Backend, maxZoneSize: Int = VinylDNSConfig.maxZoneSize ) extends ZoneViewLoader with DnsConversions @@ -70,24 +45,7 @@ case class DnsZoneViewLoader( () => monitor("dns.loadZoneView") { for { - zoneXfr <- IO { - val xfr = zoneTransfer(zone) - xfr.run() - xfr.getAXFR.asScala.map(_.asInstanceOf[DNS.Record]).toList.distinct - } - rawDnsRecords = zoneXfr.filter( - record => fromDnsRecordType(record.getType) != RecordType.UNKNOWN - ) - _ <- if (rawDnsRecords.length > maxZoneSize) - IO.raiseError(ZoneTooLargeError(zone, rawDnsRecords.length, maxZoneSize)) - else IO.pure(Unit) - dnsZoneName <- IO(zoneDnsName(zone.name)) - recordSets <- IO(rawDnsRecords.map(toRecordSet(_, dnsZoneName, zone.id))) - _ <- IO( - DnsZoneViewLoader.logger.info( - s"dns.loadDnsView zoneName=${zone.name}; rawRsCount=${zoneXfr.size}; rsCount=${recordSets.size}" - ) - ) + recordSets <- backendConnection.loadZone(zone, maxZoneSize) } yield ZoneView(zone, recordSets) } } diff --git a/modules/api/src/main/scala/vinyldns/api/engine/RecordSetChangeHandler.scala b/modules/api/src/main/scala/vinyldns/api/engine/RecordSetChangeHandler.scala index 3b4fd5d11..88733e76b 100644 --- a/modules/api/src/main/scala/vinyldns/api/engine/RecordSetChangeHandler.scala +++ b/modules/api/src/main/scala/vinyldns/api/engine/RecordSetChangeHandler.scala @@ -19,10 +19,10 @@ package vinyldns.api.engine import cats.effect.{ContextShift, IO, Timer} import cats.implicits._ import org.slf4j.LoggerFactory -import vinyldns.api.domain.dns.DnsConnection -import vinyldns.api.domain.dns.DnsProtocol.{NoError, Refused, TryAgain} +import vinyldns.api.backend.dns.DnsProtocol.TryAgain import vinyldns.api.domain.record.RecordSetChangeGenerator import vinyldns.api.domain.record.RecordSetHelpers._ +import vinyldns.core.domain.backend.{Backend, BackendResponse} import vinyldns.core.domain.batch.{BatchChangeRepository, SingleChange} import vinyldns.core.domain.record._ import vinyldns.core.domain.zone.Zone @@ -39,7 +39,7 @@ object RecordSetChangeHandler { recordSetRepository: RecordSetRepository, recordChangeRepository: RecordChangeRepository, batchChangeRepository: BatchChangeRepository - )(implicit timer: Timer[IO]): (DnsConnection, RecordSetChange) => IO[RecordSetChange] = + )(implicit timer: Timer[IO]): (Backend, RecordSetChange) => IO[RecordSetChange] = (conn, recordSetChange) => { process( recordSetRepository, @@ -54,7 +54,7 @@ object RecordSetChangeHandler { recordSetRepository: RecordSetRepository, recordChangeRepository: RecordChangeRepository, batchChangeRepository: BatchChangeRepository, - conn: DnsConnection, + conn: Backend, recordSetChange: RecordSetChange )(implicit timer: Timer[IO]): IO[RecordSetChange] = for { @@ -124,7 +124,7 @@ object RecordSetChangeHandler { def syncAndGetProcessingStatusFromDnsBackend( change: RecordSetChange, - dnsConn: DnsConnection, + conn: Backend, recordSetRepository: RecordSetRepository, recordChangeRepository: RecordChangeRepository, performSync: Boolean = false @@ -167,7 +167,7 @@ object RecordSetChangeHandler { } } - dnsConn.resolve(change.recordSet.name, change.zone.name, change.recordSet.typ).value.flatMap { + conn.resolve(change.recordSet.name, change.zone.name, change.recordSet.typ).attempt.flatMap { case Right(existingRecords) => if (performSync) { for { @@ -189,7 +189,7 @@ object RecordSetChangeHandler { private def fsm( state: ProcessorState, - conn: DnsConnection, + conn: Backend, wildcardExists: Boolean, recordSetRepository: RecordSetRepository, recordChangeRepository: RecordChangeRepository @@ -310,13 +310,13 @@ object RecordSetChangeHandler { /* Step 1: Validate the change hasn't already been applied */ private def validate( change: RecordSetChange, - dnsConn: DnsConnection, + conn: Backend, recordSetRepository: RecordSetRepository, recordChangeRepository: RecordChangeRepository ): IO[ProcessorState] = syncAndGetProcessingStatusFromDnsBackend( change, - dnsConn, + conn, recordSetRepository, recordChangeRepository, true @@ -333,12 +333,12 @@ object RecordSetChangeHandler { } /* Step 2: Apply the change to the dns backend */ - private def apply(change: RecordSetChange, dnsConn: DnsConnection): IO[ProcessorState] = - dnsConn.applyChange(change).value.map { - case Right(_: NoError) => - Applied(change) - case Left(_: Refused) => + private def apply(change: RecordSetChange, conn: Backend): IO[ProcessorState] = + conn.applyChange(change).attempt.map { + case Right(BackendResponse.Retry(_)) => Retrying(change) + case Right(BackendResponse.NoError(_)) => + Applied(change) case Left(error) => Completed( change.failed( @@ -350,13 +350,13 @@ object RecordSetChangeHandler { /* Step 3: Verify the record was created. If the ProcessorState is applied or failed we requeue the record.*/ private def verify( change: RecordSetChange, - dnsConn: DnsConnection, + conn: Backend, recordSetRepository: RecordSetRepository, recordChangeRepository: RecordChangeRepository ): IO[ProcessorState] = syncAndGetProcessingStatusFromDnsBackend( change, - dnsConn, + conn, recordSetRepository, recordChangeRepository ).map { diff --git a/modules/api/src/main/scala/vinyldns/api/engine/ZoneSyncHandler.scala b/modules/api/src/main/scala/vinyldns/api/engine/ZoneSyncHandler.scala index fcd032b2e..8c94ceb75 100644 --- a/modules/api/src/main/scala/vinyldns/api/engine/ZoneSyncHandler.scala +++ b/modules/api/src/main/scala/vinyldns/api/engine/ZoneSyncHandler.scala @@ -20,8 +20,9 @@ import cats.effect.{ContextShift, IO} import cats.syntax.all._ import org.joda.time.DateTime import org.slf4j.{Logger, LoggerFactory} -import vinyldns.api.domain.dns.DnsConversions +import vinyldns.api.backend.dns.DnsConversions import vinyldns.api.domain.zone.{DnsZoneViewLoader, VinylDNSZoneViewLoader} +import vinyldns.core.domain.backend.BackendResolver import vinyldns.core.domain.record._ import vinyldns.core.domain.zone.{Zone, ZoneStatus} import vinyldns.core.route.Monitored @@ -43,7 +44,7 @@ object ZoneSyncHandler extends DnsConversions with Monitored { recordChangeRepository: RecordChangeRepository, zoneChangeRepository: ZoneChangeRepository, zoneRepository: ZoneRepository, - dnsLoader: Zone => DnsZoneViewLoader = DnsZoneViewLoader.apply, + backendResolver: BackendResolver, vinyldnsLoader: (Zone, RecordSetRepository) => VinylDNSZoneViewLoader = VinylDNSZoneViewLoader.apply ): ZoneChange => IO[ZoneChange] = @@ -55,7 +56,7 @@ object ZoneSyncHandler extends DnsConversions with Monitored { recordSetRepository, recordChangeRepository, zoneChange, - dnsLoader, + backendResolver, vinyldnsLoader ) _ <- saveZoneAndChange(zoneRepository, zoneChangeRepository, syncChange) // final save to store zone status @@ -83,18 +84,18 @@ object ZoneSyncHandler extends DnsConversions with Monitored { recordSetRepository: RecordSetRepository, recordChangeRepository: RecordChangeRepository, zoneChange: ZoneChange, - dnsLoader: Zone => DnsZoneViewLoader = DnsZoneViewLoader.apply, + backendResolver: BackendResolver, vinyldnsLoader: (Zone, RecordSetRepository) => VinylDNSZoneViewLoader = VinylDNSZoneViewLoader.apply ): IO[ZoneChange] = monitor("zone.sync") { time(s"zone.sync; zoneName='${zoneChange.zone.name}'") { val zone = zoneChange.zone - + val dnsLoader = DnsZoneViewLoader(zone, backendResolver.resolve(zone)) val dnsView = time( s"zone.sync.loadDnsView; zoneName='${zone.name}'; zoneChange='${zoneChange.id}'" - )(dnsLoader(zone).load()) + )(dnsLoader.load()) val vinyldnsView = time(s"zone.sync.loadVinylDNSView; zoneName='${zone.name}'")( vinyldnsLoader(zone, recordSetRepository).load() ) diff --git a/modules/api/src/main/scala/vinyldns/api/route/StatusRouting.scala b/modules/api/src/main/scala/vinyldns/api/route/StatusRouting.scala index 65d488453..36fe16612 100644 --- a/modules/api/src/main/scala/vinyldns/api/route/StatusRouting.scala +++ b/modules/api/src/main/scala/vinyldns/api/route/StatusRouting.scala @@ -34,7 +34,7 @@ case class CurrentStatus( object CurrentStatus { val color = VinylDNSConfig.vinyldnsConfig.getString("color") - val vinyldnsKeyName = VinylDNSConfig.configuredDnsConnections.defaultZoneConnection.keyName + val vinyldnsKeyName = "vinyldns." val version = VinylDNSConfig.vinyldnsConfig.getString("version") } diff --git a/modules/api/src/test/scala/vinyldns/api/backend/CommandHandlerSpec.scala b/modules/api/src/test/scala/vinyldns/api/backend/CommandHandlerSpec.scala index 0fcd533bd..05646c5f8 100644 --- a/modules/api/src/test/scala/vinyldns/api/backend/CommandHandlerSpec.scala +++ b/modules/api/src/test/scala/vinyldns/api/backend/CommandHandlerSpec.scala @@ -21,20 +21,21 @@ import fs2._ import org.mockito import org.mockito.Matchers._ import org.mockito.Mockito._ -import org.mockito.{ArgumentCaptor, Mockito} +import org.mockito.Mockito import org.scalatestplus.mockito.MockitoSugar import org.scalatest.{BeforeAndAfterEach, EitherValues} import org.scalatest.matchers.should.Matchers import org.scalatest.wordspec.AnyWordSpec import vinyldns.api.VinylDNSTestHelpers import vinyldns.api.backend.CommandHandler.{DeleteMessage, RetryMessage} -import vinyldns.api.domain.dns.DnsConnection +import vinyldns.api.backend.dns.DnsBackend import vinyldns.core.domain.batch.{BatchChange, BatchChangeCommand, BatchChangeRepository} import vinyldns.core.domain.record.{RecordChangeRepository, RecordSetChange, RecordSetRepository} import vinyldns.core.domain.zone.{ZoneChange, ZoneChangeType, ZoneCommand, _} import vinyldns.core.queue.{CommandMessage, MessageCount, MessageId, MessageQueue} import vinyldns.core.TestRecordSetData._ import vinyldns.core.TestZoneData._ +import vinyldns.core.domain.backend.{Backend, BackendResolver} import scala.concurrent.ExecutionContext import scala.concurrent.duration._ @@ -66,19 +67,17 @@ class CommandHandlerSpec private val mockZoneChangeProcessor = mock[ZoneChange => IO[ZoneChange]] private val mockRecordChangeProcessor = - mock[(DnsConnection, RecordSetChange) => IO[RecordSetChange]] + mock[(Backend, RecordSetChange) => IO[RecordSetChange]] private val mockZoneSyncProcessor = mock[ZoneChange => IO[ZoneChange]] private val mockBatchChangeProcessor = mock[BatchChangeCommand => IO[Option[BatchChange]]] - private val defaultConn = - ZoneConnection("vinyldns.", "vinyldns.", "nzisn+4G2ldMn0q1CV3vsg==", "10.1.1.1") - private val connections = ConfiguredDnsConnections(defaultConn, defaultConn, List()) + private val mockBackendResolver = mock[BackendResolver] private val processor = CommandHandler.processChangeRequests( mockZoneChangeProcessor, mockRecordChangeProcessor, mockZoneSyncProcessor, mockBatchChangeProcessor, - connections + mockBackendResolver ) override protected def beforeEach(): Unit = @@ -219,37 +218,12 @@ class CommandHandlerSpec val change = TestCommandMessage(pendingCreateAAAA, "foo") doReturn(IO.pure(change)) .when(mockRecordChangeProcessor) - .apply(any[DnsConnection], any[RecordSetChange]) + .apply(any[DnsBackend], any[RecordSetChange]) Stream.emit(change).covary[IO].through(processor).compile.drain.unsafeRunSync() - verify(mockRecordChangeProcessor).apply(any[DnsConnection], any[RecordSetChange]) + verify(mockRecordChangeProcessor).apply(any[DnsBackend], any[RecordSetChange]) verifyZeroInteractions(mockZoneSyncProcessor) verifyZeroInteractions(mockZoneChangeProcessor) } - "use the default zone connection when the change zone connection is not defined" in { - val noConnChange = - pendingCreateAAAA.copy( - zone = pendingCreateAAAA.zone.copy(connection = None, transferConnection = None) - ) - val default = defaultConn.copy(primaryServer = "default.conn.test.com") - val defaultConnProcessor = - CommandHandler.processChangeRequests( - mockZoneChangeProcessor, - mockRecordChangeProcessor, - mockZoneSyncProcessor, - mockBatchChangeProcessor, - ConfiguredDnsConnections(default, default, List()) - ) - val change = TestCommandMessage(noConnChange, "foo") - doReturn(IO.pure(change)) - .when(mockRecordChangeProcessor) - .apply(any[DnsConnection], any[RecordSetChange]) - Stream.emit(change).covary[IO].through(defaultConnProcessor).compile.drain.unsafeRunSync() - - val connCaptor = ArgumentCaptor.forClass(classOf[DnsConnection]) - verify(mockRecordChangeProcessor).apply(connCaptor.capture(), any[RecordSetChange]) - val resolver = connCaptor.getValue.resolver - resolver.getAddress.getHostName shouldBe default.primaryServer - } "handle zone creates" in { val change = TestCommandMessage(zoneCreate, "foo") doReturn(IO.pure(zoneCreate)) @@ -305,7 +279,11 @@ class CommandHandlerSpec // stage our record change processing doReturn(IO.pure(cmd)) .when(mockRecordChangeProcessor) - .apply(any[DnsConnection], any[RecordSetChange]) + .apply(any[Backend], any[RecordSetChange]) + + doReturn(mock[Backend]) + .when(mockBackendResolver) + .resolve(any[Zone]) // stage removing from the queue doReturn(IO.unit).when(mq).remove(cmd) @@ -321,7 +299,7 @@ class CommandHandlerSpec count, 100.millis, stop, - connections, + mockBackendResolver, 1 ) .take(1) @@ -332,7 +310,7 @@ class CommandHandlerSpec // verify our interactions verify(mq, atLeastOnce()).receive(count) verify(mockRecordChangeProcessor) - .apply(any[DnsConnection], mockito.Matchers.eq(pendingCreateAAAA)) + .apply(any[DnsBackend], mockito.Matchers.eq(pendingCreateAAAA)) verify(mq).remove(cmd) } "continue processing on unexpected failure" in { @@ -348,7 +326,11 @@ class CommandHandlerSpec // stage our record change processing our command doReturn(IO.pure(cmd.command)) .when(mockRecordChangeProcessor) - .apply(any[DnsConnection], any[RecordSetChange]) + .apply(any[Backend], any[RecordSetChange]) + + doReturn(mock[Backend]) + .when(mockBackendResolver) + .resolve(any[Zone]) // stage removing from the queue doReturn(IO.unit).when(mq).remove(cmd) @@ -364,7 +346,7 @@ class CommandHandlerSpec count, 100.millis, stop, - connections + mockBackendResolver ) .take(1) @@ -403,6 +385,9 @@ class CommandHandlerSpec .receive(count) // stage processing for a zone update, the simplest of cases + doReturn(mock[Backend]) + .when(mockBackendResolver) + .resolve(any[Zone]) doReturn(IO.pure(Right(zoneUpdate.zone))).when(zoneRepo).save(zoneUpdate.zone) doReturn(IO.pure(zoneUpdate)).when(zoneChangeRepo).save(any[ZoneChange]) @@ -422,7 +407,7 @@ class CommandHandlerSpec recordChangeRepo, batchChangeRepo, AllNotifiers(List.empty), - connections + mockBackendResolver ) // kick off processing of messages diff --git a/modules/api/src/test/scala/vinyldns/api/domain/dns/DnsConnectionSpec.scala b/modules/api/src/test/scala/vinyldns/api/backend/dns/DnsBackendSpec.scala similarity index 85% rename from modules/api/src/test/scala/vinyldns/api/domain/dns/DnsConnectionSpec.scala rename to modules/api/src/test/scala/vinyldns/api/backend/dns/DnsBackendSpec.scala index 167b01bfd..34c488962 100644 --- a/modules/api/src/test/scala/vinyldns/api/domain/dns/DnsConnectionSpec.scala +++ b/modules/api/src/test/scala/vinyldns/api/backend/dns/DnsBackendSpec.scala @@ -14,37 +14,37 @@ * limitations under the License. */ -package vinyldns.api.domain.dns +package vinyldns.api.backend.dns -import java.net.InetAddress +import java.net.{InetAddress, SocketAddress} import cats.scalatest.EitherMatchers import org.joda.time.DateTime import org.mockito.ArgumentCaptor import org.mockito.Matchers._ import org.mockito.Mockito._ -import org.scalatest.BeforeAndAfterEach -import org.scalatestplus.mockito.MockitoSugar import org.scalatest.matchers.should.Matchers import org.scalatest.wordspec.AnyWordSpec +import org.scalatest.{BeforeAndAfterEach, EitherValues} +import org.scalatestplus.mockito.MockitoSugar import org.xbill.DNS -import org.xbill.DNS.{Lookup, Name} -import vinyldns.api.ResultHelpers -import vinyldns.api.domain.dns.DnsProtocol._ +import org.xbill.DNS.{Lookup, Name, TSIG} +import vinyldns.api.backend.dns.DnsProtocol._ +import vinyldns.core.crypto.{CryptoAlgebra, NoOpCrypto} +import vinyldns.core.domain.backend.BackendResponse import vinyldns.core.domain.record.RecordType._ import vinyldns.core.domain.record._ -import vinyldns.core.crypto.CryptoAlgebra import vinyldns.core.domain.zone.{Zone, ZoneConnection} import scala.collection.JavaConverters._ -class DnsConnectionSpec +class DnsBackendSpec extends AnyWordSpec with Matchers with MockitoSugar - with ResultHelpers with BeforeAndAfterEach - with EitherMatchers { + with EitherMatchers + with EitherValues { private val zoneConnection = ZoneConnection("vinyldns.", "vinyldns.", "nzisn+4G2ldMn0q1CV3vsg==", "10.1.1.1") @@ -82,7 +82,10 @@ class DnsConnectionSpec private val mockMessage = mock[DNS.Message] private val messageCaptor = ArgumentCaptor.forClass(classOf[DNS.Message]) private val mockDnsQuery = mock[DnsQuery] - private val underTest = new DnsConnection(mockResolver) { + private val mockSocketAddress = mock[SocketAddress] + private val mockTsig = mock[TSIG] + private val transferInfo = TransferInfo(mockSocketAddress, mockTsig) + private val underTest = new DnsBackend("test", mockResolver, transferInfo) { override def toQuery( name: String, zoneName: String, @@ -94,7 +97,7 @@ class DnsConnectionSpec case _ => Right(mockDnsQuery) } } - private val dnsQueryTest = new DnsConnection(mockResolver) + private val dnsQueryTest = new DnsBackend("query-test", mockResolver, transferInfo) override def beforeEach(): Unit = { doReturn(mockMessage).when(mockMessage).clone() @@ -125,7 +128,7 @@ class DnsConnectionSpec "Creating a Dns Connection" should { "decrypt the zone connection" in { val conn = spy(zoneConnection) - DnsConnection(conn) + DnsBackend("test", conn, None, new NoOpCrypto()) verify(conn).decrypted(any[CryptoAlgebra]) } @@ -133,7 +136,7 @@ class DnsConnectionSpec "parse the port when specified on the primary server" in { val conn = zoneConnection.copy(primaryServer = "dns.comcast.net:19001") - val dnsConn = DnsConnection(conn) + val dnsConn = DnsBackend("test", conn, None, new NoOpCrypto()) val simpleResolver = dnsConn.resolver.asInstanceOf[DNS.SimpleResolver] val address = simpleResolver.getAddress @@ -145,7 +148,7 @@ class DnsConnectionSpec "use default port of 53 when not specified" in { val conn = zoneConnection.copy(primaryServer = "dns.comcast.net") - val dnsConn = DnsConnection(conn) + val dnsConn = DnsBackend("test", conn, None, new NoOpCrypto()) val simpleResolver = dnsConn.resolver.asInstanceOf[DNS.SimpleResolver] val address = simpleResolver.getAddress @@ -158,7 +161,7 @@ class DnsConnectionSpec "Resolving records" should { "return a single record when only one DNS record is returned" in { val records: List[RecordSet] = - rightResultOf(underTest.resolve("www", "vinyldns.", RecordType.A).value) + underTest.resolve("www", "vinyldns.", RecordType.A).unsafeRunSync() records.head should have( 'name ("a-record."), 'typ (RecordType.A), @@ -182,7 +185,7 @@ class DnsConnectionSpec doReturn(List(a1, a2)).when(mockDnsQuery).run() val records: List[RecordSet] = - rightResultOf(underTest.resolve("www", "vinyldns.", RecordType.A).value) + underTest.resolve("www", "vinyldns.", RecordType.A).unsafeRunSync() records.head should have( 'name ("a-record."), 'typ (RecordType.A), @@ -194,14 +197,15 @@ class DnsConnectionSpec doReturn(DNS.Lookup.HOST_NOT_FOUND).when(mockDnsQuery).result val records: List[RecordSet] = - rightResultOf(underTest.resolve("www", "vinyldns.", RecordType.A).value) + underTest.resolve("www", "vinyldns.", RecordType.A).unsafeRunSync() records shouldBe empty } "return an Uncrecoverable error" in { doReturn(DNS.Lookup.UNRECOVERABLE).when(mockDnsQuery).result - val error = leftResultOf(underTest.resolve("www", "vinyldns.", RecordType.A).value) + val error = + underTest.resolve("www", "vinyldns.", RecordType.A).attempt.unsafeRunSync().left.value error shouldBe a[Unrecoverable] } @@ -209,7 +213,8 @@ class DnsConnectionSpec doReturn("this is bad").when(mockDnsQuery).error doReturn(DNS.Lookup.TRY_AGAIN).when(mockDnsQuery).result - val error = leftResultOf(underTest.resolve("www", "vinyldns.", RecordType.A).value) + val error = + underTest.resolve("www", "vinyldns.", RecordType.A).attempt.unsafeRunSync().left.value error shouldBe a[TryAgain] } @@ -217,7 +222,7 @@ class DnsConnectionSpec doReturn(DNS.Lookup.TYPE_NOT_FOUND).when(mockDnsQuery).result val result: List[RecordSet] = - rightResultOf(underTest.resolve("www", "vinyldns.", RecordType.A).value) + underTest.resolve("www", "vinyldns.", RecordType.A).unsafeRunSync() result shouldBe List() } @@ -227,14 +232,15 @@ class DnsConnectionSpec "return an InvalidRecord error if there are no records present" in { val noRecords = testA.copy(records = Nil) - val result = leftResultOf(underTest.addRecord(addRsChange(testZone, noRecords)).value) + val result = + underTest.addRecord(addRsChange(testZone, noRecords)).attempt.unsafeRunSync().left.value result shouldBe a[InvalidRecord] } "send an appropriate update message to the resolver" in { val change = addRsChange() - val result: DnsResponse = rightResultOf(underTest.addRecord(change).value) + val result: DnsResponse = underTest.addRecord(change).unsafeRunSync() val sentMessage = messageCaptor.getValue @@ -253,7 +259,7 @@ class DnsConnectionSpec "send an appropriate update message to the resolver when multiple record sets are present" in { val change = addRsChange(testZone, testAMultiple) - val result: DnsResponse = rightResultOf(underTest.addRecord(change).value) + val result: DnsResponse = underTest.addRecord(change).unsafeRunSync() val sentMessage = messageCaptor.getValue @@ -280,14 +286,20 @@ class DnsConnectionSpec "return an InvalidRecord error if there are no records present" in { val noRecords = testA.copy(records = Nil) - val result = leftResultOf(underTest.updateRecord(updateRsChange(testZone, noRecords)).value) + val result = + underTest + .updateRecord(updateRsChange(testZone, noRecords)) + .attempt + .unsafeRunSync() + .left + .value result shouldBe a[InvalidRecord] } "send an appropriate replace message to the resolver for a name change" in { val change = updateRsChange().copy(updates = Some(testA.copy(name = "updated-a-record"))) - val result: DnsResponse = rightResultOf(underTest.updateRecord(change).value) + val result: DnsResponse = underTest.updateRecord(change).unsafeRunSync() val sentMessage = messageCaptor.getValue @@ -313,7 +325,7 @@ class DnsConnectionSpec "send an appropriate replace message to the resolver for a TTL change" in { val change = updateRsChange(rs = testA.copy(ttl = 300)).copy(updates = Some(testA)) - val result: DnsResponse = rightResultOf(underTest.updateRecord(change).value) + val result: DnsResponse = underTest.updateRecord(change).unsafeRunSync() val sentMessage = messageCaptor.getValue @@ -339,7 +351,7 @@ class DnsConnectionSpec "send an appropriate replace message in the event that the record being replaced is None" in { val change = updateRsChange().copy(updates = None) - val result: DnsResponse = rightResultOf(underTest.updateRecord(change).value) + val result: DnsResponse = underTest.updateRecord(change).unsafeRunSync() val sentMessage = messageCaptor.getValue @@ -353,7 +365,7 @@ class DnsConnectionSpec updates = Some(testAMultiple.copy(name = "updated-a-record")) ) - val result: DnsResponse = rightResultOf(underTest.updateRecord(change).value) + val result: DnsResponse = underTest.updateRecord(change).unsafeRunSync() val sentMessage = messageCaptor.getValue @@ -391,14 +403,20 @@ class DnsConnectionSpec "return an InvalidRecord error if there are no records present" in { val noRecords = testA.copy(records = Nil) - val result = leftResultOf(underTest.updateRecord(updateRsChange(testZone, noRecords)).value) + val result = + underTest + .updateRecord(updateRsChange(testZone, noRecords)) + .attempt + .unsafeRunSync() + .left + .value result shouldBe a[InvalidRecord] } "send a message with an empty body to the resolver when no changes have occurred" in { val change = updateRsChange().copy(updates = Some(testA)) - val result: DnsResponse = rightResultOf(underTest.updateRecord(change).value) + val result: DnsResponse = underTest.updateRecord(change).unsafeRunSync() val sentMessage = messageCaptor.getValue @@ -411,7 +429,7 @@ class DnsConnectionSpec val change = updateRsChange().copy(updates = Some(testA.copy(records = List(AData("127.0.0.1"))))) - val result: DnsResponse = rightResultOf(underTest.updateRecord(change).value) + val result: DnsResponse = underTest.updateRecord(change).unsafeRunSync() val sentMessage = messageCaptor.getValue @@ -437,7 +455,7 @@ class DnsConnectionSpec "send an appropriate replace message in the event that the record being replaced is None" in { val change = updateRsChange().copy(updates = None) - val result: DnsResponse = rightResultOf(underTest.updateRecord(change).value) + val result: DnsResponse = underTest.updateRecord(change).unsafeRunSync() val sentMessage = messageCaptor.getValue @@ -451,7 +469,7 @@ class DnsConnectionSpec updates = Some(testAMultiple.copy(records = List(AData("4.4.4.4"), AData("3.3.3.3")))) ) - val result: DnsResponse = rightResultOf(underTest.updateRecord(change).value) + val result: DnsResponse = underTest.updateRecord(change).unsafeRunSync() val sentMessage = messageCaptor.getValue @@ -498,14 +516,20 @@ class DnsConnectionSpec "return an InvalidRecord error if there are no records present in the delete" in { val noRecords = testA.copy(records = Nil) - val result = leftResultOf(underTest.updateRecord(deleteRsChange(testZone, noRecords)).value) + val result = + underTest + .updateRecord(deleteRsChange(testZone, noRecords)) + .attempt + .unsafeRunSync() + .left + .value result shouldBe a[InvalidRecord] } "send an appropriate delete message to the resolver" in { val change = deleteRsChange() - val result: DnsResponse = rightResultOf(underTest.deleteRecord(change).value) + val result: DnsResponse = underTest.deleteRecord(change).unsafeRunSync() val sentMessage = messageCaptor.getValue @@ -524,7 +548,7 @@ class DnsConnectionSpec "send an appropriate delete message to the resolver for multiple records" in { val change = deleteRsChange(testZone, testAMultiple) - val result: DnsResponse = rightResultOf(underTest.deleteRecord(change).value) + val result: DnsResponse = underTest.deleteRecord(change).unsafeRunSync() val sentMessage = messageCaptor.getValue @@ -544,21 +568,17 @@ class DnsConnectionSpec "applyChange" should { "yield a successful DNS response for a create if there are no errors" in { - underTest.applyChange(addRsChange()).value.unsafeRunSync() shouldBe Right( - NoError(mockMessage) - ) + underTest.applyChange(addRsChange()).unsafeRunSync() shouldBe a[BackendResponse.NoError] } "yield a successful DNS response for an update if there are no errors" in { - underTest.applyChange(updateRsChange()).value.unsafeRunSync() shouldBe Right( - NoError(mockMessage) - ) + underTest.applyChange(updateRsChange()).unsafeRunSync() shouldBe a[BackendResponse.NoError] } "yield a successful DNS response for a delete if there are no errors" in { - underTest.applyChange(deleteRsChange()).value.unsafeRunSync() shouldBe Right( - NoError(mockMessage) - ) + underTest + .applyChange(deleteRsChange()) + .unsafeRunSync() shouldBe a[BackendResponse.NoError] } } @@ -568,7 +588,7 @@ class DnsConnectionSpec underTest .resolve(rsc.recordSet.name, rsc.zone.name, rsc.recordSet.typ) - .value + .attempt .unsafeRunSync() shouldBe left } } diff --git a/modules/api/src/test/scala/vinyldns/api/domain/dns/DnsConversionsSpec.scala b/modules/api/src/test/scala/vinyldns/api/backend/dns/DnsConversionsSpec.scala similarity index 88% rename from modules/api/src/test/scala/vinyldns/api/domain/dns/DnsConversionsSpec.scala rename to modules/api/src/test/scala/vinyldns/api/backend/dns/DnsConversionsSpec.scala index 4a9ddbd0d..52a172e34 100644 --- a/modules/api/src/test/scala/vinyldns/api/domain/dns/DnsConversionsSpec.scala +++ b/modules/api/src/test/scala/vinyldns/api/backend/dns/DnsConversionsSpec.scala @@ -14,23 +14,22 @@ * limitations under the License. */ -package vinyldns.api.domain.dns +package vinyldns.api.backend.dns import java.net.InetAddress import org.joda.time.DateTime import org.mockito.Mockito._ -import org.scalatest.BeforeAndAfterEach -import org.scalatestplus.mockito.MockitoSugar import org.scalatest.matchers.should.Matchers import org.scalatest.wordspec.AnyWordSpec +import org.scalatest.{BeforeAndAfterEach, EitherValues} +import org.scalatestplus.mockito.MockitoSugar import org.xbill.DNS -import vinyldns.api.ResultHelpers -import vinyldns.api.domain.dns.DnsProtocol._ -import vinyldns.core.domain.record._ -import vinyldns.core.domain.zone.Zone +import vinyldns.api.backend.dns.DnsProtocol._ import vinyldns.core.TestRecordSetData.ds import vinyldns.core.domain.Fqdn +import vinyldns.core.domain.record._ +import vinyldns.core.domain.zone.Zone import scala.collection.JavaConverters._ @@ -38,9 +37,9 @@ class DnsConversionsSpec extends AnyWordSpec with Matchers with MockitoSugar - with ResultHelpers with BeforeAndAfterEach - with DnsConversions { + with DnsConversions + with EitherValues { private val testZoneName = "vinyldns." private val testZone = Zone(testZoneName, "test@test.com") @@ -283,7 +282,7 @@ class DnsConversionsSpec } private def roundTrip(rs: RecordSet): RecordSet = { - val recordList = rightValue(toDnsRecords(rs, testZoneName)).map(toRecordSet(_, testZoneDnsName)) + val recordList = toDnsRecords(rs, testZoneName).right.value.map(toRecordSet(_, testZoneDnsName)) recordList.head.copy(records = recordList.flatMap(_.records)) } @@ -324,83 +323,83 @@ class DnsConversionsSpec "Converting to a DNS RRset" should { "convert A record set" in { - val result = rightValue(toDnsRRset(testA, testZoneName)) + val result = toDnsRRset(testA, testZoneName).right.value verifyMatch(result, testA) } "convert multiple record set" in { - val result = rightValue(toDnsRRset(testAMultiple, testZoneName)) + val result = toDnsRRset(testAMultiple, testZoneName).right.value verifyMatch(result, testAMultiple) } "convert AAAA record set" in { - val result = rightValue(toDnsRRset(testAAAA, testZoneName)) + val result = toDnsRRset(testAAAA, testZoneName).right.value verifyMatch(result, testAAAA) } "convert CNAME record set" in { - val result = rightValue(toDnsRRset(testCNAME, testZoneName)) + val result = toDnsRRset(testCNAME, testZoneName).right.value verifyMatch(result, testCNAME) } "convert DS record set" in { - val result = rightValue(toDnsRRset(testDS, testZoneName)) + val result = toDnsRRset(testDS, testZoneName).right.value verifyMatch(result, testDS) } "convert MX record set" in { - val result = rightValue(toDnsRRset(testMX, testZoneName)) + val result = toDnsRRset(testMX, testZoneName).right.value verifyMatch(result, testMX) } "convert NS record set" in { - val result = rightValue(toDnsRRset(testNS, testZoneName)) + val result = toDnsRRset(testNS, testZoneName).right.value verifyMatch(result, testNS) } "convert PTR record set" in { - val result = rightValue(toDnsRRset(testPTR, testZoneName)) + val result = toDnsRRset(testPTR, testZoneName).right.value verifyMatch(result, testPTR) } "convert SOA record set" in { - val result = rightValue(toDnsRRset(testSOA, testZoneName)) + val result = toDnsRRset(testSOA, testZoneName).right.value verifyMatch(result, testSOA) } "convert SPF record set" in { - val result = rightValue(toDnsRRset(testSPF, testZoneName)) + val result = toDnsRRset(testSPF, testZoneName).right.value verifyMatch(result, testSPF) } "convert SSHFP record set" in { - val result = rightValue(toDnsRRset(testSSHFP, testZoneName)) + val result = toDnsRRset(testSSHFP, testZoneName).right.value verifyMatch(result, testSSHFP) } "convert SRV record set" in { - val result = rightValue(toDnsRRset(testSRV, testZoneName)) + val result = toDnsRRset(testSRV, testZoneName).right.value verifyMatch(result, testSRV) } "convert NAPTR record set" in { - val result = rightValue(toDnsRRset(testNAPTR, testZoneName)) + val result = toDnsRRset(testNAPTR, testZoneName).right.value verifyMatch(result, testNAPTR) } "convert TXT record set" in { - val result = rightValue(toDnsRRset(testTXT, testZoneName)) + val result = toDnsRRset(testTXT, testZoneName).right.value verifyMatch(result, testTXT) } "convert long TXT record set" in { - val result = rightValue(toDnsRRset(testLongTXT, testZoneName)) + val result = toDnsRRset(testLongTXT, testZoneName).right.value verifyMatch(result, testLongTXT) } "fail to convert a bad SPF record set" in { - val result = leftValue(toDnsRRset(testLongSPF, testZoneName)) + val result = toDnsRRset(testLongSPF, testZoneName).left.value result shouldBe a[java.lang.IllegalArgumentException] } } @@ -408,67 +407,67 @@ class DnsConversionsSpec "Converting to a Dns Response" should { "return the message when NoError" in { doReturn(DNS.Rcode.NOERROR).when(mockMessage).getRcode - rightValue(toDnsResponse(mockMessage)) shouldBe NoError(mockMessage) + toDnsResponse(mockMessage).right.value shouldBe NoError(mockMessage) } "return a BadKey" in { doReturn(DNS.Rcode.BADKEY).when(mockMessage).getRcode - leftValue(toDnsResponse(mockMessage)) shouldBe a[BadKey] + toDnsResponse(mockMessage).left.value shouldBe a[BadKey] } "return a BadMode" in { doReturn(DNS.Rcode.BADMODE).when(mockMessage).getRcode - leftValue(toDnsResponse(mockMessage)) shouldBe a[BadMode] + toDnsResponse(mockMessage).left.value shouldBe a[BadMode] } "return a BadSig" in { doReturn(DNS.Rcode.BADSIG).when(mockMessage).getRcode - leftValue(toDnsResponse(mockMessage)) shouldBe a[BadSig] + toDnsResponse(mockMessage).left.value shouldBe a[BadSig] } "return a BadTime" in { doReturn(DNS.Rcode.BADTIME).when(mockMessage).getRcode - leftValue(toDnsResponse(mockMessage)) shouldBe a[BadTime] + toDnsResponse(mockMessage).left.value shouldBe a[BadTime] } "return a FormatError" in { doReturn(DNS.Rcode.FORMERR).when(mockMessage).getRcode - leftValue(toDnsResponse(mockMessage)) shouldBe a[FormatError] + toDnsResponse(mockMessage).left.value shouldBe a[FormatError] } "return a NotAuthorized" in { doReturn(DNS.Rcode.NOTAUTH).when(mockMessage).getRcode - leftValue(toDnsResponse(mockMessage)) shouldBe a[NotAuthorized] + toDnsResponse(mockMessage).left.value shouldBe a[NotAuthorized] } "return a NotImplemented" in { doReturn(DNS.Rcode.NOTIMP).when(mockMessage).getRcode - leftValue(toDnsResponse(mockMessage)) shouldBe a[NotImplemented] + toDnsResponse(mockMessage).left.value shouldBe a[NotImplemented] } "return a NotZone" in { doReturn(DNS.Rcode.NOTZONE).when(mockMessage).getRcode - leftValue(toDnsResponse(mockMessage)) shouldBe a[NotZone] + toDnsResponse(mockMessage).left.value shouldBe a[NotZone] } "return a NameNotFound" in { doReturn(DNS.Rcode.NXDOMAIN).when(mockMessage).getRcode - leftValue(toDnsResponse(mockMessage)) shouldBe a[NameNotFound] + toDnsResponse(mockMessage).left.value shouldBe a[NameNotFound] } "return a RecordSetNotFound" in { doReturn(DNS.Rcode.NXRRSET).when(mockMessage).getRcode - leftValue(toDnsResponse(mockMessage)) shouldBe a[RecordSetNotFound] + toDnsResponse(mockMessage).left.value shouldBe a[RecordSetNotFound] } "return a Refused" in { doReturn(DNS.Rcode.REFUSED).when(mockMessage).getRcode - leftValue(toDnsResponse(mockMessage)) shouldBe a[Refused] + toDnsResponse(mockMessage).left.value shouldBe a[Refused] } "return a ServerFailure" in { doReturn(DNS.Rcode.SERVFAIL).when(mockMessage).getRcode - leftValue(toDnsResponse(mockMessage)) shouldBe a[ServerFailure] + toDnsResponse(mockMessage).left.value shouldBe a[ServerFailure] } "return a NameExists" in { doReturn(DNS.Rcode.YXDOMAIN).when(mockMessage).getRcode - leftValue(toDnsResponse(mockMessage)) shouldBe a[NameExists] + toDnsResponse(mockMessage).left.value shouldBe a[NameExists] } "return a RecordSetExists" in { doReturn(DNS.Rcode.YXRRSET).when(mockMessage).getRcode - leftValue(toDnsResponse(mockMessage)) shouldBe a[RecordSetExists] + toDnsResponse(mockMessage).left.value shouldBe a[RecordSetExists] } "return a UnrecognizedResponse" in { doReturn(999).when(mockMessage).getRcode - leftValue(toDnsResponse(mockMessage)) shouldBe a[UnrecognizedResponse] + toDnsResponse(mockMessage).left.value shouldBe a[UnrecognizedResponse] } } @@ -572,7 +571,7 @@ class DnsConversionsSpec "Converting to an update message" should { "work for an Add message" in { - val dnsMessage = rightValue(toAddRecordMessage(rrset(testDnsA), testZoneName)) + val dnsMessage = toAddRecordMessage(rrset(testDnsA), testZoneName).right.value val dnsRecord = dnsMessage.getSectionArray(DNS.Section.UPDATE)(0) dnsRecord.getName.toString shouldBe "a-record." dnsRecord.getTTL shouldBe testA.ttl @@ -585,7 +584,7 @@ class DnsConversionsSpec } "work for an Update message" in { val dnsMessage = - rightValue(toUpdateRecordMessage(rrset(testDnsA), rrset(testDnsAReplace), testZoneName)) + toUpdateRecordMessage(rrset(testDnsA), rrset(testDnsAReplace), testZoneName).right.value // Update record issues a replace, the first section is an EmptyRecord containing the name and type to replace val emptyRecord = dnsMessage.getSectionArray(DNS.Section.UPDATE)(0) emptyRecord.getName.toString shouldBe "a-record-2." @@ -604,7 +603,7 @@ class DnsConversionsSpec zoneRRset.getName.toString shouldBe "vinyldns." } "work for a Delete message" in { - val dnsMessage = rightValue(toDeleteRecordMessage(rrset(testDnsA), testZoneName)) + val dnsMessage = toDeleteRecordMessage(rrset(testDnsA), testZoneName).right.value val dnsRecord = dnsMessage.getSectionArray(DNS.Section.UPDATE)(0) dnsRecord.getName.toString shouldBe "a-record." @@ -629,7 +628,7 @@ class DnsConversionsSpec "convert zone name to @" in { val actual = toDnsRecords(testAt, testZoneName) val omitFinalDot = false - rightValue(actual).head.getName.toString(omitFinalDot) shouldBe testZoneName + actual.right.value.head.getName.toString(omitFinalDot) shouldBe testZoneName } } diff --git a/modules/api/src/test/scala/vinyldns/api/domain/record/RecordSetServiceSpec.scala b/modules/api/src/test/scala/vinyldns/api/domain/record/RecordSetServiceSpec.scala index bbf3912d9..540c24736 100644 --- a/modules/api/src/test/scala/vinyldns/api/domain/record/RecordSetServiceSpec.scala +++ b/modules/api/src/test/scala/vinyldns/api/domain/record/RecordSetServiceSpec.scala @@ -24,10 +24,8 @@ import org.scalatestplus.mockito.MockitoSugar import org.scalatest.matchers.should.Matchers import org.scalatest.wordspec.AnyWordSpec import org.scalatest.BeforeAndAfterEach -import vinyldns.api.Interfaces._ import vinyldns.api.ResultHelpers import vinyldns.api.domain.access.AccessValidations -import vinyldns.api.domain.dns.DnsConnection import vinyldns.api.domain.record.RecordSetHelpers._ import vinyldns.api.domain.zone._ import vinyldns.api.route.{ListGlobalRecordSetsResponse, ListRecordSetsByZoneResponse} @@ -36,6 +34,7 @@ import vinyldns.core.TestRecordSetData._ import vinyldns.core.TestZoneData._ import vinyldns.core.domain.HighValueDomainError import vinyldns.core.domain.auth.AuthPrincipal +import vinyldns.core.domain.backend.{Backend, BackendResolver} import vinyldns.core.domain.membership.{GroupRepository, ListUsersResults, UserRepository} import vinyldns.core.domain.record._ import vinyldns.core.domain.zone._ @@ -55,12 +54,9 @@ class RecordSetServiceSpec private val mockRecordChangeRepo = mock[RecordChangeRepository] private val mockUserRepo = mock[UserRepository] private val mockMessageQueue = mock[MessageQueue] - private val zoneConnection = - ZoneConnection("vinyldns.", "vinyldns.", "nzisn+4G2ldMn0q1CV3vsg==", "10.1.1.1") - private val configuredDnsConnections = - ConfiguredDnsConnections(zoneConnection, zoneConnection, List()) - private val mockDnsConnection = - mock[DnsConnection] + private val mockBackend = + mock[Backend] + private val mockBackendResolver = mock[BackendResolver] doReturn(IO.pure(Some(okZone))).when(mockZoneRepo).getZone(okZone.id) doReturn(IO.pure(Some(zoneNotAuthorized))) @@ -70,6 +66,7 @@ class RecordSetServiceSpec doReturn(IO.pure(Some(sharedZoneRecord.copy(status = RecordSetStatus.Active)))) .when(mockRecordRepo) .getRecordSet(sharedZoneRecord.id) + doReturn(mockBackend).when(mockBackendResolver).resolve(any[Zone]) val underTest = new RecordSetService( mockZoneRepo, @@ -79,8 +76,7 @@ class RecordSetServiceSpec mockUserRepo, mockMessageQueue, new AccessValidations(), - (_, _) => mockDnsConnection, - configuredDnsConnections, + mockBackendResolver, false ) @@ -92,8 +88,7 @@ class RecordSetServiceSpec mockUserRepo, mockMessageQueue, new AccessValidations(), - (_, _) => mockDnsConnection, - configuredDnsConnections, + mockBackendResolver, true ) @@ -140,8 +135,8 @@ class RecordSetServiceSpec .when(mockRecordRepo) .getRecordSets(okZone.id, record.name, record.typ) - doReturn(IO(List(aaaa)).toResult) - .when(mockDnsConnection) + doReturn(IO(List(aaaa))) + .when(mockBackend) .resolve(record.name, okZone.name, record.typ) val result = leftResultOf(underTest.addRecordSet(aaaa, okAuth).value) @@ -298,8 +293,8 @@ class RecordSetServiceSpec doReturn(IO.pure(List(record))) .when(mockRecordRepo) .getRecordSets(okZone.id, record.name, record.typ) - doReturn(IO(List()).toResult) - .when(mockDnsConnection) + doReturn(IO(List())) + .when(mockBackend) .resolve(record.name, okZone.name, record.typ) doReturn(IO.pure(List())) .when(mockRecordRepo) @@ -653,8 +648,8 @@ class RecordSetServiceSpec doReturn(IO.pure(List(newRecord))) .when(mockRecordRepo) .getRecordSetsByName(okZone.id, newRecord.name) - doReturn(IO(List()).toResult) - .when(mockDnsConnection) + doReturn(IO(List())) + .when(mockBackend) .resolve(newRecord.name, okZone.name, newRecord.typ) val result: RecordSetChange = rightResultOf( diff --git a/modules/api/src/test/scala/vinyldns/api/domain/zone/ZoneConnectionValidatorSpec.scala b/modules/api/src/test/scala/vinyldns/api/domain/zone/ZoneConnectionValidatorSpec.scala index 44d7a5058..f565ccde3 100644 --- a/modules/api/src/test/scala/vinyldns/api/domain/zone/ZoneConnectionValidatorSpec.scala +++ b/modules/api/src/test/scala/vinyldns/api/domain/zone/ZoneConnectionValidatorSpec.scala @@ -23,14 +23,13 @@ import org.scalatestplus.mockito.MockitoSugar import org.scalatest.matchers.should.Matchers import org.scalatest.wordspec.AnyWordSpec import org.scalatest.BeforeAndAfterEach -import vinyldns.api.Interfaces._ -import vinyldns.api.domain.dns.DnsConnection -import vinyldns.api.domain.dns.DnsProtocol.TypeNotFound import vinyldns.core.domain.record._ import vinyldns.api.ResultHelpers import cats.effect._ +import org.mockito.Matchers.any import vinyldns.core.domain.Fqdn -import vinyldns.core.domain.zone.{ConfiguredDnsConnections, DnsBackend, Zone, ZoneConnection} +import vinyldns.core.domain.backend.{Backend, BackendResolver} +import vinyldns.core.domain.zone.{ConfiguredDnsConnections, LegacyDnsBackend, Zone, ZoneConnection} import scala.concurrent.duration._ @@ -43,18 +42,12 @@ class ZoneConnectionValidatorSpec with EitherMatchers with EitherValues { - private val mockDnsConnection = mock[DnsConnection] + private val mockBackend = mock[Backend] private val mockZoneView = mock[ZoneView] + private val mockBackendResolver = mock[BackendResolver] override protected def beforeEach(): Unit = - reset(mockDnsConnection, mockZoneView) - - private def testDnsConnection(conn: ZoneConnection) = - if (conn.keyName == "error.") { - throw new RuntimeException("main connection failure!") - } else { - mockDnsConnection - } + reset(mockBackend, mockZoneView) private def testLoadDns(zone: Zone) = zone.name match { case "error." => IO.raiseError(new RuntimeException("transfer connection failure!")) @@ -67,21 +60,14 @@ class ZoneConnectionValidatorSpec IO.pure(mockZoneView) } - private def testDefaultConnection: ZoneConnection = - ZoneConnection("name", "key-name", "key", "localhost:19001") - - private def testConfiguredConnections: ConfiguredDnsConnections = - ConfiguredDnsConnections(testDefaultConnection, testDefaultConnection, List()) - private def generateZoneView(zone: Zone, recordSets: RecordSet*): ZoneView = ZoneView( zone = zone, recordSets = recordSets.toList ) - class TestConnectionValidator() extends ZoneConnectionValidator(testConfiguredConnections) { + class TestConnectionValidator() extends ZoneConnectionValidator(mockBackendResolver) { override val opTimeout: FiniteDuration = 10.milliseconds - override def dnsConnection(conn: ZoneConnection): DnsConnection = testDnsConnection(conn) override def loadDns(zone: Zone): IO[ZoneView] = testLoadDns(zone) override def isValidBackendId(backendId: Option[String]): Either[Throwable, Unit] = Right(()) @@ -142,11 +128,9 @@ class ZoneConnectionValidatorSpec List(NSData(Fqdn("sub.some.test.ns."))) ) - private val mockRecordSet = mock[RecordSet] - val zc = ZoneConnection("zc.", "zc.", "zc", "10.1.1.1") val transfer = ZoneConnection("transfer.", "transfer.", "transfer", "10.1.1.1") - val backend = DnsBackend( + val backend = LegacyDnsBackend( "some-backend-id", zc.copy(name = "backend-conn"), transfer.copy(name = "backend-transfer") @@ -159,9 +143,8 @@ class ZoneConnectionValidatorSpec doReturn(generateZoneView(testZone, successSoa, successNS, delegatedNS).recordSetsMap) .when(mockZoneView) .recordSetsMap - doReturn(List(successSoa).toResult) - .when(mockDnsConnection) - .resolve(testZone.name, testZone.name, RecordType.SOA) + doReturn(IO.pure(true)).when(mockBackend).zoneExists(any[Zone]) + doReturn(mockBackend).when(mockBackendResolver).resolve(any[Zone]) val result = awaitResultOf(underTest.validateZoneConnections(testZone).value) result should be(right) @@ -172,9 +155,8 @@ class ZoneConnectionValidatorSpec doReturn(generateZoneView(testZone, successSoa, failureNs, delegatedNS).recordSetsMap) .when(mockZoneView) .recordSetsMap - doReturn(List(successSoa).toResult) - .when(mockDnsConnection) - .resolve(testZone.name, testZone.name, RecordType.SOA) + doReturn(IO.pure(true)).when(mockBackend).zoneExists(any[Zone]) + doReturn(mockBackend).when(mockBackendResolver).resolve(any[Zone]) val result = leftResultOf(underTest.validateZoneConnections(testZone).value) result shouldBe ZoneValidationFailed( @@ -187,9 +169,11 @@ class ZoneConnectionValidatorSpec "respond with a failure if no records are returned from the backend" in { doReturn(testZone).when(mockZoneView).zone doReturn(generateZoneView(testZone).recordSetsMap).when(mockZoneView).recordSetsMap - doReturn(List.empty[RecordSet].toResult) - .when(mockDnsConnection) + doReturn(IO.pure(List.empty[RecordSet])) + .when(mockBackend) .resolve(testZone.name, testZone.name, RecordType.SOA) + doReturn(IO.pure(true)).when(mockBackend).zoneExists(any[Zone]) + doReturn(mockBackend).when(mockBackendResolver).resolve(any[Zone]) val result = leftResultOf(underTest.validateZoneConnections(testZone).value) result shouldBe a[ZoneValidationFailed] @@ -200,28 +184,20 @@ class ZoneConnectionValidatorSpec ) } - "respond with a failure if any failure is returned from the backend" in { - doReturn(result(TypeNotFound("fail"))) - .when(mockDnsConnection) - .resolve(testZone.name, testZone.name, RecordType.SOA) - - val error = leftResultOf(underTest.validateZoneConnections(testZone).value) - error shouldBe ConnectionFailed(testZone, s"Unable to connect to zone: fail") - } - "respond with a failure if connection cant be made" in { val badZone = Zone( - "vinyldns.", + "error.", "test@test.com", connection = Some(ZoneConnection("error.", "error.", "nzisn+4G2ldMn0q1CV3vsg==", "10.1.1.1")), transferConnection = Some(ZoneConnection("vinyldns.", "vinyldns.", "nzisn+4G2ldMn0q1CV3vsg==", "10.1.1.1")) ) + doReturn(IO.pure(true)).when(mockBackend).zoneExists(any[Zone]) + doReturn(mockBackend).when(mockBackendResolver).resolve(any[Zone]) val result = leftResultOf(underTest.validateZoneConnections(badZone).value) result shouldBe a[ConnectionFailed] - result.getMessage should include("main connection failure!") } "respond with a failure if loadDns throws an error" in { @@ -234,9 +210,8 @@ class ZoneConnectionValidatorSpec Some(ZoneConnection("vinyldns.", "vinyldns.", "nzisn+4G2ldMn0q1CV3vsg==", "10.1.1.1")) ) - doReturn(List(mockRecordSet).toResult) - .when(mockDnsConnection) - .resolve(badZone.name, badZone.name, RecordType.SOA) + doReturn(IO.pure(true)).when(mockBackend).zoneExists(any[Zone]) + doReturn(mockBackend).when(mockBackendResolver).resolve(any[Zone]) val result = leftResultOf(underTest.validateZoneConnections(badZone).value) result shouldBe a[ConnectionFailed] @@ -244,11 +219,11 @@ class ZoneConnectionValidatorSpec } "isValidBackendId" should { - val backend = DnsBackend("some-test-backend", testDefaultConnection, testDefaultConnection) + doReturn(true).when(mockBackendResolver).isRegistered("some-test-backend") + doReturn(false).when(mockBackendResolver).isRegistered("bad") + val underTest = - new ZoneConnectionValidator( - ConfiguredDnsConnections(testDefaultConnection, testDefaultConnection, List(backend)) - ) + new ZoneConnectionValidator(mockBackendResolver) "return success if the backendId exists" in { underTest.isValidBackendId(Some("some-test-backend")) shouldBe right @@ -260,43 +235,5 @@ class ZoneConnectionValidatorSpec underTest.isValidBackendId(Some("bad")) shouldBe left } } - "getZoneConnection" should { - "get the specified zone connection if provided" in { - // both backendId and connection info specified; prefer connection info - val zone = testZone.copy(backendId = Some("some-backend-id")) - ZoneConnectionValidator.getZoneConnection(zone, connections) shouldBe zone.connection.get - } - "get a zone connection by backendID" in { - val zone = Zone("name.", "email", backendId = Some("some-backend-id")) - ZoneConnectionValidator.getZoneConnection(zone, connections) shouldBe backend.zoneConnection - } - "fall to default without connection info" in { - val zone = Zone("name.", "email") - ZoneConnectionValidator.getZoneConnection(zone, connections) shouldBe zc - } - "fall to default with an invalid backendId" in { - val zone = Zone("name.", "email", backendId = Some("bad-id")) - ZoneConnectionValidator.getZoneConnection(zone, connections) shouldBe zc - } - } - "getTransferConnection" should { - "get the specified transfer connection if provided" in { - // both backendId and connection info specified; prefer connection info - val zone = testZone.copy(backendId = Some("some-backend-id")) - ZoneConnectionValidator.getTransferConnection(zone, connections) shouldBe zone.transferConnection.get - } - "get a transfer connection by backendID" in { - val zone = Zone("name.", "email", backendId = Some("some-backend-id")) - ZoneConnectionValidator.getTransferConnection(zone, connections) shouldBe backend.transferConnection - } - "fall to default without connection info" in { - val zone = Zone("name.", "email") - ZoneConnectionValidator.getTransferConnection(zone, connections) shouldBe transfer - } - "fall to default with an invalid backendId" in { - val zone = Zone("name.", "email", backendId = Some("bad-id")) - ZoneConnectionValidator.getTransferConnection(zone, connections) shouldBe transfer - } - } } } diff --git a/modules/api/src/test/scala/vinyldns/api/domain/zone/ZoneServiceSpec.scala b/modules/api/src/test/scala/vinyldns/api/domain/zone/ZoneServiceSpec.scala index a114f00df..38117313e 100644 --- a/modules/api/src/test/scala/vinyldns/api/domain/zone/ZoneServiceSpec.scala +++ b/modules/api/src/test/scala/vinyldns/api/domain/zone/ZoneServiceSpec.scala @@ -34,6 +34,7 @@ import vinyldns.core.domain.zone._ import vinyldns.core.queue.MessageQueue import vinyldns.core.TestMembershipData._ import vinyldns.core.TestZoneData._ +import vinyldns.core.domain.backend.BackendResolver import scala.concurrent.duration._ @@ -50,6 +51,7 @@ class ZoneServiceSpec private val mockUserRepo = mock[UserRepository] private val mockZoneChangeRepo = mock[ZoneChangeRepository] private val mockMessageQueue = mock[MessageQueue] + private val mockBackendResolver = mock[BackendResolver] private val badConnection = ZoneConnection("bad", "bad", "bad", "bad") private val abcZoneSummary = ZoneSummaryInfo(abcZone, abcGroup.name, AccessLevel.Delete) private val xyzZoneSummary = ZoneSummaryInfo(xyzZone, xyzGroup.name, AccessLevel.NoAccess) @@ -76,7 +78,8 @@ class ZoneServiceSpec TestConnectionValidator, mockMessageQueue, new ZoneValidations(1000), - new AccessValidations() + new AccessValidations(), + mockBackendResolver ) private val createZoneAuthorized = CreateZoneInput( diff --git a/modules/api/src/test/scala/vinyldns/api/domain/zone/ZoneViewLoaderSpec.scala b/modules/api/src/test/scala/vinyldns/api/domain/zone/ZoneViewLoaderSpec.scala index 7b5ce4cdf..b1106f667 100644 --- a/modules/api/src/test/scala/vinyldns/api/domain/zone/ZoneViewLoaderSpec.scala +++ b/modules/api/src/test/scala/vinyldns/api/domain/zone/ZoneViewLoaderSpec.scala @@ -26,25 +26,29 @@ import org.scalatestplus.mockito.MockitoSugar import org.scalatest.matchers.should.Matchers import org.scalatest.wordspec.AnyWordSpec import org.xbill.DNS -import org.xbill.DNS.{Name, ZoneTransferIn} -import vinyldns.api.domain.dns.DnsConversions +import org.xbill.DNS.Name import vinyldns.core.domain.record._ -import scala.collection.JavaConverters._ import scala.collection.mutable import cats.effect._ +import vinyldns.api.backend.dns.DnsConversions import vinyldns.core.domain.Fqdn +import vinyldns.core.domain.backend.{Backend, BackendResolver} import vinyldns.core.domain.record.NameSort.NameSort import vinyldns.core.domain.record.RecordType.RecordType import vinyldns.core.domain.zone.{Zone, ZoneConnection, ZoneStatus} class ZoneViewLoaderSpec extends AnyWordSpec with Matchers with MockitoSugar with DnsConversions { - val testZoneName = "vinyldns." - val testZoneConnection: Option[ZoneConnection] = Some( + private val testZoneName = "vinyldns." + + private val testZoneConnection: Option[ZoneConnection] = Some( ZoneConnection(testZoneName, testZoneName, "nzisn+4G2ldMn0q1CV3vsg==", "127.0.0.1:19001") ) + private val mockBackendResolver = mock[BackendResolver] + private val mockBackend = mock[Backend] + private val testZone = Zone("vinyldns.", "test@test.com") private val records = List( RecordSet( @@ -121,8 +125,6 @@ class ZoneViewLoaderSpec extends AnyWordSpec with Matchers with MockitoSugar wit transferConnection = testZoneConnection ) - val mockTransfer = mock[ZoneTransferIn] - val expectedRecords = List( RecordSet( zoneId = testZone.id, @@ -196,12 +198,10 @@ class ZoneViewLoaderSpec extends AnyWordSpec with Matchers with MockitoSugar wit ) ) - doReturn(dnsRecords.asJava).when(mockTransfer).getAXFR + doReturn(IO.pure(expectedRecords)).when(mockBackend).loadZone(any[Zone], any[Int]) + doReturn(mockBackend).when(mockBackendResolver).resolve(any[Zone]) - val mockTransferFunc = mock[Zone => ZoneTransferIn] - doReturn(mockTransfer).when(mockTransferFunc).apply(testZone) - - val underTest = DnsZoneViewLoader(testZone, mockTransferFunc) + val underTest = DnsZoneViewLoader(testZone, mockBackend, 1000) val actual = underTest.load().unsafeToFuture() @@ -235,8 +235,6 @@ class ZoneViewLoaderSpec extends AnyWordSpec with Matchers with MockitoSugar wit transferConnection = testZoneConnection ) - val mockTransfer = mock[ZoneTransferIn] - val expectedRecords = List( RecordSet( zoneId = testZone.id, @@ -346,12 +344,10 @@ class ZoneViewLoaderSpec extends AnyWordSpec with Matchers with MockitoSugar wit ) ) - doReturn(dnsRecords.asJava).when(mockTransfer).getAXFR + doReturn(IO.pure(expectedRecords)).when(mockBackend).loadZone(any[Zone], any[Int]) + doReturn(mockBackend).when(mockBackendResolver).resolve(any[Zone]) - val mockTransferFunc = mock[Zone => ZoneTransferIn] - doReturn(mockTransfer).when(mockTransferFunc).apply(testZone) - - val underTest = DnsZoneViewLoader(testZone, mockTransferFunc) + val underTest = DnsZoneViewLoader(testZone, mockBackend, 1000) val actual = underTest.load().unsafeToFuture() diff --git a/modules/api/src/test/scala/vinyldns/api/engine/RecordSetChangeHandlerSpec.scala b/modules/api/src/test/scala/vinyldns/api/engine/RecordSetChangeHandlerSpec.scala index d43911d65..34bab7a50 100644 --- a/modules/api/src/test/scala/vinyldns/api/engine/RecordSetChangeHandlerSpec.scala +++ b/modules/api/src/test/scala/vinyldns/api/engine/RecordSetChangeHandlerSpec.scala @@ -24,13 +24,11 @@ import org.mockito.Mockito._ import org.scalatestplus.mockito.MockitoSugar import org.scalatest.matchers.should.Matchers import org.scalatest.wordspec.AnyWordSpec -import org.scalatest.BeforeAndAfterEach -import org.xbill.DNS -import vinyldns.api.domain.dns.DnsConnection -import vinyldns.api.domain.dns.DnsProtocol.{NoError, NotAuthorized, Refused, TryAgain} +import org.scalatest.{BeforeAndAfterEach, EitherValues} +import vinyldns.api.backend.dns.DnsProtocol.{NotAuthorized, TryAgain} import vinyldns.api.engine.RecordSetChangeHandler.{AlreadyApplied, ReadyToApply, Requeue} import vinyldns.api.repository.InMemoryBatchChangeRepository -import vinyldns.api.{CatsHelpers, Interfaces} +import vinyldns.api.CatsHelpers import vinyldns.core.domain.batch.{ BatchChange, BatchChangeApprovalStatus, @@ -43,19 +41,20 @@ import vinyldns.core.TestRecordSetData._ import scala.concurrent.ExecutionContext import cats.effect.ContextShift +import vinyldns.core.domain.backend.{Backend, BackendResponse} class RecordSetChangeHandlerSpec extends AnyWordSpec with Matchers with MockitoSugar with BeforeAndAfterEach - with CatsHelpers { + with CatsHelpers + with EitherValues { private implicit val timer: Timer[IO] = IO.timer(ExecutionContext.global) - private val mockConn = mock[DnsConnection] + private val mockBackend = mock[Backend] private val mockRsRepo = mock[RecordSetRepository] private val mockChangeRepo = mock[RecordChangeRepository] - private val mockDnsMessage = mock[DNS.Message] private val rsRepoCaptor = ArgumentCaptor.forClass(classOf[ChangeSet]) private val changeRepoCaptor = ArgumentCaptor.forClass(classOf[ChangeSet]) @@ -112,7 +111,7 @@ class RecordSetChangeHandlerSpec RecordSetChangeHandler(mockRsRepo, mockChangeRepo, batchRepo) override protected def beforeEach(): Unit = { - reset(mockConn, mockRsRepo, mockChangeRepo) + reset(mockBackend, mockRsRepo, mockChangeRepo) batchRepo.clear() // seed the linked batch change in the DB @@ -126,14 +125,14 @@ class RecordSetChangeHandlerSpec "Handling Pending Changes" should { "complete the change successfully if already applied" in { - doReturn(Interfaces.result(List(rs))) - .when(mockConn) + doReturn(IO.pure(List(rs))) + .when(mockBackend) .resolve(rs.name, rsChange.zone.name, rs.typ) doReturn(IO.pure(cs)).when(mockChangeRepo).save(any[ChangeSet]) doReturn(IO.pure(cs)).when(mockRsRepo).apply(any[ChangeSet]) doReturn(IO.pure(List(rs))).when(mockRsRepo).getRecordSetsByName(cs.zoneId, rs.name) - val test = underTest.apply(mockConn, rsChange) + val test = underTest.apply(mockBackend, rsChange) test.unsafeRunSync() verify(mockRsRepo).apply(rsRepoCaptor.capture()) @@ -162,17 +161,17 @@ class RecordSetChangeHandlerSpec "apply the change if not yet applied" in { // The second return is for verify - doReturn(Interfaces.result(List())) - .doReturn(Interfaces.result(List(rs))) - .when(mockConn) + doReturn(IO.pure(List())) + .doReturn(IO.pure(List(rs))) + .when(mockBackend) .resolve(rs.name, rsChange.zone.name, rs.typ) - doReturn(Interfaces.result(NoError(mockDnsMessage))).when(mockConn).applyChange(rsChange) + doReturn(IO.pure(BackendResponse.NoError("test"))).when(mockBackend).applyChange(rsChange) doReturn(IO.pure(cs)).when(mockChangeRepo).save(any[ChangeSet]) doReturn(IO.pure(cs)).when(mockRsRepo).apply(any[ChangeSet]) doReturn(IO.pure(List.empty)).when(mockRsRepo).getRecordSetsByName(cs.zoneId, rs.name) - val test = underTest.apply(mockConn, rsChange) + val test = underTest.apply(mockBackend, rsChange) test.unsafeRunSync() verify(mockRsRepo).apply(rsRepoCaptor.capture()) @@ -188,8 +187,8 @@ class RecordSetChangeHandlerSpec savedCs.changes.head.status shouldBe RecordSetChangeStatus.Complete // make sure the record was applied and then verified - verify(mockConn).applyChange(rsChange) - verify(mockConn, times(2)).resolve(rs.name, rsChange.zone.name, rs.typ) + verify(mockBackend).applyChange(rsChange) + verify(mockBackend, times(2)).resolve(rs.name, rsChange.zone.name, rs.typ) val batchChangeUpdates = await(batchRepo.getBatchChange(batchChange.id)) val updatedSingleChanges = completeCreateAAAASingleChanges.map { ch => @@ -205,19 +204,19 @@ class RecordSetChangeHandlerSpec "bypass verify and fail if the dns update fails" in { // The second return is for verify - doReturn(Interfaces.result(List())) - .doReturn(Interfaces.result(List(rs))) - .when(mockConn) + doReturn(IO.pure(List())) + .doReturn(IO.pure(List(rs))) + .when(mockBackend) .resolve(rs.name, rsChange.zone.name, rs.typ) - doReturn(Interfaces.result(Left(NotAuthorized("dns failure")))) - .when(mockConn) + doReturn(IO.raiseError(NotAuthorized("dns failure"))) + .when(mockBackend) .applyChange(rsChange) doReturn(IO.pure(cs)).when(mockChangeRepo).save(any[ChangeSet]) doReturn(IO.pure(cs)).when(mockRsRepo).apply(any[ChangeSet]) doReturn(IO.pure(List.empty)).when(mockRsRepo).getRecordSetsByName(cs.zoneId, rs.name) - val test = underTest.apply(mockConn, rsChange) + val test = underTest.apply(mockBackend, rsChange) test.unsafeRunSync() verify(mockRsRepo).apply(rsRepoCaptor.capture()) @@ -234,10 +233,10 @@ class RecordSetChangeHandlerSpec savedCs.changes.head.status shouldBe RecordSetChangeStatus.Failed // make sure the record was applied - verify(mockConn).applyChange(rsChange) + verify(mockBackend).applyChange(rsChange) // make sure we only called resolve once when validating, ensures that verify was not called - verify(mockConn, times(1)).resolve(rs.name, rsChange.zone.name, rs.typ) + verify(mockBackend, times(1)).resolve(rs.name, rsChange.zone.name, rs.typ) val batchChangeUpdates = await(batchRepo.getBatchChange(batchChange.id)) val updatedSingleChanges = completeCreateAAAASingleChanges.map { ch => @@ -253,17 +252,17 @@ class RecordSetChangeHandlerSpec "fail the change in verify if verify errors" in { // All returns after first are for verify. Retry 2 times and succeed - doReturn(Interfaces.result(List())) - .doReturn(Interfaces.result(Left(NotAuthorized("dns-fail")))) - .when(mockConn) + doReturn(IO.pure(List())) + .doReturn(IO.raiseError(NotAuthorized("dns-fail"))) + .when(mockBackend) .resolve(rs.name, rsChange.zone.name, rs.typ) - doReturn(Interfaces.result(NoError(mockDnsMessage))).when(mockConn).applyChange(rsChange) + doReturn(IO.pure(BackendResponse.NoError("test"))).when(mockBackend).applyChange(rsChange) doReturn(IO.pure(cs)).when(mockChangeRepo).save(any[ChangeSet]) doReturn(IO.pure(cs)).when(mockRsRepo).apply(any[ChangeSet]) doReturn(IO.pure(List.empty)).when(mockRsRepo).getRecordSetsByName(cs.zoneId, rs.name) - val test = underTest.apply(mockConn, rsChange) + val test = underTest.apply(mockBackend, rsChange) test.unsafeRunSync() verify(mockRsRepo).apply(rsRepoCaptor.capture()) @@ -279,10 +278,10 @@ class RecordSetChangeHandlerSpec savedCs.changes.head.status shouldBe RecordSetChangeStatus.Failed // make sure the record was applied and then verified - verify(mockConn).applyChange(rsChange) + verify(mockBackend).applyChange(rsChange) // we will retry the verify 3 times based on the mock setup - verify(mockConn, times(2)).resolve(rs.name, rsChange.zone.name, rs.typ) + verify(mockBackend, times(2)).resolve(rs.name, rsChange.zone.name, rs.typ) val batchChangeUpdates = await(batchRepo.getBatchChange(batchChange.id)) val updatedSingleChanges = completeCreateAAAASingleChanges.map { ch => @@ -297,30 +296,30 @@ class RecordSetChangeHandlerSpec } "requeue the change in verify if permissible errors" in { - doReturn(Interfaces.result(List())) - .doReturn(Interfaces.result(Left(TryAgain("dns-fail")))) - .when(mockConn) + doReturn(IO.pure(List())) + .doReturn(IO.raiseError(TryAgain("dns-fail"))) + .when(mockBackend) .resolve(rs.name, rsChange.zone.name, rs.typ) - doReturn(Interfaces.result(NoError(mockDnsMessage))).when(mockConn).applyChange(rsChange) + doReturn(IO.pure(BackendResponse.NoError("test"))).when(mockBackend).applyChange(rsChange) doReturn(IO.pure(cs)).when(mockChangeRepo).save(any[ChangeSet]) doReturn(IO.pure(cs)).when(mockRsRepo).apply(any[ChangeSet]) doReturn(IO.pure(List.empty)).when(mockRsRepo).getRecordSetsByName(cs.zoneId, rs.name) - val test = underTest.apply(mockConn, rsChange) + val test = underTest.apply(mockBackend, rsChange) a[Requeue] shouldBe thrownBy(test.unsafeRunSync()) } "fail the change if validating fails with an error" in { // Stage an error on the first resolve, which will cause validate to fail - doReturn(Interfaces.result(Left(NotAuthorized("dns-failure")))) - .when(mockConn) + doReturn(IO.raiseError(NotAuthorized("dns-failure"))) + .when(mockBackend) .resolve(rs.name, rsChange.zone.name, rs.typ) doReturn(IO.pure(cs)).when(mockChangeRepo).save(any[ChangeSet]) doReturn(IO.pure(cs)).when(mockRsRepo).apply(any[ChangeSet]) - val test = underTest.apply(mockConn, rsChange) + val test = underTest.apply(mockBackend, rsChange) test.unsafeRunSync() verify(mockRsRepo).apply(rsRepoCaptor.capture()) @@ -336,8 +335,8 @@ class RecordSetChangeHandlerSpec savedCs.changes.head.status shouldBe RecordSetChangeStatus.Failed // we failed in validation, so we should never issue a dns update - verify(mockConn, never()).applyChange(rsChange) - verify(mockConn, times(1)).resolve(rs.name, rsChange.zone.name, rs.typ) + verify(mockBackend, never()).applyChange(rsChange) + verify(mockBackend, times(1)).resolve(rs.name, rsChange.zone.name, rs.typ) val batchChangeUpdates = await(batchRepo.getBatchChange(batchChange.id)) val updatedSingleChanges = completeCreateAAAASingleChanges.map { ch => @@ -352,17 +351,17 @@ class RecordSetChangeHandlerSpec } "fail the change if applying fails with an error" in { - doReturn(Interfaces.result(List())) - .when(mockConn) + doReturn(IO.pure(List())) + .when(mockBackend) .resolve(rs.name, rsChange.zone.name, rs.typ) - doReturn(Interfaces.result(Left(NotAuthorized("dns-fail")))) - .when(mockConn) + doReturn(IO.raiseError(NotAuthorized("dns-fail"))) + .when(mockBackend) .applyChange(rsChange) doReturn(IO.pure(cs)).when(mockChangeRepo).save(any[ChangeSet]) doReturn(IO.pure(cs)).when(mockRsRepo).apply(any[ChangeSet]) doReturn(IO.pure(List.empty)).when(mockRsRepo).getRecordSetsByName(cs.zoneId, rs.name) - val test = underTest.apply(mockConn, rsChange) + val test = underTest.apply(mockBackend, rsChange) test.unsafeRunSync() verify(mockRsRepo).apply(rsRepoCaptor.capture()) @@ -378,8 +377,8 @@ class RecordSetChangeHandlerSpec savedCs.changes.head.status shouldBe RecordSetChangeStatus.Failed // we failed in apply, we should only resolve once - verify(mockConn, times(1)).applyChange(rsChange) - verify(mockConn, times(1)).resolve(rs.name, rsChange.zone.name, rs.typ) + verify(mockBackend, times(1)).applyChange(rsChange) + verify(mockBackend, times(1)).resolve(rs.name, rsChange.zone.name, rs.typ) val batchChangeUpdates = await(batchRepo.getBatchChange(batchChange.id)) val updatedSingleChanges = completeCreateAAAASingleChanges.map { ch => @@ -393,21 +392,6 @@ class RecordSetChangeHandlerSpec batchChangeUpdates.get.changes shouldBe scExpected } - "requeue the change in apply if permissible errors" in { - doReturn(Interfaces.result(List())) - .when(mockConn) - .resolve(rs.name, rsChange.zone.name, rs.typ) - doReturn(Interfaces.result(Left(Refused("dns-fail")))) - .when(mockConn) - .applyChange(rsChange) - doReturn(IO.pure(cs)).when(mockChangeRepo).save(any[ChangeSet]) - doReturn(IO.pure(cs)).when(mockRsRepo).apply(any[ChangeSet]) - doReturn(IO.pure(List.empty)).when(mockRsRepo).getRecordSetsByName(cs.zoneId, rs.name) - - val test = underTest.apply(mockConn, rsChange) - a[Requeue] shouldBe thrownBy(test.unsafeRunSync()) - } - "bypass the validate and verify steps if a wildcard record exists" in { // Return a wildcard record doReturn(IO.pure(List(rsChange.recordSet))) @@ -415,18 +399,18 @@ class RecordSetChangeHandlerSpec .getRecordSets(anyString, anyString, any(classOf[RecordType])) // The second return is for verify - doReturn(Interfaces.result(List())) - .doReturn(Interfaces.result(List(rs))) - .when(mockConn) + doReturn(IO.pure(List())) + .doReturn(IO.pure(List(rs))) + .when(mockBackend) .resolve(rs.name, rsChange.zone.name, rs.typ) - doReturn(Interfaces.result(Right(NoError(mockDnsMessage)))) - .when(mockConn) + doReturn(IO.pure(BackendResponse.NoError("test"))) + .when(mockBackend) .applyChange(rsChange) doReturn(IO.pure(cs)).when(mockChangeRepo).save(any[ChangeSet]) doReturn(IO.pure(cs)).when(mockRsRepo).apply(any[ChangeSet]) - val test = underTest.apply(mockConn, rsChange) + val test = underTest.apply(mockBackend, rsChange) val res = test.unsafeRunSync() res.status shouldBe RecordSetChangeStatus.Complete @@ -445,10 +429,10 @@ class RecordSetChangeHandlerSpec savedCs.changes.head.status shouldBe RecordSetChangeStatus.Complete // make sure the record was applied - verify(mockConn).applyChange(rsChange) + verify(mockBackend).applyChange(rsChange) // make sure we never called resolve, as we skip validate step and verify - verify(mockConn, never).resolve(rs.name, rsChange.zone.name, rs.typ) + verify(mockBackend, never).resolve(rs.name, rsChange.zone.name, rs.typ) val batchChangeUpdates = await(batchRepo.getBatchChange(batchChange.id)) val updatedSingleChanges = completeCreateAAAASingleChanges.map { ch => @@ -474,18 +458,18 @@ class RecordSetChangeHandlerSpec .getRecordSets(rsChange.recordSet.zoneId, "*", RecordType.CNAME) // The second return is for verify - doReturn(Interfaces.result(List())) - .doReturn(Interfaces.result(List(rs))) - .when(mockConn) + doReturn(IO.pure(List())) + .doReturn(IO.pure(List(rs))) + .when(mockBackend) .resolve(rs.name, rsChange.zone.name, rs.typ) - doReturn(Interfaces.result(Right(NoError(mockDnsMessage)))) - .when(mockConn) + doReturn(IO.pure(BackendResponse.NoError("test"))) + .when(mockBackend) .applyChange(rsChange) doReturn(IO.pure(cs)).when(mockChangeRepo).save(any[ChangeSet]) doReturn(IO.pure(cs)).when(mockRsRepo).apply(any[ChangeSet]) - val test = underTest.apply(mockConn, rsChange) + val test = underTest.apply(mockBackend, rsChange) val res = test.unsafeRunSync() res.status shouldBe RecordSetChangeStatus.Complete @@ -504,10 +488,10 @@ class RecordSetChangeHandlerSpec savedCs.changes.head.status shouldBe RecordSetChangeStatus.Complete // make sure the record was applied - verify(mockConn).applyChange(rsChange) + verify(mockBackend).applyChange(rsChange) // make sure we never called resolve, as we skip validate step and verify - verify(mockConn, never).resolve(rs.name, rsChange.zone.name, rs.typ) + verify(mockBackend, never).resolve(rs.name, rsChange.zone.name, rs.typ) val batchChangeUpdates = batchRepo.getBatchChange(batchChange.id).unsafeRunSync() val updatedSingleChanges = completeCreateAAAASingleChanges.map { ch => @@ -532,18 +516,18 @@ class RecordSetChangeHandlerSpec .getRecordSets(anyString, anyString, any(classOf[RecordType])) // The second return is for verify - doReturn(Interfaces.result(Right(List()))) - .doReturn(Interfaces.result(Right(List(rsNs)))) - .when(mockConn) + doReturn(IO.pure(List())) + .doReturn(IO.pure(List(rsNs))) + .when(mockBackend) .resolve(rsNs.name, rsChangeNs.zone.name, rsNs.typ) - doReturn(Interfaces.result(Right(NoError(mockDnsMessage)))) - .when(mockConn) + doReturn(IO.pure(BackendResponse.NoError("test"))) + .when(mockBackend) .applyChange(rsChangeNs) doReturn(IO.pure(csNs)).when(mockChangeRepo).save(any[ChangeSet]) doReturn(IO.pure(csNs)).when(mockRsRepo).apply(any[ChangeSet]) - val test = underTest.apply(mockConn, rsChangeNs) + val test = underTest.apply(mockBackend, rsChangeNs) val res = test.unsafeRunSync() res.status shouldBe RecordSetChangeStatus.Complete @@ -562,10 +546,10 @@ class RecordSetChangeHandlerSpec savedCs.changes.head.status shouldBe RecordSetChangeStatus.Complete // make sure the record was applied - verify(mockConn).applyChange(rsChangeNs) + verify(mockBackend).applyChange(rsChangeNs) // make sure we never called resolve, as we skip validate step and verify - verify(mockConn, never).resolve(rsNs.name, rsChangeNs.zone.name, rsNs.typ) + verify(mockBackend, never).resolve(rsNs.name, rsChangeNs.zone.name, rsNs.typ) } "complete an update successfully if the requested record set change matches the DNS backend" in { @@ -573,11 +557,11 @@ class RecordSetChangeHandlerSpec changeType = RecordSetChangeType.Update, updates = Some(rsChange.recordSet.copy(ttl = 87)) ) - doReturn(Interfaces.result(Right(List(updateChange.recordSet)))) - .when(mockConn) + doReturn(IO.pure(List(updateChange.recordSet))) + .when(mockBackend) .resolve(rsChange.recordSet.name, rsChange.zone.name, rsChange.recordSet.typ) - doReturn(Interfaces.result(Right(NoError(mockDnsMessage)))) - .when(mockConn) + doReturn(IO.pure(BackendResponse.NoError("test"))) + .when(mockBackend) .applyChange(updateChange) doReturn(IO.pure(cs)).when(mockChangeRepo).save(any[ChangeSet]) doReturn(IO.pure(cs)).when(mockRsRepo).apply(any[ChangeSet]) @@ -585,7 +569,7 @@ class RecordSetChangeHandlerSpec .when(mockRsRepo) .getRecordSetsByName(cs.zoneId, rs.name) - val test = underTest.apply(mockConn, updateChange) + val test = underTest.apply(mockBackend, updateChange) test.unsafeRunSync() verify(mockRsRepo).apply(rsRepoCaptor.capture()) @@ -618,17 +602,17 @@ class RecordSetChangeHandlerSpec updates = Some(rsChange.recordSet.copy(ttl = 87)) ) val dnsBackendRs = updateChange.recordSet.copy(ttl = 30) - doReturn(Interfaces.result(Right(List(dnsBackendRs)))) - .when(mockConn) + doReturn(IO.pure(List(dnsBackendRs))) + .when(mockBackend) .resolve(rsChange.recordSet.name, rsChange.zone.name, rsChange.recordSet.typ) - doReturn(Interfaces.result(Right(NoError(mockDnsMessage)))) - .when(mockConn) + doReturn(IO.pure(BackendResponse.NoError("test"))) + .when(mockBackend) .applyChange(updateChange) doReturn(IO.pure(cs)).when(mockChangeRepo).save(any[ChangeSet]) doReturn(IO.pure(cs)).when(mockRsRepo).apply(any[ChangeSet]) doReturn(IO.pure(List(dnsBackendRs))).when(mockRsRepo).getRecordSetsByName(cs.zoneId, rs.name) - val test = underTest.apply(mockConn, updateChange) + val test = underTest.apply(mockBackend, updateChange) test.unsafeRunSync() verify(mockRsRepo).apply(rsRepoCaptor.capture()) @@ -654,8 +638,8 @@ class RecordSetChangeHandlerSpec "getProcessingStatus for Create" should { "return ReadyToApply if there are no records in the DNS backend" in { - doReturn(Interfaces.result(Right(List()))) - .when(mockConn) + doReturn(IO.pure(List())) + .when(mockBackend) .resolve(rs.name, rsChange.zone.name, rs.typ) doReturn(IO.pure(List.empty)).when(mockRsRepo).getRecordSetsByName(cs.zoneId, rs.name) @@ -663,7 +647,7 @@ class RecordSetChangeHandlerSpec RecordSetChangeHandler .syncAndGetProcessingStatusFromDnsBackend( rsChange, - mockConn, + mockBackend, mockRsRepo, mockChangeRepo, true @@ -673,8 +657,8 @@ class RecordSetChangeHandlerSpec } "return AlreadyApplied if the change already exists in the DNS backend" in { - doReturn(Interfaces.result(Right(List(rs)))) - .when(mockConn) + doReturn(IO.pure(List(rs))) + .when(mockBackend) .resolve(rs.name, rsChange.zone.name, rs.typ) doReturn(IO.pure(List(rs))).when(mockRsRepo).getRecordSetsByName(cs.zoneId, rs.name) @@ -682,7 +666,7 @@ class RecordSetChangeHandlerSpec RecordSetChangeHandler .syncAndGetProcessingStatusFromDnsBackend( rsChange, - mockConn, + mockBackend, mockRsRepo, mockChangeRepo, true @@ -692,8 +676,8 @@ class RecordSetChangeHandlerSpec } "remove record from database for Add if record does not exist in DNS backend" in { - doReturn(Interfaces.result(Right(List()))) - .when(mockConn) + doReturn(IO.pure(List())) + .when(mockBackend) .resolve(rs.name, rsChange.zone.name, rs.typ) doReturn(IO.pure(cs)).when(mockChangeRepo).save(any[ChangeSet]) @@ -704,7 +688,7 @@ class RecordSetChangeHandlerSpec RecordSetChangeHandler .syncAndGetProcessingStatusFromDnsBackend( rsChange, - mockConn, + mockBackend, mockRsRepo, mockChangeRepo, true @@ -723,8 +707,8 @@ class RecordSetChangeHandlerSpec val storedRs = rs.copy(ttl = 300) val syncedRsChange = rsChange.copy(changeType = RecordSetChangeType.Update, updates = Some(storedRs)) - doReturn(Interfaces.result(Right(List(syncedRsChange.updates.get)))) - .when(mockConn) + doReturn(IO.pure(List(syncedRsChange.updates.get))) + .when(mockBackend) .resolve(rs.name, rsChange.zone.name, rs.typ) doReturn(IO.pure(List(storedRs))).when(mockRsRepo).getRecordSetsByName(cs.zoneId, rs.name) @@ -732,7 +716,7 @@ class RecordSetChangeHandlerSpec RecordSetChangeHandler .syncAndGetProcessingStatusFromDnsBackend( syncedRsChange, - mockConn, + mockBackend, mockRsRepo, mockChangeRepo, true @@ -742,8 +726,8 @@ class RecordSetChangeHandlerSpec } "return ReadyToApply if current record set doesn't match DNS backend and DNS backend has no records" in { - doReturn(Interfaces.result(Right(List()))) - .when(mockConn) + doReturn(IO.pure(List())) + .when(mockBackend) .resolve(rs.name, rsChange.zone.name, rs.typ) doReturn(IO.pure(List.empty)).when(mockRsRepo).getRecordSetsByName(cs.zoneId, rs.name) @@ -751,7 +735,7 @@ class RecordSetChangeHandlerSpec .syncAndGetProcessingStatusFromDnsBackend( rsChange .copy(changeType = RecordSetChangeType.Update, updates = Some(rs.copy(ttl = 300))), - mockConn, + mockBackend, mockRsRepo, mockChangeRepo, true @@ -761,8 +745,8 @@ class RecordSetChangeHandlerSpec } "return AlreadyApplied if the change already exists in the DNS backend" in { - doReturn(Interfaces.result(Right(List(rsChange.recordSet)))) - .when(mockConn) + doReturn(IO.pure(List(rsChange.recordSet))) + .when(mockBackend) .resolve(rs.name, rsChange.zone.name, rs.typ) doReturn(IO.pure(List(rsChange.recordSet))) .when(mockRsRepo) @@ -771,7 +755,7 @@ class RecordSetChangeHandlerSpec val processorStatus = RecordSetChangeHandler .syncAndGetProcessingStatusFromDnsBackend( rsChange.copy(changeType = RecordSetChangeType.Update), - mockConn, + mockBackend, mockRsRepo, mockChangeRepo, true @@ -781,8 +765,8 @@ class RecordSetChangeHandlerSpec } "sync in the DNS backend for update if record does not exist in database" in { - doReturn(Interfaces.result(Right(List(rs.copy(ttl = 100))))) - .when(mockConn) + doReturn(IO.pure(List(rs.copy(ttl = 100)))) + .when(mockBackend) .resolve(rs.name, rsChange.zone.name, rs.typ) doReturn(IO.pure(cs)).when(mockChangeRepo).save(any[ChangeSet]) @@ -796,7 +780,7 @@ class RecordSetChangeHandlerSpec .syncAndGetProcessingStatusFromDnsBackend( rsChange .copy(changeType = RecordSetChangeType.Update, updates = Some(rs.copy(ttl = 100))), - mockConn, + mockBackend, mockRsRepo, mockChangeRepo, true @@ -812,15 +796,15 @@ class RecordSetChangeHandlerSpec "getProcessingStatus for Delete" should { "return ReadyToApply if there are records in the DNS backend" in { - doReturn(Interfaces.result(Right(List(rs)))) - .when(mockConn) + doReturn(IO.pure(List(rs))) + .when(mockBackend) .resolve(rs.name, rsChange.zone.name, rs.typ) doReturn(IO.pure(List(rs))).when(mockRsRepo).getRecordSetsByName(cs.zoneId, rs.name) val processorStatus = RecordSetChangeHandler .syncAndGetProcessingStatusFromDnsBackend( rsChange.copy(changeType = RecordSetChangeType.Delete), - mockConn, + mockBackend, mockRsRepo, mockChangeRepo, true @@ -830,15 +814,15 @@ class RecordSetChangeHandlerSpec } "return AlreadyApplied if there are no records in the DNS backend" in { - doReturn(Interfaces.result(Right(List()))) - .when(mockConn) + doReturn(IO.pure(List())) + .when(mockBackend) .resolve(rs.name, rsChange.zone.name, rs.typ) doReturn(IO.pure(List.empty)).when(mockRsRepo).getRecordSetsByName(cs.zoneId, rs.name) val processorStatus = RecordSetChangeHandler .syncAndGetProcessingStatusFromDnsBackend( rsChange.copy(changeType = RecordSetChangeType.Delete), - mockConn, + mockBackend, mockRsRepo, mockChangeRepo, true @@ -848,8 +832,8 @@ class RecordSetChangeHandlerSpec } "sync in the DNS backend for Delete change if record exists" in { - doReturn(Interfaces.result(Right(List(rs)))) - .when(mockConn) + doReturn(IO.pure(List(rs))) + .when(mockBackend) .resolve(rs.name, rsChange.zone.name, rs.typ) doReturn(IO.pure(cs)).when(mockChangeRepo).save(any[ChangeSet]) @@ -863,7 +847,7 @@ class RecordSetChangeHandlerSpec .syncAndGetProcessingStatusFromDnsBackend( rsChange .copy(changeType = RecordSetChangeType.Delete), - mockConn, + mockBackend, mockRsRepo, mockChangeRepo, true diff --git a/modules/api/src/test/scala/vinyldns/api/engine/ZoneSyncHandlerSpec.scala b/modules/api/src/test/scala/vinyldns/api/engine/ZoneSyncHandlerSpec.scala index c85162c5f..54ea0b977 100644 --- a/modules/api/src/test/scala/vinyldns/api/engine/ZoneSyncHandlerSpec.scala +++ b/modules/api/src/test/scala/vinyldns/api/engine/ZoneSyncHandlerSpec.scala @@ -29,6 +29,7 @@ import vinyldns.api.VinylDNSTestHelpers import vinyldns.api.domain.record.RecordSetChangeGenerator import vinyldns.api.domain.zone.{DnsZoneViewLoader, VinylDNSZoneViewLoader, ZoneView} import vinyldns.core.domain.Fqdn +import vinyldns.core.domain.backend.{Backend, BackendResolver} import vinyldns.core.domain.record.NameSort.NameSort import vinyldns.core.domain.record.RecordType.RecordType import vinyldns.core.domain.record._ @@ -42,6 +43,8 @@ class ZoneSyncHandlerSpec with BeforeAndAfterEach with VinylDNSTestHelpers { + private val mockBackend = mock[Backend] + private val mockBackendResolver = mock[BackendResolver] private val mockDNSLoader = mock[DnsZoneViewLoader] private val mockVinylDNSLoader = mock[VinylDNSZoneViewLoader] private val recordSetRepo = mock[RecordSetRepository] @@ -151,7 +154,7 @@ class ZoneSyncHandlerSpec recordChangeRepo, zoneChangeRepo, zoneRepo, - _ => mockDNSLoader, + mockBackendResolver, (_, _) => mockVinylDNSLoader ) @@ -159,17 +162,20 @@ class ZoneSyncHandlerSpec recordSetRepo, recordChangeRepo, testZoneChange, - _ => mockDNSLoader, + mockBackendResolver, (_, _) => mockVinylDNSLoader ) override def beforeEach(): Unit = { - reset(recordSetRepo) - reset(recordChangeRepo) - reset(zoneRepo) - reset(zoneChangeRepo) - reset(mockDNSLoader) - reset(mockVinylDNSLoader) + reset( + recordSetRepo, + recordChangeRepo, + zoneRepo, + zoneChangeRepo, + mockDNSLoader, + mockVinylDNSLoader, + mockBackend + ) doReturn( IO(ListRecordSetResults(List(testRecord1), None, None, None, None, None, None, NameSort.ASC)) @@ -190,6 +196,9 @@ class ZoneSyncHandlerSpec doReturn(() => IO(testDnsView)).when(mockDNSLoader).load doReturn(() => IO(testVinylDNSView)).when(mockVinylDNSLoader).load + doReturn(IO.pure(List(testRecord1, testRecord2))) + .when(mockBackend) + .loadZone(any[Zone], any[Int]) } "ZoneSyncHandler" should { @@ -197,6 +206,7 @@ class ZoneSyncHandlerSpec doReturn(IO.pure(Right(testZoneChange))) .when(zoneRepo) .save(any[Zone]) + doReturn(mockBackend).when(mockBackendResolver).resolve(any[Zone]) val result = zoneSync(testZoneChange).unsafeRunSync() @@ -222,6 +232,7 @@ class ZoneSyncHandlerSpec } "process successful zone sync with no changes" in { + doReturn(mockBackend).when(mockBackendResolver).resolve(any[Zone]) doReturn(IO.pure(Right(testZoneChange))) .when(zoneRepo) .save(any[Zone]) @@ -252,6 +263,7 @@ class ZoneSyncHandlerSpec } "handle failed zone sync" in { + doReturn(mockBackend).when(mockBackendResolver).resolve(any[Zone]) doReturn(() => IO.raiseError(new RuntimeException("Dns Failed"))) .when(mockVinylDNSLoader) .load @@ -285,6 +297,7 @@ class ZoneSyncHandlerSpec "saveZoneAndChange" should { "save zone and zoneChange with given statuses" in { + doReturn(mockBackend).when(mockBackendResolver).resolve(any[Zone]) doReturn(IO.pure(Right(testZoneChange))).when(zoneRepo).save(testZoneChange.zone) ZoneSyncHandler.saveZoneAndChange(zoneRepo, zoneChangeRepo, testZoneChange).unsafeRunSync() @@ -300,6 +313,7 @@ class ZoneSyncHandlerSpec } "handle duplicateZoneError" in { + doReturn(mockBackend).when(mockBackendResolver).resolve(any[Zone]) doReturn(IO.pure(Left(DuplicateZoneError("error")))).when(zoneRepo).save(testZoneChange.zone) ZoneSyncHandler.saveZoneAndChange(zoneRepo, zoneChangeRepo, testZoneChange).unsafeRunSync() @@ -317,42 +331,41 @@ class ZoneSyncHandlerSpec "runSync" should { "send the correct zone to the DNSZoneViewLoader" in { + doReturn(mockBackend).when(mockBackendResolver).resolve(any[Zone]) val captor = ArgumentCaptor.forClass(classOf[Zone]) - val dnsLoader = mock[Zone => DnsZoneViewLoader] - doReturn(mockDNSLoader).when(dnsLoader).apply(any[Zone]) - ZoneSyncHandler .runSync( recordSetRepo, recordChangeRepo, testZoneChange, - dnsLoader, + mockBackendResolver, (_, _) => mockVinylDNSLoader ) .unsafeRunSync() - verify(dnsLoader).apply(captor.capture()) + verify(mockBackend).loadZone(captor.capture(), any[Int]) val req = captor.getValue req shouldBe testZone.copy(status = ZoneStatus.Syncing) - } "load the dns zone from DNSZoneViewLoader" in { + doReturn(mockBackend).when(mockBackendResolver).resolve(any[Zone]) ZoneSyncHandler .runSync( recordSetRepo, recordChangeRepo, testZoneChange, - _ => mockDNSLoader, + mockBackendResolver, (_, _) => mockVinylDNSLoader ) .unsafeRunSync() - verify(mockDNSLoader, times(1)).load + verify(mockBackend, times(1)).loadZone(any[Zone], any[Int]) } "Send the correct zone to the VinylDNSZoneViewLoader" in { + doReturn(mockBackend).when(mockBackendResolver).resolve(any[Zone]) val zoneCaptor = ArgumentCaptor.forClass(classOf[Zone]) val repoCaptor = ArgumentCaptor.forClass(classOf[RecordSetRepository]) @@ -364,7 +377,7 @@ class ZoneSyncHandlerSpec recordSetRepo, recordChangeRepo, testZoneChange, - _ => mockDNSLoader, + mockBackendResolver, vinyldnsLoader ) .unsafeRunSync() @@ -375,26 +388,14 @@ class ZoneSyncHandlerSpec } "load the dns zone from VinylDNSZoneViewLoader" in { + doReturn(mockBackend).when(mockBackendResolver).resolve(any[Zone]) runSync.unsafeRunSync() verify(mockVinylDNSLoader, times(1)).load } - "compute the diff correctly" in { - val captor = ArgumentCaptor.forClass(classOf[ZoneView]) - - val testVinylDNSView = mock[ZoneView] - doReturn(List(testRecordSetChange)).when(testVinylDNSView).diff(any[ZoneView]) - doReturn(() => IO(testVinylDNSView)).when(mockVinylDNSLoader).load - - runSync.unsafeRunSync() - - verify(testVinylDNSView).diff(captor.capture()) - val req = captor.getValue - req shouldBe testDnsView - } - "save the record changes to the recordChangeRepo" in { + doReturn(mockBackend).when(mockBackendResolver).resolve(any[Zone]) val captor = ArgumentCaptor.forClass(classOf[ChangeSet]) runSync.unsafeRunSync() @@ -404,6 +405,7 @@ class ZoneSyncHandlerSpec } "save the record sets to the recordSetRepo" in { + doReturn(mockBackend).when(mockBackendResolver).resolve(any[Zone]) val captor = ArgumentCaptor.forClass(classOf[ChangeSet]) runSync.unsafeRunSync() @@ -413,6 +415,7 @@ class ZoneSyncHandlerSpec } "returns the zone as active and sets the latest sync" in { + doReturn(mockBackend).when(mockBackendResolver).resolve(any[Zone]) val testVinylDNSView = ZoneView(testZone, List(testRecord1, testRecord2)) doReturn(() => IO(testVinylDNSView)).when(mockVinylDNSLoader).load val result = runSync.unsafeRunSync() @@ -436,6 +439,7 @@ class ZoneSyncHandlerSpec doReturn(() => IO(testVinylDNSView)).when(mockVinylDNSLoader).load doReturn(IO(correctChangeSet)).when(recordSetRepo).apply(captor.capture()) doReturn(IO(correctChangeSet)).when(recordChangeRepo).save(any[ChangeSet]) + doReturn(mockBackend).when(mockBackendResolver).resolve(any[Zone]) runSync.unsafeRunSync() @@ -457,6 +461,7 @@ class ZoneSyncHandlerSpec doReturn(() => IO(testVinylDNSView)).when(mockVinylDNSLoader).load doReturn(IO(correctChangeSet)).when(recordSetRepo).apply(captor.capture()) doReturn(IO(correctChangeSet)).when(recordChangeRepo).save(any[ChangeSet]) + doReturn(mockBackend).when(mockBackendResolver).resolve(any[Zone]) val zoneChange = ZoneChange(testReverseZone, testReverseZone.account, ZoneChangeType.Sync) @@ -465,7 +470,7 @@ class ZoneSyncHandlerSpec recordSetRepo, recordChangeRepo, zoneChange, - _ => mockDNSLoader, + mockBackendResolver, (_, _) => mockVinylDNSLoader ) .unsafeRunSync() @@ -477,6 +482,7 @@ class ZoneSyncHandlerSpec doReturn(() => IO.raiseError(new RuntimeException("Dns Failed"))) .when(mockVinylDNSLoader) .load + doReturn(mockBackend).when(mockBackendResolver).resolve(any[Zone]) val result = runSync.unsafeRunSync() result.status shouldBe ZoneChangeStatus.Failed diff --git a/modules/core/src/main/scala/vinyldns/core/crypto/NoOpCrypto.scala b/modules/core/src/main/scala/vinyldns/core/crypto/NoOpCrypto.scala index a90774fc7..b4cf06299 100644 --- a/modules/core/src/main/scala/vinyldns/core/crypto/NoOpCrypto.scala +++ b/modules/core/src/main/scala/vinyldns/core/crypto/NoOpCrypto.scala @@ -33,3 +33,7 @@ class NoOpCrypto(val config: Config) extends CryptoAlgebra { def encrypt(value: String): String = value def decrypt(value: String): String = value } + +object NoOpCrypto { + val instance = new NoOpCrypto() +} diff --git a/modules/core/src/main/scala/vinyldns/core/domain/Fqdn.scala b/modules/core/src/main/scala/vinyldns/core/domain/Fqdn.scala index 21b9fb58f..1761425f8 100644 --- a/modules/core/src/main/scala/vinyldns/core/domain/Fqdn.scala +++ b/modules/core/src/main/scala/vinyldns/core/domain/Fqdn.scala @@ -19,6 +19,13 @@ package vinyldns.core.domain import DomainHelpers.{ensureTrailingDot, removeWhitespace} case class Fqdn(fqdn: String) { + + // Everything up to the first dot / period + def firstLabel: String = fqdn.substring(0, fqdn.indexOf('.')) + + // Everything up to the first dot, includes the dot to make it absolute + def firstLabelAbsolute: String = fqdn.substring(0, fqdn.indexOf('.') + 1) + override def equals(obj: Any): Boolean = obj match { case Fqdn(otherFqdn) => otherFqdn.toLowerCase == fqdn.toLowerCase @@ -31,4 +38,22 @@ case class Fqdn(fqdn: String) { case object Fqdn { def apply(fqdn: String): Fqdn = new Fqdn(ensureTrailingDot(removeWhitespace(fqdn))) + + // Combines record name and zone name to create a valid fqdn + def merge(recordName: String, zoneName: String): Fqdn = { + def dropTrailingDot(value: String): String = + if (value.endsWith(".")) value.dropRight(1) else value + + val rname = dropTrailingDot(recordName) + val zname = dropTrailingDot(zoneName) + + val zIndex = rname.lastIndexOf(zname) + if (zIndex > 0) { + // zone name already there, or record name = zone name, so just return + Fqdn(rname + ".") + } else { + // zone name not in record name so combine + Fqdn(s"$rname.$zname.") + } + } } diff --git a/modules/core/src/main/scala/vinyldns/core/domain/backend/Backend.scala b/modules/core/src/main/scala/vinyldns/core/domain/backend/Backend.scala new file mode 100644 index 000000000..5a8c5540a --- /dev/null +++ b/modules/core/src/main/scala/vinyldns/core/domain/backend/Backend.scala @@ -0,0 +1,77 @@ +/* + * 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.domain.backend + +import cats.effect.IO +import vinyldns.core.domain.record.{RecordSet, RecordSetChange} +import vinyldns.core.domain.record.RecordType.RecordType +import vinyldns.core.domain.zone.Zone + +/** + * Provides the backend interface to work with any kind of DNS backend + * + * Implement this interface for your own backend. The default backend is the DnsBackend that talks DDNS. + */ +trait Backend { + + /** + * Identifies this backend + * @return The id for the backend + */ + def id: String + + /** + * Does a lookup for a record given the record name, zone name, and record type + * + * The record name + zone name should form the FQDN + * + * @param name The name of the record (without the zone - e.g. www) + * @param zoneName The full domain name (e.g. example.com) + * @param typ The type of record (e.g. AAAA) + * @return A list of record sets matching the name, empty if not found + */ + def resolve(name: String, zoneName: String, typ: RecordType): IO[List[RecordSet]] + + /** + * Applies a single record set change against the DNS backend + * + * @param change A RecordSetChange to apply. Note: the key for a record set is the record name + type. + * A single RecordSetChange can add or remove multiple individual records in a record set at one time. + * @return A BackendResponse that is backend provider specific + */ + def applyChange(change: RecordSetChange): IO[BackendResponse] + + /** + * Loads all record sets in a zone. Used typically for zone syncs. + * + * Note, this will cause memory issues for large zones (100,000s of records). Need to make + * zone sync memory safe before changing this + * + * @param zone The zone to load + * @param maxZoneSize The maximum number of records that we allow loading, typically configured + * @return All record sets in the zone + */ + def loadZone(zone: Zone, maxZoneSize: Int): IO[List[RecordSet]] + + /** + * Indicates if the zone is present in the backend + * + * @param zone The zone to check if exists + * @return true if it exists; false otherwise + */ + def zoneExists(zone: Zone): IO[Boolean] +} diff --git a/modules/core/src/main/scala/vinyldns/core/domain/backend/BackendConfigs.scala b/modules/core/src/main/scala/vinyldns/core/domain/backend/BackendConfigs.scala new file mode 100644 index 000000000..b14aa21fa --- /dev/null +++ b/modules/core/src/main/scala/vinyldns/core/domain/backend/BackendConfigs.scala @@ -0,0 +1,36 @@ +/* + * 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.domain.backend + +import cats.effect.{Blocker, ContextShift, IO} +import com.typesafe.config.Config +import pureconfig._ +import pureconfig.generic.auto._ +import pureconfig.module.catseffect.syntax._ + +/* The main VinylDNS backend configs, loaded by the BackendRegistry */ +final case class BackendConfigs( + defaultBackendId: String, + backendProviders: List[BackendProviderConfig] +) + +object BackendConfigs { + def load(config: Config)(implicit cs: ContextShift[IO]): IO[BackendConfigs] = + Blocker[IO].use( + ConfigSource.fromConfig(config).loadF[IO, BackendConfigs](_) + ) +} diff --git a/modules/core/src/main/scala/vinyldns/core/domain/backend/BackendLoader.scala b/modules/core/src/main/scala/vinyldns/core/domain/backend/BackendLoader.scala new file mode 100644 index 000000000..aa9920dbc --- /dev/null +++ b/modules/core/src/main/scala/vinyldns/core/domain/backend/BackendLoader.scala @@ -0,0 +1,43 @@ +/* + * 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.domain.backend + +import cats.implicits._ +import cats.effect.IO +import org.slf4j.LoggerFactory + +object BackendLoader { + + private val logger = LoggerFactory.getLogger("BackendLoader") + + def load(configs: List[BackendProviderConfig]): IO[List[BackendProvider]] = { + def loadOne(config: BackendProviderConfig): IO[BackendProvider] = + for { + _ <- IO(logger.error(s"Attempting to load backend ${config.className}")) + provider <- IO( + Class + .forName(config.className) + .getDeclaredConstructor() + .newInstance() + .asInstanceOf[BackendProviderLoader] + ) + backend <- provider.load(config) + } yield backend + + configs.traverse(loadOne) + } +} diff --git a/modules/core/src/main/scala/vinyldns/core/domain/backend/BackendProvider.scala b/modules/core/src/main/scala/vinyldns/core/domain/backend/BackendProvider.scala new file mode 100644 index 000000000..f1179992e --- /dev/null +++ b/modules/core/src/main/scala/vinyldns/core/domain/backend/BackendProvider.scala @@ -0,0 +1,46 @@ +/* + * 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.domain.backend + +import vinyldns.core.domain.zone.Zone + +/** + * Implemented by each provider, provides a means of looking up a `BackendConnection` + * as well as showing which backend ids are registered on this provider + */ +trait BackendProvider { + + /** + * Given a zone, returns a connection to the zone, returns None if cannot connect + * + * @param zone The zone to attempt to connect to + * @return A backend that is usable, or None if it could not connect + */ + def connect(zone: Zone): Option[Backend] + + /** + * Given a backend id, looks up the backend for this provider if it exists + * + * @return A backend that is usable, or None if could not connect + */ + def connectById(backendId: String): Option[Backend] + + /** + * @return The backend ids loaded with this provider + */ + def ids: List[String] +} diff --git a/modules/core/src/main/scala/vinyldns/core/domain/backend/BackendProviderConfig.scala b/modules/core/src/main/scala/vinyldns/core/domain/backend/BackendProviderConfig.scala new file mode 100644 index 000000000..41b70e950 --- /dev/null +++ b/modules/core/src/main/scala/vinyldns/core/domain/backend/BackendProviderConfig.scala @@ -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.core.domain.backend + +import com.typesafe.config.Config + +/** + * Config section loaded for a specific backend + * @param className The fully qualified class name of the provider to be loaded + * @param settings A generic typesafe Config object that holds settings to be interpreted by the provider + */ +final case class BackendProviderConfig( + className: String, + settings: Config +) diff --git a/modules/core/src/main/scala/vinyldns/core/domain/backend/BackendProviderLoader.scala b/modules/core/src/main/scala/vinyldns/core/domain/backend/BackendProviderLoader.scala new file mode 100644 index 000000000..ecf5d6386 --- /dev/null +++ b/modules/core/src/main/scala/vinyldns/core/domain/backend/BackendProviderLoader.scala @@ -0,0 +1,39 @@ +/* + * 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.domain.backend + +import cats.effect.IO + +/** + * To be implemented by other DNS Backend providers. This handles the loading of the backend config, + * typically comprised of multiple connections. + * + * All takes place inside IO, allowing implementers to do anything they need to ready the backend + * for integration with VinylDNS + */ +trait BackendProviderLoader { + + /** + * Loads a backend based on the provided config so that it is ready to use + * This is internally used typically during startup + * + * @param config The BackendConfig, has settings that are specific to this backend + * + * @return A ready-to-use Backend instance, or does an IO.raiseError if something bad occurred. + */ + def load(config: BackendProviderConfig): IO[BackendProvider] +} diff --git a/modules/core/src/main/scala/vinyldns/core/domain/backend/BackendResolver.scala b/modules/core/src/main/scala/vinyldns/core/domain/backend/BackendResolver.scala new file mode 100644 index 000000000..5175bcf5d --- /dev/null +++ b/modules/core/src/main/scala/vinyldns/core/domain/backend/BackendResolver.scala @@ -0,0 +1,109 @@ +/* + * 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.domain.backend + +import cats.data.NonEmptyList +import cats.effect.IO +import cats.implicits._ +import vinyldns.core.domain.zone.Zone +import vinyldns.core.health.HealthCheck +import vinyldns.core.health.HealthCheck.HealthCheck + +/** + * Provides the means to discover backends for zones + */ +trait BackendResolver { + + /** + * Attempts to get the backend for a given zone, falls back to + * using the default-backend-id if no zones can be found + * + * @param zone A `Zone` to get a backend for + * @return A working `Backend`, the default if necessary + */ + def resolve(zone: Zone): Backend + + /** + * Performs whatever health check considered necessary to ensure that the backends are in good health + * + * @param timeout Timeout in seconds to wait before raising an error + * + * @return A HealthCheck that can be run to determine the health of the registered backends + */ + def healthCheck(timeout: Int): HealthCheck + + /** + * Determines if a given backend id is registered + * + * @param backendId The id to lookup + * + * @return true if it is registered; false otherwise + */ + def isRegistered(backendId: String): Boolean + + /** + * @return All of the backend ids registered + */ + def ids: NonEmptyList[String] +} +object BackendResolver { + def apply(configs: BackendConfigs): IO[BackendResolver] = + for { + backends <- BackendLoader.load(configs.backendProviders) + defaultConn <- IO.fromOption( + backends.collectFirstSome(_.connectById(configs.defaultBackendId)) + )( + new RuntimeException( + s"Unable to find default backend for configured id '${configs.defaultBackendId}''" + ) + ) + } yield new BackendResolver { + + /** + * Attempts to get the backend for a given zone, returns `None` if not found + * + * @param zone A `Zone` to get a backend for + * @return A working `Backend`, or `None` if the backend could not be found for this zone + */ + def resolve(zone: Zone): Backend = + backends.collectFirstSome(_.connect(zone)).getOrElse(defaultConn) + + /** + * Performs whatever health check considered necessary to ensure that the backends are in good health + * + * @param timeout Timeout in seconds to wait before raising an error + * @return A HealthCheck that can be run to determine the health of the registered backends + */ + def healthCheck(timeout: Int): HealthCheck = + IO.pure(().asRight[HealthCheck.HealthCheckError]) + + /** + * Determines if a given backend id is registered + * + * @param backendId The id to lookup + * @return true if it is registered; false otherwise + */ + def isRegistered(backendId: String): Boolean = + backends.collectFirstSome(_.connectById(backendId)).isDefined + + /** + * @return All of the backend ids registered + */ + val ids: NonEmptyList[String] = + NonEmptyList(defaultConn.id, backends.toList.flatMap(_.ids)).distinct + } +} diff --git a/modules/core/src/main/scala/vinyldns/core/domain/backend/BackendResponse.scala b/modules/core/src/main/scala/vinyldns/core/domain/backend/BackendResponse.scala new file mode 100644 index 000000000..431f4d05c --- /dev/null +++ b/modules/core/src/main/scala/vinyldns/core/domain/backend/BackendResponse.scala @@ -0,0 +1,27 @@ +/* + * 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.domain.backend + +/* Response from applying a change to a backend */ +sealed trait BackendResponse +object BackendResponse { + /* Indicates the backend request worked properly */ + final case class NoError(message: String) extends BackendResponse + + /* Indicates there was a failure that maybe recoverable with a try again */ + final case class Retry(message: String) extends BackendResponse +} diff --git a/modules/core/src/main/scala/vinyldns/core/domain/record/RecordData.scala b/modules/core/src/main/scala/vinyldns/core/domain/record/RecordData.scala index 467de9986..33e5ceda0 100644 --- a/modules/core/src/main/scala/vinyldns/core/domain/record/RecordData.scala +++ b/modules/core/src/main/scala/vinyldns/core/domain/record/RecordData.scala @@ -19,38 +19,106 @@ package vinyldns.core.domain.record import scodec.bits.ByteVector import vinyldns.core.domain.Fqdn -sealed trait RecordData +import scala.util.Try +import RecordData._ +import vinyldns.core.domain.record.RecordType._ -final case class AData(address: String) extends RecordData +sealed trait RecordData { + def toString: String +} +object RecordData { + def toInt(value: String): Option[Int] = + Try(value.toInt).toOption -final case class AAAAData(address: String) extends RecordData + def toLong(value: String): Option[Long] = + Try(value.toLong).toOption -final case class CNAMEData(cname: Fqdn) extends RecordData + def fromString(value: String, typ: RecordType): Option[RecordData] = typ match { + case A => AData.fromString(value) + case AAAA => AAAAData.fromString(value) + case CNAME => CNAMEData.fromString(value) + case DS => DSData.fromString(value) + case MX => MXData.fromString(value) + case NAPTR => NAPTRData.fromString(value) + case NS => NSData.fromString(value) + case PTR => PTRData.fromString(value) + case SPF => SPFData.fromString(value) + case SRV => SRVData.fromString(value) + case SSHFP => SSHFPData.fromString(value) + case TXT => TXTData.fromString(value) + case UNKNOWN => None + } +} +final case class AData(address: String) extends RecordData { + override def toString: String = address +} +object AData { + def fromString(value: String): Option[AData] = Option(value).map(AData(_)) +} + +final case class AAAAData(address: String) extends RecordData { + override def toString: String = address +} +object AAAAData { + def fromString(value: String): Option[AAAAData] = Option(value).map(AAAAData(_)) +} + +final case class CNAMEData(cname: Fqdn) extends RecordData { + override def toString: String = cname.fqdn +} object CNAMEData { def apply(cname: Fqdn): CNAMEData = new CNAMEData(cname) + + def fromString(value: String): Option[CNAMEData] = + Option(value).map(Fqdn.apply).map(CNAMEData.apply) } -final case class MXData(preference: Integer, exchange: Fqdn) extends RecordData +final case class MXData(preference: Integer, exchange: Fqdn) extends RecordData { + override def toString: String = s"$preference ${exchange.fqdn}" +} object MXData { def apply(preference: Integer, exchange: Fqdn): MXData = new MXData(preference, exchange) + + /* Assumes format preference fqdn, e.g. 10 www.example.com; otherwise returns None */ + def fromString(value: String): Option[MXData] = + Option(value).flatMap { v => + val parts = v.split(' ') + if (parts.length != 2) { + None + } else { + toInt(parts(0)).map { pref => + new MXData(pref, Fqdn(parts(1))) + } + } + } } -final case class NSData(nsdname: Fqdn) extends RecordData +final case class NSData(nsdname: Fqdn) extends RecordData { + override def toString: String = nsdname.fqdn +} object NSData { def apply(nsdname: Fqdn): NSData = new NSData(nsdname) + + def fromString(value: String): Option[NSData] = + Option(value).map(Fqdn.apply).map(NSData.apply) } -final case class PTRData(ptrdname: Fqdn) extends RecordData +final case class PTRData(ptrdname: Fqdn) extends RecordData { + override def toString: String = ptrdname.fqdn +} object PTRData { def apply(ptrdname: Fqdn): PTRData = new PTRData(ptrdname) + + def fromString(value: String): Option[PTRData] = + Option(value).map(Fqdn.apply).map(PTRData.apply) } final case class SOAData( @@ -61,16 +129,67 @@ final case class SOAData( retry: Long, expire: Long, minimum: Long -) extends RecordData +) extends RecordData { + override def toString: String = s"${mname.fqdn} ${rname} $serial $refresh $retry $expire $minimum" +} +object SOAData { + def fromString(value: String): Option[SOAData] = + Option(value).flatMap { v => + val parts = v.split(' ') + if (parts.length != 7) { + None + } else { + for { + serial <- toLong(parts(2)) + refresh <- toLong(parts(3)) + retry <- toLong(parts(4)) + expire <- toLong(parts(5)) + minimum <- toLong(parts(6)) + } yield SOAData( + Fqdn(parts(0)), + parts(1), + serial, + refresh, + retry, + expire, + minimum + ) + } + } +} -final case class SPFData(text: String) extends RecordData +final case class SPFData(text: String) extends RecordData { + override def toString: String = text +} +object SPFData { + def fromString(value: String): Option[SPFData] = Option(value).map(SPFData(_)) +} final case class SRVData(priority: Integer, weight: Integer, port: Integer, target: Fqdn) - extends RecordData + extends RecordData { + override def toString: String = s"$priority $weight $port ${target.fqdn}" +} object SRVData { - def apply(priority: Integer, weight: Integer, port: Integer, target: Fqdn): SRVData = - new SRVData(priority, weight, port, target) + def fromString(value: String): Option[SRVData] = + Option(value).flatMap { v => + val parts = v.split(' ') + if (parts.length != 7) { + None + } else { + for { + priority <- toInt(parts(0)) + weight <- toInt(parts(1)) + port <- toInt(parts(2)) + target = Fqdn(parts(3)) + } yield SRVData( + priority, + weight, + port, + target + ) + } + } } final case class NAPTRData( @@ -80,7 +199,9 @@ final case class NAPTRData( service: String, regexp: String, replacement: Fqdn -) extends RecordData +) extends RecordData { + override def toString: String = s"$order $preference $flags $service $regexp ${replacement.fqdn}" +} object NAPTRData { def apply( @@ -92,11 +213,51 @@ object NAPTRData { replacement: Fqdn ): NAPTRData = new NAPTRData(order, preference, flags, service, regexp, replacement) + + def fromString(value: String): Option[NAPTRData] = + Option(value).flatMap { v => + val parts = v.split(' ') + if (parts.length != 6) { + None + } else { + for { + order <- toInt(parts(0)) + pref <- toInt(parts(1)) + flags = parts(2) + service = parts(3) + reg = parts(4) + rep = Fqdn(parts(5)) + } yield NAPTRData(order, pref, flags, service, reg, rep) + } + } } -final case class SSHFPData(algorithm: Integer, typ: Integer, fingerprint: String) extends RecordData +final case class SSHFPData(algorithm: Integer, typ: Integer, fingerprint: String) + extends RecordData { + override def toString: String = s"$algorithm $typ $fingerprint" +} +object SSHFPData { + def fromString(value: String): Option[SSHFPData] = + Option(value).flatMap { v => + val parts = v.split(' ') + if (parts.length != 3) { + None + } else { + for { + alg <- toInt(parts(0)) + typ <- toInt(parts(1)) + fp = parts(2) + } yield SSHFPData(alg, typ, fp) + } + } +} -final case class TXTData(text: String) extends RecordData +final case class TXTData(text: String) extends RecordData { + override def toString: String = text +} +object TXTData { + def fromString(value: String): Option[TXTData] = Option(value).map(TXTData(_)) +} sealed abstract class DigestType(val value: Int) object DigestType { @@ -159,4 +320,22 @@ final case class DSData( algorithm: DnsSecAlgorithm, digestType: DigestType, //digestid in DNSJava digest: ByteVector -) extends RecordData +) extends RecordData { + override def toString: String = s"$keyTag $algorithm $digestType $digest" +} +object DSData { + def fromString(value: String): Option[DSData] = + Option(value).flatMap { v => + val parts = v.split(' ') + if (parts.length != 3) { + None + } else { + for { + kt <- toInt(parts(0)) + alg <- toInt(parts(1)).map(DnsSecAlgorithm.apply) + dt <- toInt(parts(2)).map(DigestType.apply) + dig <- Some(ByteVector(parts(3).getBytes)) + } yield DSData(kt, alg, dt, dig) + } + } +} diff --git a/modules/core/src/main/scala/vinyldns/core/domain/zone/Zone.scala b/modules/core/src/main/scala/vinyldns/core/domain/zone/Zone.scala index 212f8bb44..d7cb9a7f3 100644 --- a/modules/core/src/main/scala/vinyldns/core/domain/zone/Zone.scala +++ b/modules/core/src/main/scala/vinyldns/core/domain/zone/Zone.scala @@ -148,13 +148,13 @@ case class ZoneConnection(name: String, keyName: String, key: String, primarySer copy(key = crypto.decrypt(key)) } -final case class DnsBackend( +final case class LegacyDnsBackend( id: String, zoneConnection: ZoneConnection, transferConnection: ZoneConnection ) { - def encrypted(crypto: CryptoAlgebra): DnsBackend = copy( + def encrypted(crypto: CryptoAlgebra): LegacyDnsBackend = copy( zoneConnection = zoneConnection.encrypted(crypto), transferConnection = transferConnection.encrypted(crypto) ) @@ -163,5 +163,5 @@ final case class DnsBackend( final case class ConfiguredDnsConnections( defaultZoneConnection: ZoneConnection, defaultTransferConnection: ZoneConnection, - dnsBackends: List[DnsBackend] + dnsBackends: List[LegacyDnsBackend] ) diff --git a/modules/r53/src/it/resources/application.conf b/modules/r53/src/it/resources/application.conf new file mode 100644 index 000000000..418102027 --- /dev/null +++ b/modules/r53/src/it/resources/application.conf @@ -0,0 +1,22 @@ +vinyldns { + backend { + default-backend-id = "r53" + + backend-providers = [ + { + class-name = "vinyldns.route53.backend.Route53BackendProviderLoader" + settings = { + backends = [ + { + id = "test" + access-key = "vinyldnsTest" + secret-key = "notNeededForSnsLocal" + service-endpoint = "http://127.0.0.1:19009" + signing-region = "us-east-1" + } + ] + } + } + ] + } +} diff --git a/modules/r53/src/it/scala/vinyldns/route53/backend/Route53IntegrationSpec.scala b/modules/r53/src/it/scala/vinyldns/route53/backend/Route53IntegrationSpec.scala new file mode 100644 index 000000000..871518e0c --- /dev/null +++ b/modules/r53/src/it/scala/vinyldns/route53/backend/Route53IntegrationSpec.scala @@ -0,0 +1,131 @@ +/* + * 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.route53.backend + +import com.amazonaws.services.route53.model.DeleteHostedZoneRequest +import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach} +import org.scalatest.matchers.should.Matchers +import org.scalatest.wordspec.AnyWordSpec +import vinyldns.core.domain.zone.Zone + +import scala.collection.JavaConverters._ +import org.scalatest.OptionValues._ +import org.scalatest.EitherValues._ +import vinyldns.core.domain.backend.BackendResponse +import vinyldns.core.domain.{Fqdn, record} +import vinyldns.core.domain.record.{RecordSet, RecordType} + +class Route53IntegrationSpec + extends AnyWordSpec + with BeforeAndAfterAll + with BeforeAndAfterEach + with Matchers { + + import vinyldns.core.TestRecordSetData._ + import vinyldns.core.TestZoneData._ + + private val testZone = Zone("example.com.", "test@test.com", backendId = Some("test")) + + override def beforeAll(): Unit = { + deleteZone() + createZone() + } + + private def testConnection: Route53Backend = + Route53Backend + .load( + Route53BackendConfig("test", "access", "secret", "http://127.0.0.1:19009", "us-east-1") + ) + .unsafeRunSync() + + private def deleteZone(): Unit = { + val zoneIds = testConnection.client.listHostedZones().getHostedZones.asScala.map(_.getId).toList + zoneIds.foreach { id => + testConnection.client.deleteHostedZone(new DeleteHostedZoneRequest().withId(id)) + } + } + + private def createZone(): Unit = + testConnection.createZone(testZone).unsafeRunSync() + + private def checkRecordExists(rs: RecordSet, zone: Zone): Unit = { + val resolveResult = + testConnection.resolve(rs.name, zone.name, rs.typ).unsafeRunSync().headOption.value + resolveResult.records should contain theSameElementsAs rs.records + resolveResult.name shouldBe rs.name + resolveResult.ttl shouldBe rs.ttl + resolveResult.typ shouldBe rs.typ + } + + private def checkRecordNotExists(rs: RecordSet, zone: Zone): Unit = + testConnection.resolve(rs.name, zone.name, rs.typ).unsafeRunSync() shouldBe empty + + private def testRecordSet(rs: RecordSet, zone: Zone): Unit = { + val conn = testConnection + val testRecord = rs.copy(zoneId = zone.id) + val change = makeTestAddChange(testRecord, zone, "test-user") + val result = conn.applyChange(change).unsafeRunSync() + result shouldBe a[BackendResponse.NoError] + + // We should be able to resolve now + checkRecordExists(testRecord, zone) + + val del = makePendingTestDeleteChange(testRecord, zone, "test-user") + conn.applyChange(del).unsafeRunSync() + + // Record should not be found + checkRecordNotExists(testRecord, zone) + } + + "Route53 Connections" should { + "return nothing if the zone does not exist" in { + testConnection.resolve("foo", "bar", RecordType.A).unsafeRunSync() shouldBe empty + } + "work for a" in { + testRecordSet(rsOk, testZone) + } + "work for aaaa" in { + testRecordSet(aaaa, testZone) + } + "work for cname" in { + testRecordSet(cname, testZone) + } + "work for naptr" in { + testRecordSet(naptr, testZone) + } + "work for mx" in { + val testMxData = record.MXData(10, Fqdn("mx.example.com.")) + val testMx = mx.copy(records = List(testMxData)) + testRecordSet(testMx, testZone) + } + "work for txt" in { + testRecordSet(txt, testZone) + } + "check if zone exists" in { + val notFound = Zone("blah.foo.", "test@test.com", backendId = Some("test")) + testConnection.zoneExists(notFound).unsafeRunSync() shouldBe false + testConnection.zoneExists(testZone).unsafeRunSync() shouldBe true + } + "fail when applying the change and it does not exist" in { + val testRecord = aaaa + val testZone = okZone + val change = makeTestAddChange(testRecord, testZone, "test-user") + val result = testConnection.applyChange(change).attempt.unsafeRunSync() + result.left.value shouldBe a[Route53BackendResponse.ZoneNotFoundError] + } + } +} diff --git a/modules/r53/src/main/scala/vinyldns/route53/backend/Route53Backend.scala b/modules/r53/src/main/scala/vinyldns/route53/backend/Route53Backend.scala new file mode 100644 index 000000000..f3957456e --- /dev/null +++ b/modules/r53/src/main/scala/vinyldns/route53/backend/Route53Backend.scala @@ -0,0 +1,284 @@ +/* + * 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.route53.backend + +import cats.data.OptionT +import cats.effect.IO +import com.amazonaws.auth.{AWSStaticCredentialsProvider, BasicAWSCredentials} +import com.amazonaws.client.builder.AwsClientBuilder.EndpointConfiguration +import com.amazonaws.handlers.AsyncHandler +import com.amazonaws.services.route53.{AmazonRoute53Async, AmazonRoute53AsyncClientBuilder} +import com.amazonaws.services.route53.model._ +import com.amazonaws.{AmazonWebServiceRequest, AmazonWebServiceResult} +import org.slf4j.LoggerFactory +import vinyldns.core.domain.Fqdn +import vinyldns.core.domain.backend.{Backend, BackendResponse} +import vinyldns.core.domain.record.RecordSetChangeType.RecordSetChangeType +import vinyldns.core.domain.record.RecordType.RecordType +import vinyldns.core.domain.record.{RecordSet, RecordSetChange, RecordSetChangeType} +import vinyldns.core.domain.zone.{Zone, ZoneStatus} + +import scala.collection.JavaConverters._ +import scala.collection.concurrent.TrieMap + +/** + * Backend for a single AWS account + * + * @param id VinylDNS backend identifier used to connect to route 53 + * @param hostedZones A list of hosted zones, loaded when the application is started. Necessary + * as most interactions with Route53 go through the zone id, not the zone name. + * This will be used as a cache, and on cache miss will lookup the zone in real time + * @param client A route 53 client with credentials that can talk to this route 53 aws account + */ +class Route53Backend( + val id: String, + hostedZones: List[HostedZone], + val client: AmazonRoute53Async +) extends Backend + with Route53Conversions { + import Route53Backend.r53 + + private val logger = LoggerFactory.getLogger(classOf[Route53Backend]) + + /* Concurrent friendly map */ + private val zoneMap: TrieMap[String, String] = TrieMap( + hostedZones.map(z => z.getName -> z.getId): _* + ) + + /* Lookup in the local cache, if a new zone is added since start, we have to retrieve it in real time */ + private def lookupHostedZone(zoneName: String): OptionT[IO, String] = { + // pain but we must parse to use the hosted zone ids from the cache + def parseHostedZoneId(hzid: String): String = { + val lastSlash = hzid.lastIndexOf('/') + if (lastSlash > 0) { + hzid.substring(lastSlash + 1) + } else { + hzid + } + } + OptionT.fromOption[IO](zoneMap.get(zoneName)).orElseF { + r53( + new ListHostedZonesByNameRequest().withDNSName(zoneName), + client.listHostedZonesByNameAsync + ).map { result => + // We must parse the hosted zone id which is annoying + val found = result.getHostedZones.asScala.toList.headOption.map { hz => + val hzid = parseHostedZoneId(hz.getId) + + // adds the hozted zone name and id to our cache if not present + zoneMap.putIfAbsent(hz.getName, hzid) + hzid + } + if (found.isEmpty) { + logger.warn(s"Unable to find hosted zone for '$zoneName'") + } + found + } + } + } + + /** + * Does a lookup for a record given the record name, zone name, and record type + * + * The record name + zone name should form the FQDN + * + * @param name The name of the record (without the zone - e.g. www) + * @param zoneName The full domain name (e.g. example.com) + * @param typ The type of record (e.g. AAAA) + * @return A list of record sets matching the name, empty if not found + */ + def resolve(name: String, zoneName: String, typ: RecordType): IO[List[RecordSet]] = { + for { + hostedZoneId <- lookupHostedZone(zoneName) + awsRRType <- OptionT.fromOption[IO](toRoute53RecordType(typ)) + fqdn = Fqdn.merge(name, zoneName).fqdn + result <- OptionT.liftF { + r53( + new ListResourceRecordSetsRequest() + .withHostedZoneId(hostedZoneId) + .withStartRecordName(fqdn) + .withStartRecordType(awsRRType), + client.listResourceRecordSetsAsync + ) + } + } yield toVinylRecordSets(result.getResourceRecordSets, zoneName: String) + }.getOrElse(Nil) + + /** + * Applies a single record set change against the DNS backend + * + * @param change A RecordSetChange to apply. Note: the key for a record set is the record name + type. + * A single RecordSetChange can add or remove multiple individual records in a record set at one time. + * @return A BackendResponse that is backend provider specific + */ + def applyChange(change: RecordSetChange): IO[BackendResponse] = { + def changeAction(typ: RecordSetChangeType): ChangeAction = typ match { + case RecordSetChangeType.Create => ChangeAction.CREATE + case RecordSetChangeType.Update => ChangeAction.UPSERT + case RecordSetChangeType.Delete => ChangeAction.DELETE + } + + def changeRequest( + typ: RecordSetChangeType, + rs: ResourceRecordSet + ): ChangeResourceRecordSetsRequest = { + logger.debug(s"applying change to zone, record set is $rs") + new ChangeResourceRecordSetsRequest().withChangeBatch( + new ChangeBatch().withChanges( + new Change().withAction(changeAction(typ)).withResourceRecordSet(rs) + ) + ) + } + + // We want to FAIL if unrecoverable errors occur so that the change ultimately is marked as failed + for { + hostedZoneId <- lookupHostedZone(change.zone.name).value.flatMap { + case Some(x) => IO(x) + case None => + IO.raiseError( + Route53BackendResponse.ZoneNotFoundError( + s"Unable to find hosted zone for zone name ${change.zone.name}" + ) + ) + } + + r53RecordSet <- IO.fromOption(toR53RecordSet(change.zone, change.recordSet))( + Route53BackendResponse.ConversionError( + s"Unable to convert record set to route 53 format for ${change.recordSet}" + ) + ) + + result <- r53( + changeRequest(change.changeType, r53RecordSet).withHostedZoneId(hostedZoneId), + client.changeResourceRecordSetsAsync + ).map { response => + logger.debug(s"applied record change $change, change result is ${response.getChangeInfo}") + BackendResponse.NoError(response.toString) + } + } yield result + } + + /** + * Loads all record sets in a zone. Used typically for zone syncs. + * + * Note, this will cause memory issues for large zones (100,000s of records). Need to make + * zone sync memory safe before changing this + * + * @param zone The zone to load + * @param maxZoneSize The maximum number of records that we allow loading, typically configured + * @return All record sets in the zone + */ + def loadZone(zone: Zone, maxZoneSize: Int): IO[List[RecordSet]] = { + // Loads a single page, up to 100 record sets + def loadPage(request: ListResourceRecordSetsRequest): IO[ListResourceRecordSetsResult] = + r53( + request, + client.listResourceRecordSetsAsync + ) + + // recursively pages through, exits once we hit the last page + def recurseLoadNextPage( + request: ListResourceRecordSetsRequest, + result: ListResourceRecordSetsResult, + acc: List[RecordSet] + ): IO[List[RecordSet]] = { + val updatedAcc = acc ++ toVinylRecordSets(result.getResourceRecordSets, zone.name) + + // Here is our base case right here, getIsTruncated returns true if there are more records + if (result.getIsTruncated) { + loadPage( + request + .withStartRecordName(result.getNextRecordName) + .withStartRecordType(result.getNextRecordType) + ).flatMap(nextResult => recurseLoadNextPage(request, nextResult, updatedAcc)) + } else { + IO(updatedAcc) + } + } + + for { + hz <- lookupHostedZone(zone.name) + recordSets <- OptionT.liftF { + val req = new ListResourceRecordSetsRequest().withHostedZoneId(hz) + + // recurse to load all pages + loadPage(req).flatMap(recurseLoadNextPage(req, _, Nil)) + } + } yield recordSets + }.getOrElse(Nil) + + /** + * Indicates if the zone is present in the backend + * + * @param zone The zone to check if exists + * @return true if it exists; false otherwise + */ + def zoneExists(zone: Zone): IO[Boolean] = lookupHostedZone(zone.name).isDefined + + /* Note: naive implementation to assist in testing, not meant for production yet */ + def createZone(zone: Zone): IO[Zone] = + for { + result <- r53( + new CreateHostedZoneRequest().withCallerReference(zone.id).withName(zone.name), + client.createHostedZoneAsync + ) + _ <- IO(logger.info(s"create zone result is $result")) + } yield zone.copy(status = ZoneStatus.Active) +} + +object Route53Backend { + + /* Convenience method for working async with AWS */ + def r53[A <: AmazonWebServiceRequest, B <: AmazonWebServiceResult[_]]( + request: A, + f: (A, AsyncHandler[A, B]) => java.util.concurrent.Future[B] + ): IO[B] = + IO.async[B] { complete: (Either[Throwable, B] => Unit) => + val asyncHandler = new AsyncHandler[A, B] { + def onError(exception: Exception): Unit = complete(Left(exception)) + + def onSuccess(request: A, result: B): Unit = complete(Right(result)) + } + + f(request, asyncHandler) + } + + // Loads a Route53 backend + def load(config: Route53BackendConfig): IO[Route53Backend] = { + val clientIO = IO { + AmazonRoute53AsyncClientBuilder.standard + .withEndpointConfiguration( + new EndpointConfiguration(config.serviceEndpoint, config.signingRegion) + ) + .withCredentials( + new AWSStaticCredentialsProvider( + new BasicAWSCredentials(config.accessKey, config.secretKey) + ) + ) + .build() + } + + // Connect to the client AND load the zones + for { + client <- clientIO + result <- r53( + new ListHostedZonesRequest(), + client.listHostedZonesAsync + ) + } yield new Route53Backend(config.id, result.getHostedZones.asScala.toList, client) + } +} diff --git a/modules/r53/src/main/scala/vinyldns/route53/backend/Route53BackendProvider.scala b/modules/r53/src/main/scala/vinyldns/route53/backend/Route53BackendProvider.scala new file mode 100644 index 000000000..08f91783c --- /dev/null +++ b/modules/r53/src/main/scala/vinyldns/route53/backend/Route53BackendProvider.scala @@ -0,0 +1,48 @@ +/* + * 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.route53.backend + +import vinyldns.core.domain.backend.{Backend, BackendProvider} +import vinyldns.core.domain.zone.Zone + +class Route53BackendProvider(connections: List[Route53Backend]) extends BackendProvider { + + private val connMap: Map[String, Route53Backend] = connections.map(c => c.id -> c).toMap + + /** + * Given a zone, returns a connection to the zone, returns None if cannot connect + * + * @param zone The zone to attempt to connect to + * @return A backend that is usable, or None if it could not connect + */ + def connect(zone: Zone): Option[Backend] = + // only way to connect is via backend id right now + zone.backendId.flatMap(connectById) + + /** + * Given a backend id, looks up the backend for this provider if it exists + * + * @return A backend that is usable, or None if could not connect + */ + def connectById(backendId: String): Option[Backend] = + connMap.get(backendId) + + /** + * @return The backend ids loaded with this provider + */ + def ids: List[String] = connMap.keys.toList +} diff --git a/modules/r53/src/main/scala/vinyldns/route53/backend/Route53BackendProviderLoader.scala b/modules/r53/src/main/scala/vinyldns/route53/backend/Route53BackendProviderLoader.scala new file mode 100644 index 000000000..db92e7b5e --- /dev/null +++ b/modules/r53/src/main/scala/vinyldns/route53/backend/Route53BackendProviderLoader.scala @@ -0,0 +1,41 @@ +/* + * 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.route53.backend + +import cats.implicits._ +import cats.effect.{ContextShift, IO} +import vinyldns.core.domain.backend.{BackendProviderConfig, BackendProvider, BackendProviderLoader} + +class Route53BackendProviderLoader extends BackendProviderLoader { + + private implicit val cs: ContextShift[IO] = + IO.contextShift(scala.concurrent.ExecutionContext.global) + + /** + * Loads a backend based on the provided config so that it is ready to use + * This is internally used typically during startup + * + * @param config The BackendConfig, has settings that are specific to this backend + * @return A ready-to-use Backend instance, or does an IO.raiseError if something bad occurred. + */ + def load(config: BackendProviderConfig): IO[BackendProvider] = + Route53ProviderConfig.load(config.settings).flatMap { bec => + bec.backends.traverse(Route53Backend.load).map { conns => + new Route53BackendProvider(conns) + } + } +} diff --git a/modules/r53/src/main/scala/vinyldns/route53/backend/Route53BackendResponse.scala b/modules/r53/src/main/scala/vinyldns/route53/backend/Route53BackendResponse.scala new file mode 100644 index 000000000..c8a242cf2 --- /dev/null +++ b/modules/r53/src/main/scala/vinyldns/route53/backend/Route53BackendResponse.scala @@ -0,0 +1,23 @@ +/* + * 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.route53.backend + +sealed trait Route53BackendResponse +object Route53BackendResponse { + final case class ZoneNotFoundError(message: String) extends Throwable(message) + final case class ConversionError(message: String) extends Throwable(message) +} diff --git a/modules/r53/src/main/scala/vinyldns/route53/backend/Route53Conversions.scala b/modules/r53/src/main/scala/vinyldns/route53/backend/Route53Conversions.scala new file mode 100644 index 000000000..39f5595dd --- /dev/null +++ b/modules/r53/src/main/scala/vinyldns/route53/backend/Route53Conversions.scala @@ -0,0 +1,93 @@ +/* + * 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.route53.backend + +import com.amazonaws.services.route53.model.{RRType, ResourceRecord, ResourceRecordSet} +import org.joda.time.DateTime +import vinyldns.core.domain.Fqdn +import vinyldns.core.domain.record.{RecordData, RecordSet, RecordSetStatus} +import vinyldns.core.domain.record.RecordType.RecordType +import vinyldns.core.domain.record.RecordType._ +import vinyldns.core.domain.zone.Zone + +import scala.collection.JavaConverters._ + +trait Route53Conversions { + + def toRoute53RecordType(typ: RecordType): Option[RRType] = typ match { + case A => Some(RRType.A) + case AAAA => Some(RRType.AAAA) + case CNAME => Some(RRType.CNAME) + case MX => Some(RRType.MX) + case NAPTR => Some(RRType.NAPTR) + case NS => Some(RRType.NS) + case PTR => Some(RRType.PTR) + case SPF => Some(RRType.SPF) + case SRV => Some(RRType.SRV) + case TXT => Some(RRType.TXT) + case _ => None + } + + def toVinylRecordType(typ: RRType): RecordType = typ match { + case RRType.A => A + case RRType.AAAA => AAAA + case RRType.CNAME => CNAME + case RRType.MX => MX + case RRType.NAPTR => NAPTR + case RRType.NS => NS + case RRType.PTR => PTR + case RRType.SPF => SPF + case RRType.SRV => SRV + case RRType.TXT => TXT + case _ => UNKNOWN + } + + def toVinyl(typ: RecordType, resourceRecord: ResourceRecord): Option[RecordData] = + RecordData.fromString(resourceRecord.getValue, typ) + + def toVinylRecordSet(zoneName: String, r53RecordSet: ResourceRecordSet): RecordSet = { + val typ = toVinylRecordType(RRType.fromValue(r53RecordSet.getType)) + RecordSet( + "unknown", + Fqdn.merge(r53RecordSet.getName, zoneName).firstLabel, + typ, + r53RecordSet.getTTL, + RecordSetStatus.Active, + DateTime.now, + Some(DateTime.now), + r53RecordSet.getResourceRecords.asScala.toList.flatMap(toVinyl(typ, _)), + fqdn = Some(r53RecordSet.getName) + ) + } + + def toVinylRecordSets( + r53RecordSets: java.util.List[ResourceRecordSet], + zoneName: String + ): List[RecordSet] = + r53RecordSets.asScala.toList.map(toVinylRecordSet(zoneName, _)) + + def toR53RecordSet(zone: Zone, vinylRecordSet: RecordSet): Option[ResourceRecordSet] = + toRoute53RecordType(vinylRecordSet.typ).map { typ => + new ResourceRecordSet() + .withName(Fqdn.merge(vinylRecordSet.name, zone.name).fqdn) + .withTTL(vinylRecordSet.ttl) + .withType(typ) + .withResourceRecords( + vinylRecordSet.records.map(rd => new ResourceRecord().withValue(rd.toString)).asJava + ) + } +} diff --git a/modules/r53/src/main/scala/vinyldns/route53/backend/Route53ProviderConfig.scala b/modules/r53/src/main/scala/vinyldns/route53/backend/Route53ProviderConfig.scala new file mode 100644 index 000000000..265691524 --- /dev/null +++ b/modules/r53/src/main/scala/vinyldns/route53/backend/Route53ProviderConfig.scala @@ -0,0 +1,40 @@ +/* + * 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.route53.backend + +import cats.effect.{Blocker, ContextShift, IO} +import com.typesafe.config.Config +import pureconfig.ConfigSource +import pureconfig.generic.auto._ +import pureconfig.module.catseffect.syntax.CatsEffectConfigSource + +// TODO: Add delegation set id and VPC options especially wrt CreateZone +final case class Route53BackendConfig( + id: String, + accessKey: String, + secretKey: String, + serviceEndpoint: String, + signingRegion: String +) +final case class Route53ProviderConfig(backends: List[Route53BackendConfig]) +object Route53ProviderConfig { + + def load(config: Config)(implicit cs: ContextShift[IO]): IO[Route53ProviderConfig] = + Blocker[IO].use( + ConfigSource.fromConfig(config).loadF[IO, Route53ProviderConfig](_) + ) +} diff --git a/project/Dependencies.scala b/project/Dependencies.scala index 9e3f82c26..ff39db557 100644 --- a/project/Dependencies.scala +++ b/project/Dependencies.scala @@ -94,6 +94,11 @@ object Dependencies { "com.amazonaws" % "aws-java-sdk-sqs" % awsV withSources() ) + lazy val r53Dependencies = Seq( + "com.amazonaws" % "aws-java-sdk-core" % awsV withSources(), + "com.amazonaws" % "aws-java-sdk-route53" % awsV withSources() + ) + lazy val commonTestDependencies = Seq( "org.scalatest" %% "scalatest" % scalaTestV, "org.scalacheck" %% "scalacheck" % "1.14.3",