2
0
mirror of https://github.com/VinylDNS/vinyldns synced 2025-08-22 02:02:14 +00:00

Add backend provider (#980)

Introduces the concept of a `Backend` into VinylDNS.  This will allow support for any DNS backend in the future, including AwS Route53 for example.  This is consistent with other "provider" things for dynamic loading of classes (Notifier, Repository, Queue, etc.)

The initial implementation builds on what we have already, that is when creating a zone one can choose a `backendId` that is configured in the `application.conf`.  If no `backendId` is specified, we attempt to map like we do today, so the exact same functionality.

We expand that by allowing one to map a `backendId` to a different provider (like aws). 

After this PR:
1. If someone specifies a zone connection on a zone, it will work exactly like it does today, namely go through the `DnsBackend` to connect.
2. If someone specifies a `backendId` when setting up a zone, the naive mapping will take place to map that zone to the `Backend` implementation that is configured with that `backendId`.   For example, if you have configured a backend id `aws` that connects to Route53, and you specify `aws` when connecting the zone, it will connect to it in Route 53 **Note: we still do not support zone create, but that is much closer to reality with this PR, much much**
3. If someone specifies NEITHER, the `defaultBackendId` will be used, which could be on any one of the backend providers configured.

To start, there is a new `vinyldns.core.domain.backend` package that contains the main classes for the system.  In there you will find the following:

- `BackendProvider` - this is to be implemented by each provider.  Adds a means of pre-loading zones, and providing connections to zones. 
- `Backend` - provides connectivity to a particular backend instance.  For example, a particular DNS Authoritative server.  This is where the real work happens of interacting with whatever backend.  For example, `DnsConnection` implements this to send DDNS messages to the DNS system.  Consider this the "main" thing to implement, where the rubber meets the road, the meat and potatoes
- `BackendProviderLoader` - to be implemented by each provider, knows how to load it's single instance `BackendProvider`, as well as possibly pre-loading configured `Backends` or anything else it needs to do to get ready.  It provides a dynamic hook via the `def load` method that is called by the `BackendLoader` to load a specific `Backend`
- `BackendResolver` - the main, default, BackendResolver.  It holds all `BackendProvider` instances loaded via the `BackendLoader` and provides right now a naive lookup mechanism to find `Backend`s.  Really, this is more of a `Router` or `Resolver`, as in the future it could use more advanced techniques to finding connections than right now
- `BackendConfigs` - used by the `BackendRegistry` as the entrypoint into configuration for all backends
- `BackendProviderConfig` - a single backend provider configuration, specifies a `className` that should be the `BackendProviderLoader` implementation to be loaded, and a `settings` that is passed into the `BackendProvider` to load itself.  This is consistent with other providers.
- `BackendResponse` - uniform responses across all providers to the rest of the VinylDNS System

**Workflow**
During initialization of the system:

1. The `BackendResolver` loads the `BackendConfigs` from the application configuration.  This contains configuration for ALL backends
2. The `BackendResolver` utilizes the `BackendLoader` to dynamically load each backend individually.  If any backend cannot be loaded, it will fail.
3. The `BackendLoader` creates a new instance of each `className` for each `BackendConfig`, this points to the `BackendProviderLoader` implementation which takes care of loading the specific `BackendProvider` provided the configuration
4. The `BackendProviderLoader` does any initialization necessary to ensure it is ready.  In the case of `Route53`, it will pre-load and cache all hosted zones that are available for the AWS account that is configured.  For Route53, a single `Route53Backend` is setup right now.  For `DnsBackend`, a connection (server, port, tsig key) is setup for each DNS Authoritative system to integrate with.

During runtime of the system:

1. When anything is needed, the `BackendResolver` is consulted that will determine how to lookup the `Backend` that is needed.  This is done right now by naively scanning all `BackendProvider` instances it has to say "can anyone connect to this zone".  More intelligent discovery rules can be added in the future
2. Once a `Backend` is obtained, any operation can be performed:
    1. `ZoneConnectionValidator` uses `zoneExists` and `loadZone` to validate a zone is usable by VinylDNS
    2. `RecordSetChangeHandler` uses `resolve` and `applyChange` to apply changes to the DNS backend
    3. `ZoneSyncHandler` and `DnsZoneViewLoader` use `loadZone` in order to load records into VinylDNS

**What else is here**

- Provided an implementation of a backend provider for DNS via `Backend`
- Updated all of VinylDNS to use `Backends` instead of hard coded to DNS
- Provided an implementation of a backend provider for AWS Route 53 as an example to follow for other providers


**Example configuration**

```
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"
            }
          ]
        }
      }
    ]
  }
}
```
This commit is contained in:
Paul Cleary 2020-09-30 09:17:32 -04:00 committed by GitHub
parent 20a3708c42
commit a988bcd9a8
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
62 changed files with 2191 additions and 852 deletions

View File

@ -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;" +

View File

@ -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:

View File

@ -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)

View File

@ -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
}
}
}

View File

@ -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 =

View File

@ -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)'"
)
)
}
}
}

View File

@ -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
)
}

View File

@ -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()
)
}
}
}
}

View File

@ -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

View File

@ -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 =

View File

@ -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))

View File

@ -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

View File

@ -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)

View File

@ -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)
}

View File

@ -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
}

View File

@ -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](_)
)
}

View File

@ -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
)
}
}
}

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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._

View File

@ -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._

View File

@ -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 {

View File

@ -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(())

View File

@ -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 {

View File

@ -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)
}

View File

@ -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

View File

@ -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}

View File

@ -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)
}
}

View File

@ -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 {

View File

@ -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()
)

View File

@ -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")
}

View File

@ -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

View File

@ -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
}
}

View File

@ -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
}
}

View File

@ -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(

View File

@ -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
}
}
}
}

View File

@ -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(

View File

@ -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()

View File

@ -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

View File

@ -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

View File

@ -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()
}

View File

@ -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.")
}
}
}

View File

@ -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]
}

View File

@ -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](_)
)
}

View File

@ -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)
}
}

View File

@ -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]
}

View File

@ -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
)

View File

@ -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]
}

View File

@ -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
}
}

View File

@ -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
}

View File

@ -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)
}
}
}

View File

@ -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]
)

View File

@ -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"
}
]
}
}
]
}
}

View File

@ -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]
}
}
}

View File

@ -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)
}
}

View File

@ -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
}

View File

@ -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)
}
}
}

View File

@ -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)
}

View File

@ -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
)
}
}

View File

@ -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](_)
)
}

View File

@ -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",