diff --git a/modules/api/src/it/resources/application.conf b/modules/api/src/it/resources/application.conf index a64eddac2..dd3b7006e 100644 --- a/modules/api/src/it/resources/application.conf +++ b/modules/api/src/it/resources/application.conf @@ -163,6 +163,28 @@ vinyldns { "ns1.parent.com4." ] + # approved zones, individual users, users in groups, record types and no.of.dots that are allowed for dotted hosts + dotted-hosts = { + # for local testing + allowed-settings = [ + { + zone = "*mmy." + user-list = ["testuser"] + group-list = ["dummy-group"] + record-types = ["AAAA"] + dots-limit = 3 + }, + { + # for wildcard zones. Settings will be applied to all matching zones + zone = "parent.com." + user-list = ["professor", "testuser"] + group-list = ["testing-group"] + record-types = ["A", "CNAME"] + dots-limit = 3 + } + ] + } + # Note: This MUST match the Portal or strange errors will ensue, NoOpCrypto should not be used for production crypto { type = "vinyldns.core.crypto.NoOpCrypto" diff --git a/modules/api/src/it/scala/vinyldns/api/domain/record/RecordSetServiceIntegrationSpec.scala b/modules/api/src/it/scala/vinyldns/api/domain/record/RecordSetServiceIntegrationSpec.scala index df845b477..c789d8305 100644 --- a/modules/api/src/it/scala/vinyldns/api/domain/record/RecordSetServiceIntegrationSpec.scala +++ b/modules/api/src/it/scala/vinyldns/api/domain/record/RecordSetServiceIntegrationSpec.scala @@ -32,7 +32,6 @@ import vinyldns.api.domain.access.AccessValidations import vinyldns.api.domain.zone._ import vinyldns.api.engine.TestMessageQueue import vinyldns.mysql.TransactionProvider -import vinyldns.core.TestMembershipData._ import vinyldns.core.TestZoneData.testConnection import vinyldns.core.domain.{Fqdn, HighValueDomainError} import vinyldns.core.domain.auth.AuthPrincipal @@ -64,13 +63,16 @@ class RecordSetServiceIntegrationSpec private var testRecordSetService: RecordSetServiceAlgebra = _ private val user = User("live-test-user", "key", "secret") + private val testUser = User("testuser", "key", "secret") private val user2 = User("shared-record-test-user", "key-shared", "secret-shared") private val group = Group(s"test-group", "test@test.com", adminUserIds = Set(user.id)) + private val dummyGroup = Group(s"dummy-group", "test@test.com", adminUserIds = Set(testUser.id)) private val group2 = Group(s"test-group", "test@test.com", adminUserIds = Set(user.id, user2.id)) private val sharedGroup = Group(s"test-shared-group", "test@test.com", adminUserIds = Set(user.id, user2.id)) private val auth = AuthPrincipal(user, Seq(group.id, sharedGroup.id)) private val auth2 = AuthPrincipal(user2, Seq(sharedGroup.id, group2.id)) + val dummyAuth: AuthPrincipal = AuthPrincipal(testUser, Seq(dummyGroup.id)) private val zone = Zone( s"live-zone-test.", @@ -167,6 +169,14 @@ class RecordSetServiceIntegrationSpec adminGroupId = group.id ) + private val dummyZone = Zone( + s"dummy.", + "test@test.com", + status = ZoneStatus.Active, + connection = testConnection, + adminGroupId = dummyGroup.id + ) + private val highValueDomainRecord = RecordSet( zone.id, "high-value-domain-existing", @@ -254,8 +264,8 @@ class RecordSetServiceIntegrationSpec groupRepo.save(db, group) } - List(group, group2, sharedGroup).traverse(g => saveGroupData(groupRepo, g).void).unsafeRunSync() - List(zone, zoneTestNameConflicts, zoneTestAddRecords, sharedZone) + List(group, group2, sharedGroup, dummyGroup).traverse(g => saveGroupData(groupRepo, g).void).unsafeRunSync() + List(zone, dummyZone, zoneTestNameConflicts, zoneTestAddRecords, sharedZone) .traverse( z => zoneRepo.save(z) ) @@ -300,6 +310,7 @@ class RecordSetServiceIntegrationSpec mockBackendResolver, false, vinyldnsConfig.highValueDomainConfig, + vinyldnsConfig.dottedHostsConfig, vinyldnsConfig.serverConfig.approvedNameServers, useRecordSetCache = true ) @@ -338,6 +349,70 @@ class RecordSetServiceIntegrationSpec .name shouldBe "zone-test-add-records." } + "create dotted record fails if it doesn't satisfy dotted hosts config" in { + val newRecord = RecordSet( + zoneTestAddRecords.id, + "test.dot", + A, + 38400, + RecordSetStatus.Active, + DateTime.now, + None, + List(AData("10.1.1.1")) + ) + val result = + testRecordSetService + .addRecordSet(newRecord, auth) + .value + .unsafeRunSync() + leftValue(result) shouldBe a[InvalidRequest] + } + + "create dotted record succeeds if it satisfies all dotted hosts config" in { + val newRecord = RecordSet( + dummyZone.id, + "test.dotted", + AAAA, + 38400, + RecordSetStatus.Active, + DateTime.now, + None, + List(AAAAData("fd69:27cc:fe91::60")) + ) + // succeeds as zone, user and record type is allowed as defined in application.conf + val result = + testRecordSetService + .addRecordSet(newRecord, dummyAuth) + .value + .unsafeRunSync() + rightValue(result) + .asInstanceOf[RecordSetChange] + .recordSet + .name shouldBe "test.dotted" + } + + "fail creating dotted record if it satisfies all dotted hosts config except dots-limit for the zone" in { + val newRecord = RecordSet( + dummyZone.id, + "test.dotted.more.dots.than.allowed", + AAAA, + 38400, + RecordSetStatus.Active, + DateTime.now, + None, + List(AAAAData("fd69:27cc:fe91::60")) + ) + + // The number of dots allowed in the record name for this zone as defined in the config is 3. + // Creating with 4 dots results in an error + val result = + testRecordSetService + .addRecordSet(newRecord, dummyAuth) + .value + .unsafeRunSync() + leftValue(result) shouldBe a[InvalidRequest] + } + "update apex A record and add trailing dot" in { val newRecord = apexTestRecordA.copy(ttl = 200) val result = testRecordSetService diff --git a/modules/api/src/main/resources/application.conf b/modules/api/src/main/resources/application.conf index a8aa7175c..7407c07e4 100644 --- a/modules/api/src/main/resources/application.conf +++ b/modules/api/src/main/resources/application.conf @@ -165,6 +165,19 @@ vinyldns { "ns1.parent.com4." ] + # approved zones, individual users, users in groups, record types and no.of.dots that are allowed for dotted hosts + dotted-hosts = { + allowed-settings = [ + { + zone = "zonenamehere." + user-list = [] + group-list = [] + record-types = [] + dots-limit = 0 + } + ] + } + # Note: This MUST match the Portal or strange errors will ensue, NoOpCrypto should not be used for production crypto { type = "vinyldns.core.crypto.NoOpCrypto" diff --git a/modules/api/src/main/resources/reference.conf b/modules/api/src/main/resources/reference.conf index b82cb74d6..f7a74e2f9 100644 --- a/modules/api/src/main/resources/reference.conf +++ b/modules/api/src/main/resources/reference.conf @@ -90,6 +90,29 @@ vinyldns { "ns1.parent.com." ] + # approved zones, individual users, users in groups, record types and no.of.dots that are allowed for dotted hosts + dotted-hosts = { + # for local testing + allowed-settings = [ + { + # for wildcard zones. Settings will be applied to all matching zones + zone = "*ent.com*." + user-list = ["ok"] + group-list = ["dummy-group"] + record-types = ["CNAME"] + dots-limit = 3 + }, + { + # for wildcard zones. Settings will be applied to all matching zones + zone = "dummy*." + user-list = ["sharedZoneUser"] + group-list = ["history-group1"] + record-types = ["A"] + dots-limit = 3 + } + ] + } + # color should be green or blue, used in order to do blue/green deployment color = "green" diff --git a/modules/api/src/main/scala/vinyldns/api/Boot.scala b/modules/api/src/main/scala/vinyldns/api/Boot.scala index 24db9b737..ae4077f49 100644 --- a/modules/api/src/main/scala/vinyldns/api/Boot.scala +++ b/modules/api/src/main/scala/vinyldns/api/Boot.scala @@ -139,6 +139,7 @@ object Boot extends App { backendResolver, vinyldnsConfig.serverConfig.validateRecordLookupAgainstDnsBackend, vinyldnsConfig.highValueDomainConfig, + vinyldnsConfig.dottedHostsConfig, vinyldnsConfig.serverConfig.approvedNameServers, vinyldnsConfig.serverConfig.useRecordSetCache ) diff --git a/modules/api/src/main/scala/vinyldns/api/config/DottedHostsConfig.scala b/modules/api/src/main/scala/vinyldns/api/config/DottedHostsConfig.scala new file mode 100644 index 000000000..a5d1e546d --- /dev/null +++ b/modules/api/src/main/scala/vinyldns/api/config/DottedHostsConfig.scala @@ -0,0 +1,31 @@ +/* + * 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.config + +import pureconfig.ConfigReader +import pureconfig.generic.auto._ + +final case class ZoneAuthConfigs(zone: String, userList: List[String], groupList: List[String], recordTypes: List[String], dotsLimit: Int) +final case class DottedHostsConfig(zoneAuthConfigs: List[ZoneAuthConfigs]) + +object DottedHostsConfig { + implicit val configReader: ConfigReader[DottedHostsConfig] = + ConfigReader.forProduct1[DottedHostsConfig, List[ZoneAuthConfigs]]( + "allowed-settings", + )(zoneAuthConfigs => + DottedHostsConfig(zoneAuthConfigs)) +} diff --git a/modules/api/src/main/scala/vinyldns/api/config/VinylDNSConfig.scala b/modules/api/src/main/scala/vinyldns/api/config/VinylDNSConfig.scala index abb75c4be..dd41bb3a0 100644 --- a/modules/api/src/main/scala/vinyldns/api/config/VinylDNSConfig.scala +++ b/modules/api/src/main/scala/vinyldns/api/config/VinylDNSConfig.scala @@ -47,6 +47,7 @@ final case class VinylDNSConfig( notifierConfigs: List[NotifierConfig], dataStoreConfigs: List[DataStoreConfig], backendConfigs: BackendConfigs, + dottedHostsConfig: DottedHostsConfig, configuredDnsConnections: ConfiguredDnsConnections, apiMetricSettings: APIMetricsSettings, crypto: CryptoAlgebra, @@ -85,6 +86,7 @@ object VinylDNSConfig { serverConfig <- loadIO[ServerConfig](config, "vinyldns") batchChangeConfig <- loadIO[BatchChangeConfig](config, "vinyldns") backendConfigs <- loadIO[BackendConfigs](config, "vinyldns.backend") + dottedHostsConfig <- loadIO[DottedHostsConfig](config, "vinyldns.dotted-hosts") httpConfig <- loadIO[HttpConfig](config, "vinyldns.rest") hvdConfig <- loadIO[HighValueDomainConfig](config, "vinyldns.high-value-domains") scheduledChangesConfig <- loadIO[ScheduledChangesConfig](config, "vinyldns") @@ -110,6 +112,7 @@ object VinylDNSConfig { notifierConfigs, dataStoreConfigs, backendConfigs, + dottedHostsConfig, connections, metricSettings, crypto, diff --git a/modules/api/src/main/scala/vinyldns/api/domain/batch/BatchChangeService.scala b/modules/api/src/main/scala/vinyldns/api/domain/batch/BatchChangeService.scala index 2bccc1699..ae2ff56c4 100644 --- a/modules/api/src/main/scala/vinyldns/api/domain/batch/BatchChangeService.scala +++ b/modules/api/src/main/scala/vinyldns/api/domain/batch/BatchChangeService.scala @@ -32,19 +32,8 @@ import vinyldns.core.domain.auth.AuthPrincipal import vinyldns.core.domain.batch.BatchChangeApprovalStatus.BatchChangeApprovalStatus import vinyldns.core.domain.batch._ import vinyldns.core.domain.batch.BatchChangeApprovalStatus._ -import vinyldns.core.domain.{ - CnameAtZoneApexError, - SingleChangeError, - UserIsNotAuthorizedError, - ZoneDiscoveryError -} -import vinyldns.core.domain.membership.{ - Group, - GroupRepository, - ListUsersResults, - User, - UserRepository -} +import vinyldns.core.domain.{CnameAtZoneApexError, SingleChangeError, UserIsNotAuthorizedError, ZoneDiscoveryError} +import vinyldns.core.domain.membership.{Group, GroupRepository, ListUsersResults, User, UserRepository} import vinyldns.core.domain.record.RecordType._ import vinyldns.core.domain.record.RecordSetRepository import vinyldns.core.domain.zone.ZoneRepository diff --git a/modules/api/src/main/scala/vinyldns/api/domain/batch/BatchChangeValidations.scala b/modules/api/src/main/scala/vinyldns/api/domain/batch/BatchChangeValidations.scala index 0ed20d0cf..27ad124ed 100644 --- a/modules/api/src/main/scala/vinyldns/api/domain/batch/BatchChangeValidations.scala +++ b/modules/api/src/main/scala/vinyldns/api/domain/batch/BatchChangeValidations.scala @@ -45,10 +45,10 @@ trait BatchChangeValidationsAlgebra { ): ValidatedBatch[ChangeInput] def validateChangesWithContext( - groupedChanges: ChangeForValidationMap, - auth: AuthPrincipal, - isApproved: Boolean, - batchOwnerGroupId: Option[String] + groupedChanges: ChangeForValidationMap, + auth: AuthPrincipal, + isApproved: Boolean, + batchOwnerGroupId: Option[String] ): ValidatedBatch[ChangeForValidation] def canGetBatchChange( @@ -273,17 +273,17 @@ class BatchChangeValidations( /* context validations */ def validateChangesWithContext( - groupedChanges: ChangeForValidationMap, - auth: AuthPrincipal, - isApproved: Boolean, - batchOwnerGroupId: Option[String] + groupedChanges: ChangeForValidationMap, + auth: AuthPrincipal, + isApproved: Boolean, + batchOwnerGroupId: Option[String] ): ValidatedBatch[ChangeForValidation] = - // Updates are a combination of an add and delete for a record with the same name and type in a zone. + // Updates are a combination of an add and delete for a record with the same name and type in a zone. groupedChanges.changes.mapValid { case add: AddChangeForValidation - if groupedChanges - .getLogicalChangeType(add.recordKey) - .contains(LogicalChangeType.Add) => + if groupedChanges + .getLogicalChangeType(add.recordKey) + .contains(LogicalChangeType.Add) => validateAddWithContext(add, groupedChanges, auth, isApproved, batchOwnerGroupId) case addUpdate: AddChangeForValidation => validateAddUpdateWithContext(addUpdate, groupedChanges, auth, isApproved, batchOwnerGroupId) @@ -409,11 +409,11 @@ class BatchChangeValidations( } def validateAddWithContext( - change: AddChangeForValidation, - groupedChanges: ChangeForValidationMap, - auth: AuthPrincipal, - isApproved: Boolean, - ownerGroupId: Option[String] + change: AddChangeForValidation, + groupedChanges: ChangeForValidationMap, + auth: AuthPrincipal, + isApproved: Boolean, + ownerGroupId: Option[String] ): SingleValidation[ChangeForValidation] = { val typedValidations = change.inputChange.typ match { case A | AAAA | MX => diff --git a/modules/api/src/main/scala/vinyldns/api/domain/record/RecordSetService.scala b/modules/api/src/main/scala/vinyldns/api/domain/record/RecordSetService.scala index cc4f18331..0a38bae89 100644 --- a/modules/api/src/main/scala/vinyldns/api/domain/record/RecordSetService.scala +++ b/modules/api/src/main/scala/vinyldns/api/domain/record/RecordSetService.scala @@ -28,7 +28,7 @@ import vinyldns.core.queue.MessageQueue import cats.data._ import cats.effect.IO import org.xbill.DNS.ReverseMap -import vinyldns.api.config.HighValueDomainConfig +import vinyldns.api.config.{ZoneAuthConfigs, DottedHostsConfig, HighValueDomainConfig} import vinyldns.api.domain.DomainValidations.{validateIpv4Address, validateIpv6Address} import vinyldns.api.domain.access.AccessValidationsAlgebra import vinyldns.core.domain.record.NameSort.NameSort @@ -46,6 +46,7 @@ object RecordSetService { backendResolver: BackendResolver, validateRecordLookupAgainstDnsBackend: Boolean, highValueDomainConfig: HighValueDomainConfig, + dottedHostsConfig: DottedHostsConfig, approvedNameServers: List[Regex], useRecordSetCache: Boolean ): RecordSetService = @@ -61,6 +62,7 @@ object RecordSetService { backendResolver, validateRecordLookupAgainstDnsBackend, highValueDomainConfig, + dottedHostsConfig, approvedNameServers, useRecordSetCache ) @@ -78,6 +80,7 @@ class RecordSetService( backendResolver: BackendResolver, validateRecordLookupAgainstDnsBackend: Boolean, highValueDomainConfig: HighValueDomainConfig, + dottedHostsConfig: DottedHostsConfig, approvedNameServers: List[Regex], useRecordSetCache: Boolean ) extends RecordSetServiceAlgebra { @@ -88,6 +91,7 @@ class RecordSetService( def addRecordSet(recordSet: RecordSet, auth: AuthPrincipal): Result[ZoneCommandResult] = for { zone <- getZone(recordSet.zoneId) + authZones = dottedHostsConfig.zoneAuthConfigs.map(x => x.zone) change <- RecordSetChangeGenerator.forAdd(recordSet, zone, Some(auth)).toResult // because changes happen to the RS in forAdd itself, converting 1st and validating on that rsForValidations = change.recordSet @@ -107,13 +111,27 @@ class RecordSetService( ownerGroup <- getGroupIfProvided(rsForValidations.ownerGroupId) _ <- canUseOwnerGroup(rsForValidations.ownerGroupId, ownerGroup, auth).toResult _ <- noCnameWithNewName(rsForValidations, existingRecordsWithName, zone).toResult + allowedZoneList <- getAllowedZones(authZones).toResult[Set[String]] + isInAllowedUsers = checkIfInAllowedUsers(zone, dottedHostsConfig, auth) + isUserInAllowedGroups <- checkIfInAllowedGroups(zone, dottedHostsConfig, auth).toResult[Boolean] + isAllowedUser = isInAllowedUsers || isUserInAllowedGroups + isRecordTypeAllowed = checkIfInAllowedRecordType(zone, dottedHostsConfig, rsForValidations) + isRecordTypeAndUserAllowed = isAllowedUser && isRecordTypeAllowed + allowedDotsLimit = getAllowedDotsLimit(zone, dottedHostsConfig) + recordFqdnDoesNotAlreadyExist <- recordFQDNDoesNotExist(rsForValidations, zone).toResult[Boolean] _ <- typeSpecificValidations( rsForValidations, existingRecordsWithName, zone, None, - approvedNameServers + approvedNameServers, + recordFqdnDoesNotAlreadyExist, + allowedZoneList, + isRecordTypeAndUserAllowed, + allowedDotsLimit ).toResult + _ <- if(allowedZoneList.contains(zone.name)) checkAllowedDots(allowedDotsLimit, rsForValidations, zone).toResult else ().toResult + _ <- if(allowedZoneList.contains(zone.name)) isNotApexEndsWithDot(rsForValidations, zone).toResult else ().toResult _ <- messageQueue.send(change).toResult[Unit] } yield change @@ -143,13 +161,28 @@ class RecordSetService( validateRecordLookupAgainstDnsBackend ) _ <- noCnameWithNewName(rsForValidations, existingRecordsWithName, zone).toResult + authZones = dottedHostsConfig.zoneAuthConfigs.map(x => x.zone) + allowedZoneList <- getAllowedZones(authZones).toResult[Set[String]] + isInAllowedUsers = checkIfInAllowedUsers(zone, dottedHostsConfig, auth) + isUserInAllowedGroups <- checkIfInAllowedGroups(zone, dottedHostsConfig, auth).toResult[Boolean] + isAllowedUser = isInAllowedUsers || isUserInAllowedGroups + isRecordTypeAllowed = checkIfInAllowedRecordType(zone, dottedHostsConfig, rsForValidations) + isRecordTypeAndUserAllowed = isAllowedUser && isRecordTypeAllowed + allowedDotsLimit = getAllowedDotsLimit(zone, dottedHostsConfig) + recordFqdnDoesNotAlreadyExist <- recordFQDNDoesNotExist(rsForValidations, zone).toResult[Boolean] _ <- typeSpecificValidations( rsForValidations, existingRecordsWithName, zone, Some(existing), - approvedNameServers + approvedNameServers, + recordFqdnDoesNotAlreadyExist, + allowedZoneList, + isRecordTypeAndUserAllowed, + allowedDotsLimit ).toResult + _ <- if(existing.name == rsForValidations.name) ().toResult else if(allowedZoneList.contains(zone.name)) checkAllowedDots(allowedDotsLimit, rsForValidations, zone).toResult else ().toResult + _ <- if(allowedZoneList.contains(zone.name)) isNotApexEndsWithDot(rsForValidations, zone).toResult else ().toResult _ <- messageQueue.send(change).toResult[Unit] } yield change @@ -169,6 +202,178 @@ class RecordSetService( _ <- messageQueue.send(change).toResult[Unit] } yield change + // For dotted hosts. Check if a record that may conflict with dotted host exist or not + def recordFQDNDoesNotExist(newRecordSet: RecordSet, zone: Zone): IO[Boolean] = { + // Use fqdn for searching through `recordset` mysql table to see if it already exist + val newRecordFqdn = if(newRecordSet.name != zone.name) newRecordSet.name + "." + zone.name else newRecordSet.name + + for { + record <- recordSetRepository.getRecordSetsByFQDNs(Set(newRecordFqdn)) + isRecordAlreadyExist = doesRecordWithSameTypeExist(record, newRecordSet) + doesNotExist = if(isRecordAlreadyExist) false else true + } yield doesNotExist + } + + // Check if a record with same type already exist in 'recordset' mysql table + def doesRecordWithSameTypeExist(oldRecord: List[RecordSet], newRecord: RecordSet): Boolean = { + if(oldRecord.nonEmpty) { + val typeExists = oldRecord.map(x => x.typ == newRecord.typ) + if (typeExists.contains(true)) true else false + } + else { + false + } + } + + // Get zones that are allowed to create dotted hosts using the zones present in dotted hosts config + def getAllowedZones(zones: List[String]): IO[Set[String]] = { + if(zones.isEmpty){ + val noZones: IO[Set[String]] = IO(Set.empty) + noZones + } + else { + // Wildcard zones needs to be passed to a separate method + val wildcardZones = zones.filter(_.contains("*")).map(_.replace("*", "%")) + // Zones without wildcard character are passed to a separate function + val namedZones = zones.filter(zone => !zone.contains("*")) + for{ + namedZoneResult <- zoneRepository.getZonesByNames(namedZones.toSet) + wildcardZoneResult <- zoneRepository.getZonesByFilters(wildcardZones.toSet) + zoneResult = namedZoneResult ++ wildcardZoneResult // Combine the zones + } yield zoneResult.map(x => x.name) + } + } + + // Check if user is allowed to create dotted hosts using the users present in dotted hosts config + def getAllowedDotsLimit(zone: Zone, config: DottedHostsConfig): Int = { + val configZones = config.zoneAuthConfigs.map(x => x.zone) + val zoneName = if(zone.name.takeRight(1) != ".") zone.name + "." else zone.name + val dottedZoneConfig = configZones.filter(_.contains("*")).map(_.replace("*", "[A-Za-z0-9.]*")) + val isContainWildcardZone = dottedZoneConfig.exists(x => zoneName.matches(x)) + val isContainNormalZone = configZones.contains(zoneName) + if(isContainNormalZone){ + config.zoneAuthConfigs.filter(x => x.zone == zoneName).head.dotsLimit + } + else if(isContainWildcardZone){ + config.zoneAuthConfigs.filter(x => zoneName.matches(x.zone.replace("*", "[A-Za-z0-9.]*"))).head.dotsLimit + } + else { + 0 + } + } + + // Check if user is allowed to create dotted hosts using the users present in dotted hosts config + def checkIfInAllowedUsers(zone: Zone, config: DottedHostsConfig, auth: AuthPrincipal): Boolean = { + val configZones = config.zoneAuthConfigs.map(x => x.zone) + val zoneName = if(zone.name.takeRight(1) != ".") zone.name + "." else zone.name + val dottedZoneConfig = configZones.filter(_.contains("*")).map(_.replace("*", "[A-Za-z0-9.]*")) + val isContainWildcardZone = dottedZoneConfig.exists(x => zoneName.matches(x)) + val isContainNormalZone = configZones.contains(zoneName) + if(isContainNormalZone){ + val users = config.zoneAuthConfigs.flatMap { + x: ZoneAuthConfigs => + if (x.zone == zoneName) x.userList else List.empty + } + if(users.contains(auth.signedInUser.userName)){ + true + } + else { + false + } + } + else if(isContainWildcardZone){ + val users = config.zoneAuthConfigs.flatMap { + x: ZoneAuthConfigs => + if (x.zone.contains("*")) { + val wildcardZone = x.zone.replace("*", "[A-Za-z0-9.]*") + if (zoneName.matches(wildcardZone)) x.userList else List.empty + } else List.empty + } + if(users.contains(auth.signedInUser.userName)){ + true + } + else { + false + } + } + else { + false + } + } + + // Check if user is allowed to create dotted hosts using the record types present in dotted hosts config + def checkIfInAllowedRecordType(zone: Zone, config: DottedHostsConfig, rs: RecordSet): Boolean = { + val configZones = config.zoneAuthConfigs.map(x => x.zone) + val zoneName = if(zone.name.takeRight(1) != ".") zone.name + "." else zone.name + val dottedZoneConfig = configZones.filter(_.contains("*")).map(_.replace("*", "[A-Za-z0-9.]*")) + val isContainWildcardZone = dottedZoneConfig.exists(x => zoneName.matches(x)) + val isContainNormalZone = configZones.contains(zoneName) + if(isContainNormalZone){ + val rType = config.zoneAuthConfigs.flatMap { + x: ZoneAuthConfigs => + if (x.zone == zoneName) x.recordTypes else List.empty + } + if(rType.contains(rs.typ.toString)){ + true + } + else { + false + } + } + else if(isContainWildcardZone){ + val rType = config.zoneAuthConfigs.flatMap { + x: ZoneAuthConfigs => + if (x.zone.contains("*")) { + val wildcardZone = x.zone.replace("*", "[A-Za-z0-9.]*") + if (zoneName.matches(wildcardZone)) x.recordTypes else List.empty + } else List.empty + } + if(rType.contains(rs.typ.toString)){ + true + } + else { + false + } + } + else { + false + } + } + + // Check if user is allowed to create dotted hosts using the groups present in dotted hosts config + def checkIfInAllowedGroups(zone: Zone, config: DottedHostsConfig, auth: AuthPrincipal): IO[Boolean] = { + val configZones = config.zoneAuthConfigs.map(x => x.zone) + val zoneName = if(zone.name.takeRight(1) != ".") zone.name + "." else zone.name + val dottedZoneConfig = configZones.filter(_.contains("*")).map(_.replace("*", "[A-Za-z0-9.]*")) + val isContainWildcardZone = dottedZoneConfig.exists(x => zoneName.matches(x)) + val isContainNormalZone = configZones.contains(zoneName) + val groups = if(isContainNormalZone){ + config.zoneAuthConfigs.flatMap { + x: ZoneAuthConfigs => + if (x.zone == zoneName) x.groupList else List.empty + } + } + else if(isContainWildcardZone){ + config.zoneAuthConfigs.flatMap { + x: ZoneAuthConfigs => + if (x.zone.contains("*")) { + val wildcardZone = x.zone.replace("*", "[A-Za-z0-9.]*") + if (zoneName.matches(wildcardZone)) x.groupList else List.empty + } else List.empty + } + } + else { + List.empty + } + for{ + groupsInConfig <- groupRepository.getGroupsByName(groups.toSet) + members = groupsInConfig.flatMap(x => x.memberIds) + usersList <- if(members.isEmpty) IO(Seq.empty) else userRepository.getUsers(members, None, None).map(x => x.users) + users = if(usersList.isEmpty) Seq.empty else usersList.map(x => x.userName) + isPresent = users.contains(auth.signedInUser.userName) + } yield isPresent + } + def getRecordSet( recordSetId: String, authPrincipal: AuthPrincipal diff --git a/modules/api/src/main/scala/vinyldns/api/domain/record/RecordSetValidations.scala b/modules/api/src/main/scala/vinyldns/api/domain/record/RecordSetValidations.scala index 0bff5984a..cbbd35b60 100644 --- a/modules/api/src/main/scala/vinyldns/api/domain/record/RecordSetValidations.scala +++ b/modules/api/src/main/scala/vinyldns/api/domain/record/RecordSetValidations.scala @@ -26,7 +26,7 @@ import vinyldns.core.domain.record.RecordType._ import vinyldns.api.domain.zone._ import vinyldns.core.domain.auth.AuthPrincipal import vinyldns.core.domain.membership.Group -import vinyldns.core.domain.record.{RecordType, RecordSet} +import vinyldns.core.domain.record.{RecordSet, RecordType} import vinyldns.core.domain.zone.Zone import vinyldns.core.Messages._ @@ -90,6 +90,69 @@ object RecordSetValidations { !existingRecordsWithName.exists(rs => rs.id != newRecordSet.id && rs.typ == newRecordSet.typ) ) + // Check whether the record has dot or not + def checkForDot( + newRecordSet: RecordSet, + zone: Zone, + existingRecordSet: Option[RecordSet] = None, + recordFqdnDoesNotExist: Boolean, + dottedHostZoneConfig: Set[String], + isRecordTypeAndUserAllowed: Boolean, + allowedDotsLimit: Int = 0 + ): Either[Throwable, Unit] = { + + val zoneName = if(zone.name.takeRight(1) != ".") zone.name + "." else zone.name + // Check if the zone of the recordset is present in dotted hosts config list + val isDomainAllowed = dottedHostZoneConfig.contains(zoneName) + + // Check if record set contains dot and if it is in zone which is allowed to have dotted records from dotted hosts config + if(allowedDotsLimit != 0 && newRecordSet.name.contains(".") && isDomainAllowed && newRecordSet.name != zone.name) { + if(!isRecordTypeAndUserAllowed){ + isUserAndRecordTypeAuthorized(newRecordSet, zone, existingRecordSet, recordFqdnDoesNotExist, isRecordTypeAndUserAllowed) + } + else { + isDotted(newRecordSet, zone, existingRecordSet, recordFqdnDoesNotExist, isRecordTypeAndUserAllowed) + } + } + else { + isNotDotted(newRecordSet, zone, existingRecordSet) + } + } + + // For dotted host. Check if a record is already present which conflicts with the new dotted record. If so, throw an error + def isDotted( + newRecordSet: RecordSet, + zone: Zone, + existingRecordSet: Option[RecordSet] = None, + recordFqdnDoesNotExist: Boolean, + isRecordTypeAndUserAllowed: Boolean + ): Either[Throwable, Unit] = + ensuring( + InvalidRequest( + s"Record with fqdn '${newRecordSet.name}.${zone.name}' cannot be created. " + + s"Please check if a record with the same FQDN and type already exist and make the change there." + ) + )( + (newRecordSet.name != zone.name || existingRecordSet.exists(_.name == newRecordSet.name)) && recordFqdnDoesNotExist && isRecordTypeAndUserAllowed + ) + + // For dotted host. Check if the user is authorized and the record type is allowed. If not, throw an error + def isUserAndRecordTypeAuthorized( + newRecordSet: RecordSet, + zone: Zone, + existingRecordSet: Option[RecordSet] = None, + recordFqdnDoesNotExist: Boolean, + isRecordTypeAndUserAllowed: Boolean + ): Either[Throwable, Unit] = + ensuring( + InvalidRequest( + s"Record type is not allowed or the user is not authorized to create a dotted host in the zone '${zone.name}'" + ) + )( + (newRecordSet.name != zone.name || existingRecordSet.exists(_.name == newRecordSet.name)) && recordFqdnDoesNotExist && isRecordTypeAndUserAllowed + ) + + // Check if the recordset contains dot but is not in the allowed zones to create dotted records. If so, throw an error def isNotDotted( newRecordSet: RecordSet, zone: Zone, @@ -110,16 +173,20 @@ object RecordSetValidations { existingRecordsWithName: List[RecordSet], zone: Zone, existingRecordSet: Option[RecordSet], - approvedNameServers: List[Regex] + approvedNameServers: List[Regex], + recordFqdnDoesNotExist: Boolean, + dottedHostZoneConfig: Set[String], + isRecordTypeAndUserAllowed: Boolean, + allowedDotsLimit: Int = 0 ): Either[Throwable, Unit] = newRecordSet.typ match { - case CNAME => cnameValidations(newRecordSet, existingRecordsWithName, zone, existingRecordSet) - case NS => nsValidations(newRecordSet, zone, existingRecordSet, approvedNameServers) - case SOA => soaValidations(newRecordSet, zone) + case CNAME => cnameValidations(newRecordSet, existingRecordsWithName, zone, existingRecordSet, recordFqdnDoesNotExist, dottedHostZoneConfig, isRecordTypeAndUserAllowed, allowedDotsLimit) + case NS => nsValidations(newRecordSet, zone, existingRecordSet, approvedNameServers, recordFqdnDoesNotExist, dottedHostZoneConfig, isRecordTypeAndUserAllowed, allowedDotsLimit) + case SOA => soaValidations(newRecordSet, zone, recordFqdnDoesNotExist, dottedHostZoneConfig, isRecordTypeAndUserAllowed, allowedDotsLimit) case PTR => ptrValidations(newRecordSet, zone) case SRV | TXT | NAPTR => ().asRight // SRV, TXT and NAPTR do not go through dotted host check - case DS => dsValidations(newRecordSet, existingRecordsWithName, zone) - case _ => isNotDotted(newRecordSet, zone, existingRecordSet) + case DS => dsValidations(newRecordSet, existingRecordsWithName, zone, recordFqdnDoesNotExist, dottedHostZoneConfig, isRecordTypeAndUserAllowed, allowedDotsLimit) + case _ => checkForDot(newRecordSet, zone, existingRecordSet, recordFqdnDoesNotExist, dottedHostZoneConfig, isRecordTypeAndUserAllowed, allowedDotsLimit) } def typeSpecificDeleteValidations(recordSet: RecordSet, zone: Zone): Either[Throwable, Unit] = @@ -140,7 +207,11 @@ object RecordSetValidations { newRecordSet: RecordSet, existingRecordsWithName: List[RecordSet], zone: Zone, - existingRecordSet: Option[RecordSet] = None + existingRecordSet: Option[RecordSet] = None, + recordFqdnDoesNotExist: Boolean, + dottedHostZoneConfig: Set[String], + isRecordTypeAndUserAllowed: Boolean, + allowedDotsLimit: Int = 0 ): Either[Throwable, Unit] = { // cannot create a cname record if a record with the same exists val noRecordWithName = { @@ -173,7 +244,7 @@ object RecordSetValidations { ) _ <- noRecordWithName _ <- RDataWithConsecutiveDots - _ <- isNotDotted(newRecordSet, zone, existingRecordSet) + _ <- checkForDot(newRecordSet, zone, existingRecordSet, recordFqdnDoesNotExist, dottedHostZoneConfig, isRecordTypeAndUserAllowed, allowedDotsLimit) } yield () } @@ -181,7 +252,11 @@ object RecordSetValidations { def dsValidations( newRecordSet: RecordSet, existingRecordsWithName: List[RecordSet], - zone: Zone + zone: Zone, + recordFqdnDoesNotExist: Boolean, + dottedHostZoneConfig: Set[String], + isRecordTypeAndUserAllowed: Boolean, + allowedDotsLimit: Int = 0 ): Either[Throwable, Unit] = { // see https://tools.ietf.org/html/rfc4035#section-2.4 val nsChecks = existingRecordsWithName.find(_.typ == NS) match { @@ -194,7 +269,7 @@ object RecordSetValidations { } for { - _ <- isNotDotted(newRecordSet, zone) + _ <- checkForDot(newRecordSet, zone, None, recordFqdnDoesNotExist, dottedHostZoneConfig, isRecordTypeAndUserAllowed, allowedDotsLimit) _ <- isNotOrigin( newRecordSet, zone, @@ -208,10 +283,14 @@ object RecordSetValidations { newRecordSet: RecordSet, zone: Zone, oldRecordSet: Option[RecordSet], - approvedNameServers: List[Regex] + approvedNameServers: List[Regex], + recordFqdnDoesNotExist: Boolean, + dottedHostZoneConfig: Set[String], + isRecordTypeAndUserAllowed: Boolean, + allowedDotsLimit: Int = 0 ): Either[Throwable, Unit] = { // TODO kept consistency with old validation. Not sure why NS could be dotted in reverse specifically - val isNotDottedHost = if (!zone.isReverse) isNotDotted(newRecordSet, zone) else ().asRight + val isNotDottedHost = if (!zone.isReverse) checkForDot(newRecordSet, zone, None, recordFqdnDoesNotExist, dottedHostZoneConfig, isRecordTypeAndUserAllowed, allowedDotsLimit) else ().asRight for { _ <- isNotDottedHost @@ -233,9 +312,9 @@ object RecordSetValidations { } yield () } - def soaValidations(newRecordSet: RecordSet, zone: Zone): Either[Throwable, Unit] = + def soaValidations(newRecordSet: RecordSet, zone: Zone, recordFqdnDoesNotExist: Boolean, dottedHostZoneConfig: Set[String], isRecordTypeAndUserAllowed: Boolean, allowedDotsLimit: Int = 0): Either[Throwable, Unit] = // TODO kept consistency with old validation. in theory if SOA always == zone name, no special case is needed here - if (!zone.isReverse) isNotDotted(newRecordSet, zone) else ().asRight + if (!zone.isReverse) checkForDot(newRecordSet, zone, None, recordFqdnDoesNotExist, dottedHostZoneConfig, isRecordTypeAndUserAllowed, allowedDotsLimit) else ().asRight def ptrValidations(newRecordSet: RecordSet, zone: Zone): Either[Throwable, Unit] = // TODO we don't check for PTR as dotted...not sure why @@ -278,6 +357,29 @@ object RecordSetValidations { .leftMap(errors => InvalidRequest(errors.toList.map(_.message).mkString(", "))) } + def checkAllowedDots(allowedDotsLimit: Int, recordSet: RecordSet, zone: Zone): Either[Throwable, Unit] = { + ensuring( + InvalidRequest( + s"RecordSet with name ${recordSet.name} has more dots than that is allowed in config for this zone " + + s"which is, 'dots-limit = $allowedDotsLimit'." + ) + )( + recordSet.name.count(_ == '.') <= allowedDotsLimit || (recordSet.name.count(_ == '.') == 1 && + recordSet.name.takeRight(1) == ".") || recordSet.name == zone.name || + (recordSet.typ.toString == "PTR" || recordSet.typ.toString == "SRV" || recordSet.typ.toString == "TXT" || recordSet.typ.toString == "NAPTR") + ) + } + + def isNotApexEndsWithDot(recordSet: RecordSet, zone: Zone): Either[Throwable, Unit] = { + ensuring( + InvalidRequest( + "RecordSet name cannot end with a dot, unless it's an apex record." + ) + )( + recordSet.name.endsWith(zone.name) || !recordSet.name.endsWith(".") + ) + } + def canUseOwnerGroup( ownerGroupId: Option[String], group: Option[Group], diff --git a/modules/api/src/test/functional/tests/recordsets/create_recordset_test.py b/modules/api/src/test/functional/tests/recordsets/create_recordset_test.py index c86d0e6a7..f04152c13 100644 --- a/modules/api/src/test/functional/tests/recordsets/create_recordset_test.py +++ b/modules/api/src/test/functional/tests/recordsets/create_recordset_test.py @@ -508,9 +508,11 @@ def test_create_invalid_record_data(shared_zone_test_context): )) -def test_create_dotted_a_record_not_apex_fails(shared_zone_test_context): +def test_create_dotted_a_record_not_apex_fails_when_dotted_hosts_config_not_satisfied(shared_zone_test_context): """ - Test that creating a dotted host name A record set fails. + Test that creating a dotted host name A record set fails + Here the zone and user (individual) is allowed but record type is not allowed. Hence the test fails + Config present in reference.conf """ client = shared_zone_test_context.ok_vinyldns_client @@ -524,8 +526,57 @@ def test_create_dotted_a_record_not_apex_fails(shared_zone_test_context): zone_name = shared_zone_test_context.parent_zone["name"] error = client.create_recordset(dotted_host_a_record, status=422) - assert_that(error, is_("Record with name " + dotted_host_a_record["name"] + " and type A is a dotted host which " - "is not allowed in zone " + zone_name)) + assert_that(error, is_("Record type is not allowed or the user is not authorized to create a dotted host in the " + "zone '" + zone_name + "'")) + + +def test_create_dotted_a_record_succeeds_if_all_dotted_hosts_config_satisfied(shared_zone_test_context): + """ + Test that creating a A record set with dotted host record name succeeds + Here the zone, user (in group) and record type is allowed. Hence the test succeeds + Config present in reference.conf + """ + client = shared_zone_test_context.history_client + zone = shared_zone_test_context.dummy_zone + dotted_host_a_record = { + "zoneId": zone["id"], + "name": "dot.ted", + "type": "A", + "ttl": 500, + "records": [{"address": "127.0.0.1"}] + } + + dotted_a_record = None + try: + dotted_cname_response = client.create_recordset(dotted_host_a_record, status=202) + dotted_a_record = client.wait_until_recordset_change_status(dotted_cname_response, "Complete")["recordSet"] + assert_that(dotted_a_record["name"], is_(dotted_host_a_record["name"])) + finally: + if dotted_a_record: + delete_result = client.delete_recordset(dotted_a_record["zoneId"], dotted_a_record["id"], status=202) + client.wait_until_recordset_change_status(delete_result, "Complete") + + +def test_create_dotted_a_record_fails_if_all_dotted_hosts_config_not_satisfied(shared_zone_test_context): + """ + Test that creating a A record set with dotted host record name fails + Here the zone, user (in group) and record type is allowed. + But the record name has more dots than the number of dots allowed for this zone. Hence the test fails + The 'dots-limit' config from dotted-hosts config is not satisfied. Config present in reference.conf + """ + client = shared_zone_test_context.history_client + zone = shared_zone_test_context.dummy_zone + dotted_host_a_record = { + "zoneId": zone["id"], + "name": "dot.ted.trial.test.host", + "type": "A", + "ttl": 500, + "records": [{"address": "127.0.0.1"}] + } + + error = client.create_recordset(dotted_host_a_record, status=422) + assert_that(error, is_("RecordSet with name " + dotted_host_a_record["name"] + " has more dots than that is " + "allowed in config for this zone which is, 'dots-limit = 3'.")) def test_create_dotted_a_record_apex_succeeds(shared_zone_test_context): @@ -581,13 +632,15 @@ def test_create_dotted_a_record_apex_with_trailing_dot_succeeds(shared_zone_test client.wait_until_recordset_change_status(delete_result, "Complete") -def test_create_dotted_cname_record_fails(shared_zone_test_context): +def test_create_dotted_cname_record_fails_when_dotted_hosts_config_not_satisfied(shared_zone_test_context): """ - Test that creating a CNAME record set with dotted host record name returns an error. + Test that creating a CNAME record set with dotted host record name returns an error + Here the zone is allowed but user (individual or in group) and record type is not allowed. Hence the test fails + Config present in reference.conf """ - client = shared_zone_test_context.ok_vinyldns_client - zone = shared_zone_test_context.parent_zone - apex_cname_rs = { + client = shared_zone_test_context.dummy_vinyldns_client + zone = shared_zone_test_context.dummy_zone + dotted_host_cname_record = { "zoneId": zone["id"], "name": "dot.ted", "type": "CNAME", @@ -595,8 +648,37 @@ def test_create_dotted_cname_record_fails(shared_zone_test_context): "records": [{"cname": "foo.bar."}] } - error = client.create_recordset(apex_cname_rs, status=422) - assert_that(error, is_(f'Record with name dot.ted and type CNAME is a dotted host which is not allowed in zone {zone["name"]}')) + error = client.create_recordset(dotted_host_cname_record, status=422) + assert_that(error, is_("Record type is not allowed or the user is not authorized to create a dotted host in the " + "zone '" + zone["name"] + "'")) + + +def test_create_dotted_cname_record_succeeds_if_all_dotted_hosts_config_satisfied(shared_zone_test_context): + """ + Test that creating a CNAME record set with dotted host record name succeeds. + Here the zone, user (individual) and record type is allowed. Hence the test succeeds + Config present in reference.conf + """ + client = shared_zone_test_context.ok_vinyldns_client + zone = shared_zone_test_context.parent_zone + dotted_host_cname_record = { + "zoneId": zone["id"], + "name": "dot.ted", + "type": "CNAME", + "ttl": 500, + "records": [{"cname": "foo.bar."}] + } + + dotted_cname_record = None + try: + dotted_cname_response = client.create_recordset(dotted_host_cname_record, status=202) + dotted_cname_record = client.wait_until_recordset_change_status(dotted_cname_response, "Complete")["recordSet"] + assert_that(dotted_cname_record["name"], is_(dotted_host_cname_record["name"])) + finally: + if dotted_cname_record: + delete_result = client.delete_recordset(dotted_cname_record["zoneId"], dotted_cname_record["id"], + status=202) + client.wait_until_recordset_change_status(delete_result, "Complete") def test_create_cname_with_multiple_records(shared_zone_test_context): @@ -701,7 +783,8 @@ def test_create_cname_with_existing_record_with_name_fails(shared_zone_test_cont a_record = client.wait_until_recordset_change_status(a_create, "Complete")["recordSet"] error = client.create_recordset(cname_rs, status=409) - assert_that(error, is_(f'RecordSet with name duplicate-test-name already exists in zone {zone["name"]}, CNAME record cannot use duplicate name')) + assert_that(error, + is_(f'RecordSet with name duplicate-test-name already exists in zone {zone["name"]}, CNAME record cannot use duplicate name')) finally: if a_record: delete_result = client.delete_recordset(a_record["zoneId"], a_record["id"], status=202) @@ -744,7 +827,8 @@ def test_create_record_with_existing_cname_fails(shared_zone_test_context): cname_record = client.wait_until_recordset_change_status(cname_create, "Complete")["recordSet"] error = client.create_recordset(a_rs, status=409) - assert_that(error, is_(f'RecordSet with name duplicate-test-name and type CNAME already exists in zone {zone["name"]}')) + assert_that(error, + is_(f'RecordSet with name duplicate-test-name and type CNAME already exists in zone {zone["name"]}')) finally: if cname_record: delete_result = client.delete_recordset(cname_record["zoneId"], cname_record["id"], status=202) @@ -1368,7 +1452,6 @@ def test_at_create_recordset(shared_zone_test_context): } result = client.create_recordset(new_rs, status=202) - assert_that(result["changeType"], is_("Create")) assert_that(result["status"], is_("Pending")) assert_that(result["created"], is_not(none())) @@ -1418,7 +1501,6 @@ def test_create_record_with_escape_characters_in_record_data_succeeds(shared_zon } result = client.create_recordset(new_rs, status=202) - assert_that(result["changeType"], is_("Create")) assert_that(result["status"], is_("Pending")) assert_that(result["created"], is_not(none())) @@ -1743,7 +1825,8 @@ def test_create_high_value_domain_fails(shared_zone_test_context): } error = client.create_recordset(new_rs, status=422) - assert_that(error, is_(f'Record name "high-value-domain.{zone["name"]}" is configured as a High Value Domain, so it cannot be modified.')) + assert_that(error, + is_(f'Record name "high-value-domain.{zone["name"]}" is configured as a High Value Domain, so it cannot be modified.')) def test_create_high_value_domain_fails_case_insensitive(shared_zone_test_context): @@ -1765,7 +1848,8 @@ def test_create_high_value_domain_fails_case_insensitive(shared_zone_test_contex } error = client.create_recordset(new_rs, status=422) - assert_that(error, is_(f'Record name "hIgH-vAlUe-dOmAiN.{zone["name"]}" is configured as a High Value Domain, so it cannot be modified.')) + assert_that(error, + is_(f'Record name "hIgH-vAlUe-dOmAiN.{zone["name"]}" is configured as a High Value Domain, so it cannot be modified.')) def test_create_high_value_domain_fails_for_ip4_ptr(shared_zone_test_context): @@ -1786,7 +1870,8 @@ def test_create_high_value_domain_fails_for_ip4_ptr(shared_zone_test_context): } error_ptr = client.create_recordset(ptr, status=422) - assert_that(error_ptr, is_(f'Record name "{shared_zone_test_context.ip4_classless_prefix}.252" is configured as a High Value Domain, so it cannot be modified.')) + assert_that(error_ptr, + is_(f'Record name "{shared_zone_test_context.ip4_classless_prefix}.252" is configured as a High Value Domain, so it cannot be modified.')) def test_create_high_value_domain_fails_for_ip6_ptr(shared_zone_test_context): @@ -1807,7 +1892,8 @@ def test_create_high_value_domain_fails_for_ip6_ptr(shared_zone_test_context): } error_ptr = client.create_recordset(ptr, status=422) - assert_that(error_ptr, is_(f'Record name "{shared_zone_test_context.ip6_prefix}:0000:0000:0000:0000:ffff" is configured as a High Value Domain, so it cannot be modified.')) + assert_that(error_ptr, + is_(f'Record name "{shared_zone_test_context.ip6_prefix}:0000:0000:0000:0000:ffff" is configured as a High Value Domain, so it cannot be modified.')) def test_create_with_owner_group_in_private_zone_by_admin_passes(shared_zone_test_context): @@ -1874,7 +1960,8 @@ def test_create_with_owner_group_in_private_zone_by_acl_passes(shared_zone_test_ finally: clear_ok_acl_rules(shared_zone_test_context) if create_rs: - delete_result = shared_zone_test_context.ok_vinyldns_client.delete_recordset(zone["id"], create_rs["id"], status=202) + delete_result = shared_zone_test_context.ok_vinyldns_client.delete_recordset(zone["id"], create_rs["id"], + status=202) shared_zone_test_context.ok_vinyldns_client.wait_until_recordset_change_status(delete_result, "Complete") @@ -1900,8 +1987,11 @@ def test_create_with_owner_group_in_shared_zone_by_acl_passes(shared_zone_test_c finally: clear_shared_zone_acl_rules(shared_zone_test_context) if create_rs: - delete_result = shared_zone_test_context.shared_zone_vinyldns_client.delete_recordset(zone["id"], create_rs["id"], status=202) - shared_zone_test_context.shared_zone_vinyldns_client.wait_until_recordset_change_status(delete_result, "Complete") + delete_result = shared_zone_test_context.shared_zone_vinyldns_client.delete_recordset(zone["id"], + create_rs["id"], + status=202) + shared_zone_test_context.shared_zone_vinyldns_client.wait_until_recordset_change_status(delete_result, + "Complete") def test_create_in_shared_zone_without_owner_group_id_succeeds(shared_zone_test_context): @@ -1955,10 +2045,12 @@ def test_create_in_shared_zone_by_unassociated_user_fails_if_record_type_is_not_ zone = shared_zone_test_context.shared_zone group = shared_zone_test_context.dummy_group - record_json = create_recordset(zone, "test_shared_not_approved_record_type", "MX", [{"preference": 3, "exchange": "mx"}]) + record_json = create_recordset(zone, "test_shared_not_approved_record_type", "MX", + [{"preference": 3, "exchange": "mx"}]) record_json["ownerGroupId"] = group["id"] error = client.create_recordset(record_json, status=403) - assert_that(error, is_(f'User dummy does not have access to create test-shared-not-approved-record-type.{zone["name"]}')) + assert_that(error, + is_(f'User dummy does not have access to create test-shared-not-approved-record-type.{zone["name"]}')) def test_create_with_not_found_owner_group_fails(shared_zone_test_context): @@ -1997,7 +2089,8 @@ def test_create_ds_success(shared_zone_test_context): zone = shared_zone_test_context.ds_zone record_data = [ {"keytag": 60485, "algorithm": 5, "digesttype": 1, "digest": "2BB183AF5F22588179A53B0A98631FAD1A292118"}, - {"keytag": 60485, "algorithm": 5, "digesttype": 2, "digest": "D4B7D520E7BB5F0F67674A0CCEB1E3E0614B93C4F9E99B8383F6A1E4469DA50A"} + {"keytag": 60485, "algorithm": 5, "digesttype": 2, + "digest": "D4B7D520E7BB5F0F67674A0CCEB1E3E0614B93C4F9E99B8383F6A1E4469DA50A"} ] record_json = create_recordset(zone, "dskey", "DS", record_data, ttl=3600) result_rs = None @@ -2039,7 +2132,8 @@ def test_create_ds_unknown_algorithm(shared_zone_test_context): """ client = shared_zone_test_context.ok_vinyldns_client zone = shared_zone_test_context.ds_zone - record_data = [{"keytag": 60485, "algorithm": 0, "digesttype": 1, "digest": "2BB183AF5F22588179A53B0A98631FAD1A292118"}] + record_data = [ + {"keytag": 60485, "algorithm": 0, "digesttype": 1, "digest": "2BB183AF5F22588179A53B0A98631FAD1A292118"}] record_json = create_recordset(zone, "dskey", "DS", record_data) errors = client.create_recordset(record_json, status=400)["errors"] assert_that(errors, contains_inanyorder("Algorithm 0 is not a supported DNSSEC algorithm")) @@ -2051,7 +2145,8 @@ def test_create_ds_unknown_digest_type(shared_zone_test_context): """ client = shared_zone_test_context.ok_vinyldns_client zone = shared_zone_test_context.ds_zone - record_data = [{"keytag": 60485, "algorithm": 5, "digesttype": 0, "digest": "2BB183AF5F22588179A53B0A98631FAD1A292118"}] + record_data = [ + {"keytag": 60485, "algorithm": 5, "digesttype": 0, "digest": "2BB183AF5F22588179A53B0A98631FAD1A292118"}] record_json = create_recordset(zone, "dskey", "DS", record_data) errors = client.create_recordset(record_json, status=400)["errors"] assert_that(errors, contains_inanyorder("Digest Type 0 is not a supported DS record digest type")) @@ -2063,10 +2158,12 @@ def test_create_ds_no_ns_fails(shared_zone_test_context): """ client = shared_zone_test_context.ok_vinyldns_client zone = shared_zone_test_context.ds_zone - record_data = [{"keytag": 60485, "algorithm": 5, "digesttype": 1, "digest": "2BB183AF5F22588179A53B0A98631FAD1A292118"}] + record_data = [ + {"keytag": 60485, "algorithm": 5, "digesttype": 1, "digest": "2BB183AF5F22588179A53B0A98631FAD1A292118"}] record_json = create_recordset(zone, "no-ns-exists", "DS", record_data, ttl=3600) error = client.create_recordset(record_json, status=422) - assert_that(error, is_(f'DS record [no-ns-exists] is invalid because there is no NS record with that name in the zone [{zone["name"]}]')) + assert_that(error, + is_(f'DS record [no-ns-exists] is invalid because there is no NS record with that name in the zone [{zone["name"]}]')) def test_create_apex_ds_fails(shared_zone_test_context): @@ -2075,7 +2172,8 @@ def test_create_apex_ds_fails(shared_zone_test_context): """ client = shared_zone_test_context.ok_vinyldns_client zone = shared_zone_test_context.ds_zone - record_data = [{"keytag": 60485, "algorithm": 5, "digesttype": 1, "digest": "2BB183AF5F22588179A53B0A98631FAD1A292118"}] + record_data = [ + {"keytag": 60485, "algorithm": 5, "digesttype": 1, "digest": "2BB183AF5F22588179A53B0A98631FAD1A292118"}] record_json = create_recordset(zone, "@", "DS", record_data, ttl=100) error = client.create_recordset(record_json, status=422) assert_that(error, is_(f'Record with name [{zone["name"]}] is an DS record at apex and cannot be added')) @@ -2087,7 +2185,9 @@ def test_create_dotted_ds_fails(shared_zone_test_context): """ client = shared_zone_test_context.ok_vinyldns_client zone = shared_zone_test_context.ds_zone - record_data = [{"keytag": 60485, "algorithm": 5, "digesttype": 1, "digest": "2BB183AF5F22588179A53B0A98631FAD1A292118"}] + record_data = [ + {"keytag": 60485, "algorithm": 5, "digesttype": 1, "digest": "2BB183AF5F22588179A53B0A98631FAD1A292118"}] record_json = create_recordset(zone, "dotted.ds", "DS", record_data, ttl=100) error = client.create_recordset(record_json, status=422) - assert_that(error, is_(f'Record with name dotted.ds and type DS is a dotted host which is not allowed in zone {zone["name"]}')) + assert_that(error, + is_(f'Record with name dotted.ds and type DS is a dotted host which is not allowed in zone {zone["name"]}')) diff --git a/modules/api/src/test/functional/tests/shared_zone_test_context.py b/modules/api/src/test/functional/tests/shared_zone_test_context.py index 8d919cb23..1206d608c 100644 --- a/modules/api/src/test/functional/tests/shared_zone_test_context.py +++ b/modules/api/src/test/functional/tests/shared_zone_test_context.py @@ -174,6 +174,15 @@ class SharedZoneTestContext(object): "shared": False, "adminGroupId": self.dummy_group["id"], "isTest": True, + "acl": { + "rules": [ + { + "accessLevel": "Delete", + "description": "some_test_rule", + "userId": "history-id" + } + ] + }, "connection": { "name": "dummy.", "keyName": VinylDNSTestContext.dns_key_name, diff --git a/modules/api/src/test/scala/vinyldns/api/VinylDNSTestHelpers.scala b/modules/api/src/test/scala/vinyldns/api/VinylDNSTestHelpers.scala index d071b846d..94eabc111 100644 --- a/modules/api/src/test/scala/vinyldns/api/VinylDNSTestHelpers.scala +++ b/modules/api/src/test/scala/vinyldns/api/VinylDNSTestHelpers.scala @@ -18,7 +18,7 @@ package vinyldns.api import com.comcast.ip4s.IpAddress import org.joda.time.DateTime -import vinyldns.api.config.{BatchChangeConfig, HighValueDomainConfig, LimitsConfig, ManualReviewConfig, ScheduledChangesConfig} +import vinyldns.api.config.{ZoneAuthConfigs, BatchChangeConfig, DottedHostsConfig, HighValueDomainConfig, LimitsConfig, ManualReviewConfig, ScheduledChangesConfig} import vinyldns.api.domain.batch.V6DiscoveryNibbleBoundaries import vinyldns.core.domain.record._ import vinyldns.core.domain.zone._ @@ -40,6 +40,10 @@ trait VinylDNSTestHelpers { val approvedNameServers: List[Regex] = List(new Regex("some.test.ns.")) + val dottedHostsConfig: DottedHostsConfig = DottedHostsConfig(List(ZoneAuthConfigs("dotted.xyz.",List("xyz"),List("dummy"),List("CNAME"), 3), ZoneAuthConfigs("abc.zone.recordsets.",List("locked"),List("dummy"),List("CNAME"), 3), ZoneAuthConfigs("xyz.",List("super"),List("xyz"),List("CNAME"), 3), ZoneAuthConfigs("dot.xyz.",List("super"),List("xyz"),List("CNAME"), 0))) + + val emptyDottedHostsConfig: DottedHostsConfig = DottedHostsConfig(List.empty) + val defaultTtl: Long = 7200 val manualReviewDomainList: List[Regex] = List(new Regex("needs-review.*")) diff --git a/modules/api/src/test/scala/vinyldns/api/domain/batch/BatchChangeServiceSpec.scala b/modules/api/src/test/scala/vinyldns/api/domain/batch/BatchChangeServiceSpec.scala index d56e65c06..34abec536 100644 --- a/modules/api/src/test/scala/vinyldns/api/domain/batch/BatchChangeServiceSpec.scala +++ b/modules/api/src/test/scala/vinyldns/api/domain/batch/BatchChangeServiceSpec.scala @@ -55,7 +55,7 @@ import vinyldns.api.domain.access.AccessValidations import scala.concurrent.ExecutionContext class BatchChangeServiceSpec - extends AnyWordSpec + extends AnyWordSpec with Matchers with MockitoSugar with CatsHelpers @@ -2559,4 +2559,4 @@ class BatchChangeServiceSpec ) } } -} +} \ No newline at end of file diff --git a/modules/api/src/test/scala/vinyldns/api/domain/batch/BatchChangeValidationsSpec.scala b/modules/api/src/test/scala/vinyldns/api/domain/batch/BatchChangeValidationsSpec.scala index 6ecf64778..0131059d4 100644 --- a/modules/api/src/test/scala/vinyldns/api/domain/batch/BatchChangeValidationsSpec.scala +++ b/modules/api/src/test/scala/vinyldns/api/domain/batch/BatchChangeValidationsSpec.scala @@ -41,7 +41,7 @@ import vinyldns.core.domain.zone.{ACLRule, AccessLevel, Zone, ZoneStatus} import scala.util.Random class BatchChangeValidationsSpec - extends AnyPropSpec + extends AnyPropSpec with Matchers with ScalaCheckDrivenPropertyChecks with EitherMatchers @@ -192,9 +192,9 @@ class BatchChangeValidationsSpec ) private def makeAddUpdateRecord( - recordName: String, - aData: AData = AData("1.2.3.4") - ): AddChangeForValidation = + recordName: String, + aData: AData = AData("1.2.3.4") + ): AddChangeForValidation = AddChangeForValidation( okZone, s"$recordName", @@ -203,9 +203,9 @@ class BatchChangeValidationsSpec ) private def makeDeleteUpdateDeleteRRSet( - recordName: String, - recordData: Option[RecordData] = None - ): DeleteRRSetChangeForValidation = + recordName: String, + recordData: Option[RecordData] = None + ): DeleteRRSetChangeForValidation = DeleteRRSetChangeForValidation( okZone, s"$recordName", @@ -666,7 +666,7 @@ class BatchChangeValidationsSpec } property("""validateAddChangeInput: should fail with InvalidIpv4Address - |if validateRecordData fails for an invalid ipv4 address""".stripMargin) { + |if validateRecordData fails for an invalid ipv4 address""".stripMargin) { val invalidIpv4 = "invalidIpv4:123" val change = AddChangeInput("test.comcast.com.", RecordType.A, ttl, AData(invalidIpv4)) val result = validateAddChangeInput(change, false) @@ -1038,7 +1038,7 @@ class BatchChangeValidationsSpec property( """validateChangesWithContext: should succeed for update in shared zone if user belongs to record - | owner group""".stripMargin + | owner group""".stripMargin ) { val existingRecord = sharedZoneRecord.copy( @@ -1073,7 +1073,7 @@ class BatchChangeValidationsSpec } property("""validateChangesWithContext: should succeed adding a record - |if an existing CNAME with the same name exists but is being deleted""".stripMargin) { + |if an existing CNAME with the same name exists but is being deleted""".stripMargin) { val existingCname = rsOk.copy(name = "deleteRRSet", typ = RecordType.CNAME) val existingCname2 = existingCname.copy(name = "deleteRecord", records = List(CNAMEData(Fqdn("cname.data.")))) @@ -1218,7 +1218,7 @@ class BatchChangeValidationsSpec } property("""validateChangesWithContext: should fail with CnameIsNotUniqueError - |if CNAME record name already exists""".stripMargin) { + |if CNAME record name already exists""".stripMargin) { val addCname = AddChangeForValidation( validZone, "existingCname", @@ -1240,7 +1240,7 @@ class BatchChangeValidationsSpec } property("""validateChangesWithContext: should succeed for CNAME record - |if there's a duplicate PTR ipv4 record that is being deleted""".stripMargin) { + |if there's a duplicate PTR ipv4 record that is being deleted""".stripMargin) { val addCname = AddChangeForValidation( validIp4ReverseZone, "30", @@ -1268,7 +1268,7 @@ class BatchChangeValidationsSpec } property("""validateChangesWithContext: should fail with CnameIsNotUniqueError for CNAME record - |if there's a duplicate PTR ipv6 record""".stripMargin) { + |if there's a duplicate PTR ipv6 record""".stripMargin) { val addCname = AddChangeForValidation( validZone, "0.6.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0", @@ -1294,7 +1294,7 @@ class BatchChangeValidationsSpec } property("""validateChangesWithContext: CNAME record should pass - |if no other changes in batch change have same record name""".stripMargin) { + |if no other changes in batch change have same record name""".stripMargin) { val addA = AddChangeForValidation( okZone, "test", @@ -1329,7 +1329,7 @@ class BatchChangeValidationsSpec } property("""validateChangesWithContext: CNAME record should fail - |if another add change in batch change has the same record name""".stripMargin) { + |if another add change in batch change has the same record name""".stripMargin) { val addA = AddChangeForValidation( okZone, "test", @@ -1369,7 +1369,7 @@ class BatchChangeValidationsSpec } property("""validateChangesWithContext: both CNAME records should fail - |if there are duplicate CNAME add change inputs""".stripMargin) { + |if there are duplicate CNAME add change inputs""".stripMargin) { val addA = AddChangeForValidation( okZone, "test", @@ -1411,7 +1411,7 @@ class BatchChangeValidationsSpec } property("""validateChangesWithContext: both PTR records should succeed - |if there are duplicate PTR add change inputs""".stripMargin) { + |if there are duplicate PTR add change inputs""".stripMargin) { val addA = AddChangeForValidation( okZone, "test", @@ -1444,7 +1444,7 @@ class BatchChangeValidationsSpec } property("""validateChangesWithContext: should succeed for AddChangeForValidation - |if user has group admin access""".stripMargin) { + |if user has group admin access""".stripMargin) { val addA = AddChangeForValidation( validZone, "valid", @@ -1533,7 +1533,7 @@ class BatchChangeValidationsSpec } property("""validateChangesWithContext: should fail with RecordNameNotUniqueInBatch for PTR record - |if valid CNAME with same name exists in batch""".stripMargin) { + |if valid CNAME with same name exists in batch""".stripMargin) { val addCname = AddChangeForValidation( validZone, "existing", @@ -1604,7 +1604,7 @@ class BatchChangeValidationsSpec } property("""validateChangesWithContext: should succeed for DeleteChangeForValidation - |if record set status is Active""".stripMargin) { + |if record set status is Active""".stripMargin) { val deleteA = DeleteRRSetChangeForValidation( validZone, "Active-record-status", @@ -1629,7 +1629,7 @@ class BatchChangeValidationsSpec } property("""validateChangesWithContext: should succeed for DeleteChangeForValidation - |if user has group admin access"""".stripMargin) { + |if user has group admin access"""".stripMargin) { val deleteA = DeleteRRSetChangeForValidation( validZone, @@ -1651,7 +1651,7 @@ class BatchChangeValidationsSpec } property(""" validateChangesWithContext: should fail for DeleteChangeForValidation - | if user is superUser with no other access""".stripMargin) { + | if user is superUser with no other access""".stripMargin) { val deleteA = DeleteRRSetChangeForValidation( validZone, @@ -1734,7 +1734,7 @@ class BatchChangeValidationsSpec } property("""validateChangesWithContext: should properly process batch that contains - |a CNAME and different type record with the same name""".stripMargin) { + |a CNAME and different type record with the same name""".stripMargin) { val addDuplicateA = AddChangeForValidation( okZone, "test", @@ -2445,7 +2445,7 @@ class BatchChangeValidationsSpec property( """validateChangesWithContext: should fail validateAddWithContext with - |ZoneDiscoveryError if new record is dotted host but not a TXT record type""".stripMargin + |ZoneDiscoveryError if new record is dotted host but not a TXT record type""".stripMargin ) { val addA = AddChangeForValidation( okZone, diff --git a/modules/api/src/test/scala/vinyldns/api/domain/record/RecordSetServiceSpec.scala b/modules/api/src/test/scala/vinyldns/api/domain/record/RecordSetServiceSpec.scala index 3d629e43b..520a4bd9e 100644 --- a/modules/api/src/test/scala/vinyldns/api/domain/record/RecordSetServiceSpec.scala +++ b/modules/api/src/test/scala/vinyldns/api/domain/record/RecordSetServiceSpec.scala @@ -24,6 +24,7 @@ import org.scalatestplus.mockito.MockitoSugar import org.scalatest.matchers.should.Matchers import org.scalatest.wordspec.AnyWordSpec import org.scalatest.BeforeAndAfterEach +import vinyldns.api.config.{ZoneAuthConfigs, DottedHostsConfig} import vinyldns.api.{ResultHelpers, VinylDNSTestHelpers} import vinyldns.api.domain.access.AccessValidations import vinyldns.api.domain.record.RecordSetHelpers._ @@ -83,6 +84,7 @@ class RecordSetServiceSpec mockBackendResolver, false, VinylDNSTestHelpers.highValueDomainConfig, + VinylDNSTestHelpers.dottedHostsConfig, VinylDNSTestHelpers.approvedNameServers, true ) @@ -101,10 +103,57 @@ class RecordSetServiceSpec mockBackendResolver, true, VinylDNSTestHelpers.highValueDomainConfig, + VinylDNSTestHelpers.dottedHostsConfig, VinylDNSTestHelpers.approvedNameServers, true ) + val underTestWithEmptyDottedHostsConfig = new RecordSetService( + mockZoneRepo, + mockGroupRepo, + mockRecordRepo, + mockRecordDataRepo, + mockRecordChangeRepo, + mockUserRepo, + mockMessageQueue, + new AccessValidations( + sharedApprovedTypes = VinylDNSTestHelpers.sharedApprovedTypes + ), + mockBackendResolver, + true, + VinylDNSTestHelpers.highValueDomainConfig, + VinylDNSTestHelpers.emptyDottedHostsConfig, + VinylDNSTestHelpers.approvedNameServers, + true + ) + + def getDottedHostsConfigGroupsAllowed(zone: Zone, config: DottedHostsConfig): List[String] = { + val configZones = config.zoneAuthConfigs.map(x => x.zone) + val zoneName = if(zone.name.takeRight(1) != ".") zone.name + "." else zone.name + val dottedZoneConfig = configZones.filter(_.contains("*")).map(_.replace("*", "[A-Za-z.]*")) + val isContainWildcardZone = dottedZoneConfig.exists(x => zoneName.substring(0, zoneName.length - 1).matches(x)) + val isContainNormalZone = configZones.contains(zoneName) + val groups = if (isContainWildcardZone || isContainNormalZone) { + config.zoneAuthConfigs.flatMap { + x: ZoneAuthConfigs => + if (x.zone.contains("*")) { + val wildcardZone = x.zone.replace("*", "[A-Za-z.]*") + if (zoneName.substring(0, zoneName.length - 1).matches(wildcardZone)) x.groupList else List.empty + } else { + if (x.zone == zoneName) x.groupList else List.empty + } + } + } + else { + List.empty + } + groups + } + + val dottedHostsConfigZonesAllowed: List[String] = VinylDNSTestHelpers.dottedHostsConfig.zoneAuthConfigs.map(x => x.zone) + + val dottedHostsConfigGroupsAllowed: List[String] = getDottedHostsConfigGroupsAllowed(okZone, VinylDNSTestHelpers.dottedHostsConfig) + "addRecordSet" should { "return the recordSet change as the result" in { val record = aaaa.copy(zoneId = okZone.id) @@ -115,6 +164,27 @@ class RecordSetServiceSpec doReturn(IO.pure(List())) .when(mockRecordRepo) .getRecordSetsByName(okZone.id, record.name) + doReturn(IO.pure(Set(dottedZone, abcZone, xyzZone, dotZone))) + .when(mockZoneRepo) + .getZonesByNames(dottedHostsConfigZonesAllowed.toSet) + doReturn(IO.pure(Set())) + .when(mockZoneRepo) + .getZonesByFilters(Set.empty) + doReturn(IO.pure(None)) + .when(mockZoneRepo) + .getZoneByName(record.name + "." + okZone.name) + doReturn(IO.pure(List())) + .when(mockRecordRepo) + .getRecordSetsByFQDNs(Set(record.name + "." + okZone.name)) + doReturn(IO.pure(Set())) + .when(mockZoneRepo) + .getZonesByFilters(Set.empty) + doReturn(IO.pure(Set())) + .when(mockGroupRepo) + .getGroupsByName(dottedHostsConfigGroupsAllowed.toSet) + doReturn(IO.pure(ListUsersResults(Seq(), None))) + .when(mockUserRepo) + .getUsers(Set.empty, None, None) val result: RecordSetChange = rightResultOf( @@ -132,7 +202,6 @@ class RecordSetServiceSpec val result = leftResultOf(underTest.getRecordSetByZone(aaaa.id, mockZone.id, okAuth).value) result shouldBe a[ZoneNotFoundError] } - "fail when the account is not authorized" in { doReturn(IO.pure(Some(aaaa))) .when(mockRecordRepo) @@ -155,7 +224,7 @@ class RecordSetServiceSpec val result = leftResultOf(underTest.addRecordSet(aaaa, okAuth).value) result shouldBe a[RecordSetAlreadyExists] } - "fail if the record is dotted" in { + "fail if the record is dotted and does not satisfy properties in dotted hosts config" in { val record = aaaa.copy(name = "new.name", zoneId = okZone.id, status = RecordSetStatus.Active) @@ -165,10 +234,66 @@ class RecordSetServiceSpec doReturn(IO.pure(List())) .when(mockRecordRepo) .getRecordSetsByName(okZone.id, record.name) + doReturn(IO.pure(Set(dottedZone, abcZone, xyzZone, dotZone))) + .when(mockZoneRepo) + .getZonesByNames(dottedHostsConfigZonesAllowed.toSet) + doReturn(IO.pure(Set())) + .when(mockZoneRepo) + .getZonesByFilters(Set.empty) + doReturn(IO.pure(None)) + .when(mockZoneRepo) + .getZoneByName(record.name + "." + okZone.name) + doReturn(IO.pure(List())) + .when(mockRecordRepo) + .getRecordSetsByFQDNs(Set(record.name + "." + okZone.name)) + doReturn(IO.pure(Set())) + .when(mockZoneRepo) + .getZonesByFilters(record.name.split('.').map(x => x + "." + okZone.name).toSet) + doReturn(IO.pure(Set())) + .when(mockGroupRepo) + .getGroupsByName(dottedHostsConfigGroupsAllowed.toSet) + doReturn(IO.pure(ListUsersResults(Seq(), None))) + .when(mockUserRepo) + .getUsers(Set.empty, None, None) val result = leftResultOf(underTest.addRecordSet(record, okAuth).value) result shouldBe an[InvalidRequest] } + "fail if the record is dotted and dotted hosts config is empty" in { + val record = + aaaa.copy(name = "new.name", zoneId = okZone.id, status = RecordSetStatus.Active) + + doReturn(IO.pure(List())) + .when(mockRecordRepo) + .getRecordSets(okZone.id, record.name, record.typ) + doReturn(IO.pure(List())) + .when(mockRecordRepo) + .getRecordSetsByName(okZone.id, record.name) + doReturn(IO.pure(Set())) + .when(mockZoneRepo) + .getZonesByNames(Set.empty) + doReturn(IO.pure(Set())) + .when(mockZoneRepo) + .getZonesByFilters(Set.empty) + doReturn(IO.pure(None)) + .when(mockZoneRepo) + .getZoneByName(record.name + "." + okZone.name) + doReturn(IO.pure(List())) + .when(mockRecordRepo) + .getRecordSetsByFQDNs(Set(record.name + "." + okZone.name)) + doReturn(IO.pure(Set())) + .when(mockZoneRepo) + .getZonesByFilters(record.name.split('.').map(x => x + "." + okZone.name).toSet) + doReturn(IO.pure(Set())) + .when(mockGroupRepo) + .getGroupsByName(dottedHostsConfigGroupsAllowed.toSet) + doReturn(IO.pure(ListUsersResults(Seq(), None))) + .when(mockUserRepo) + .getUsers(Set.empty, None, None) + + val result = leftResultOf(underTestWithEmptyDottedHostsConfig.addRecordSet(record, okAuth).value) + result shouldBe an[InvalidRequest] + } "fail if the record is relative with trailing dot" in { val record = aaaa.copy(name = "new.", zoneId = okZone.id, status = RecordSetStatus.Active) @@ -179,6 +304,27 @@ class RecordSetServiceSpec doReturn(IO.pure(List())) .when(mockRecordRepo) .getRecordSetsByName(okZone.id, record.name) + doReturn(IO.pure(Set(dottedZone, abcZone, xyzZone, dotZone))) + .when(mockZoneRepo) + .getZonesByNames(dottedHostsConfigZonesAllowed.toSet) + doReturn(IO.pure(Set())) + .when(mockZoneRepo) + .getZonesByFilters(Set.empty) + doReturn(IO.pure(None)) + .when(mockZoneRepo) + .getZoneByName(record.name + "." + okZone.name) + doReturn(IO.pure(List())) + .when(mockRecordRepo) + .getRecordSetsByFQDNs(Set(record.name + "." + okZone.name)) + doReturn(IO.pure(Set())) + .when(mockZoneRepo) + .getZonesByFilters(record.name.split('.').map(x => x + "." + okZone.name).toSet) + doReturn(IO.pure(Set())) + .when(mockGroupRepo) + .getGroupsByName(dottedHostsConfigGroupsAllowed.toSet) + doReturn(IO.pure(ListUsersResults(Seq(), None))) + .when(mockUserRepo) + .getUsers(Set.empty, None, None) val result = leftResultOf(underTestWithDnsBackendValidations.addRecordSet(record, okAuth).value) @@ -204,6 +350,27 @@ class RecordSetServiceSpec doReturn(IO.pure(List())) .when(mockRecordRepo) .getRecordSetsByName(okZone.id, record.name) + doReturn(IO.pure(Set(dottedZone, abcZone, xyzZone, dotZone))) + .when(mockZoneRepo) + .getZonesByNames(dottedHostsConfigZonesAllowed.toSet) + doReturn(IO.pure(Set())) + .when(mockZoneRepo) + .getZonesByFilters(Set.empty) + doReturn(IO.pure(None)) + .when(mockZoneRepo) + .getZoneByName(record.name) + doReturn(IO.pure(List())) + .when(mockRecordRepo) + .getRecordSetsByFQDNs(Set(record.name)) + doReturn(IO.pure(Set())) + .when(mockZoneRepo) + .getZonesByFilters(record.name.split('.').map(x => x + "." + okZone.name).toSet) + doReturn(IO.pure(Set())) + .when(mockGroupRepo) + .getGroupsByName(dottedHostsConfigGroupsAllowed.toSet) + doReturn(IO.pure(ListUsersResults(Seq(), None))) + .when(mockUserRepo) + .getUsers(Set.empty, None, None) val result: RecordSetChange = rightResultOf( underTest.addRecordSet(record, okAuth).map(_.asInstanceOf[RecordSetChange]).value @@ -222,6 +389,27 @@ class RecordSetServiceSpec doReturn(IO.pure(List())) .when(mockRecordRepo) .getRecordSetsByName(okZone.id, record.name) + doReturn(IO.pure(Set(dottedZone, abcZone, xyzZone, dotZone))) + .when(mockZoneRepo) + .getZonesByNames(dottedHostsConfigZonesAllowed.toSet) + doReturn(IO.pure(Set())) + .when(mockZoneRepo) + .getZonesByFilters(Set.empty) + doReturn(IO.pure(None)) + .when(mockZoneRepo) + .getZoneByName(record.name) + doReturn(IO.pure(List())) + .when(mockRecordRepo) + .getRecordSetsByFQDNs(Set(record.name)) + doReturn(IO.pure(Set())) + .when(mockZoneRepo) + .getZonesByFilters(record.name.split('.').map(x => x + "." + okZone.name).toSet) + doReturn(IO.pure(Set())) + .when(mockGroupRepo) + .getGroupsByName(dottedHostsConfigGroupsAllowed.toSet) + doReturn(IO.pure(ListUsersResults(Seq(), None))) + .when(mockUserRepo) + .getUsers(Set.empty, None, None) val result: RecordSetChange = rightResultOf( underTest.addRecordSet(record, okAuth).map(_.asInstanceOf[RecordSetChange]).value @@ -259,6 +447,27 @@ class RecordSetServiceSpec doReturn(IO.pure(Some(okGroup))) .when(mockGroupRepo) .getGroup(okGroup.id) + doReturn(IO.pure(Set(dottedZone, abcZone, xyzZone, dotZone))) + .when(mockZoneRepo) + .getZonesByNames(dottedHostsConfigZonesAllowed.toSet) + doReturn(IO.pure(Set())) + .when(mockZoneRepo) + .getZonesByFilters(Set.empty) + doReturn(IO.pure(None)) + .when(mockZoneRepo) + .getZoneByName(record.name + "." + okZone.name) + doReturn(IO.pure(List())) + .when(mockRecordRepo) + .getRecordSetsByFQDNs(Set(record.name + "." + okZone.name)) + doReturn(IO.pure(Set())) + .when(mockZoneRepo) + .getZonesByFilters(Set.empty) + doReturn(IO.pure(Set())) + .when(mockGroupRepo) + .getGroupsByName(dottedHostsConfigGroupsAllowed.toSet) + doReturn(IO.pure(ListUsersResults(Seq(), None))) + .when(mockUserRepo) + .getUsers(Set.empty, None, None) val result: RecordSetChange = rightResultOf( underTest.addRecordSet(record, okAuth).map(_.asInstanceOf[RecordSetChange]).value @@ -312,6 +521,27 @@ class RecordSetServiceSpec doReturn(IO.pure(List())) .when(mockRecordRepo) .getRecordSetsByName(okZone.id, record.name) + doReturn(IO.pure(Set(dottedZone, abcZone, xyzZone, dotZone))) + .when(mockZoneRepo) + .getZonesByNames(dottedHostsConfigZonesAllowed.toSet) + doReturn(IO.pure(Set())) + .when(mockZoneRepo) + .getZonesByFilters(Set.empty) + doReturn(IO.pure(None)) + .when(mockZoneRepo) + .getZoneByName(record.name + "." + okZone.name) + doReturn(IO.pure(List())) + .when(mockRecordRepo) + .getRecordSetsByFQDNs(Set(record.name + "." + okZone.name)) + doReturn(IO.pure(Set())) + .when(mockZoneRepo) + .getZonesByFilters(Set.empty) + doReturn(IO.pure(Set())) + .when(mockGroupRepo) + .getGroupsByName(dottedHostsConfigGroupsAllowed.toSet) + doReturn(IO.pure(ListUsersResults(Seq(), None))) + .when(mockUserRepo) + .getUsers(Set.empty, None, None) val result: RecordSetChange = rightResultOf( @@ -326,6 +556,296 @@ class RecordSetServiceSpec result.status shouldBe RecordSetChangeStatus.Pending } } + "succeed if the record is dotted and zone, user, record type is in allowed dotted hosts config" in { + val record = + cname.copy(name = "new.name", zoneId = dottedZone.id, status = RecordSetStatus.Active) + + val dottedHostsConfigZonesAllowed: List[String] = VinylDNSTestHelpers.dottedHostsConfig.zoneAuthConfigs.map(x => x.zone) + + val dottedHostsConfigGroupsAllowed: List[String] = getDottedHostsConfigGroupsAllowed(dottedZone, VinylDNSTestHelpers.dottedHostsConfig) + + doReturn(IO.pure(Some(dottedZone))).when(mockZoneRepo).getZone(dottedZone.id) + doReturn(IO.pure(List())) + .when(mockRecordRepo) + .getRecordSets(dottedZone.id, record.name, record.typ) + doReturn(IO.pure(List())) + .when(mockRecordRepo) + .getRecordSetsByName(dottedZone.id, record.name) + doReturn(IO.pure(Set(dottedZone, abcZone, xyzZone, dotZone))) + .when(mockZoneRepo) + .getZonesByNames(dottedHostsConfigZonesAllowed.toSet) + doReturn(IO.pure(Set())) + .when(mockZoneRepo) + .getZonesByFilters(Set.empty) + doReturn(IO.pure(None)) + .when(mockZoneRepo) + .getZoneByName(record.name + "." + dottedZone.name) + doReturn(IO.pure(List())) + .when(mockRecordRepo) + .getRecordSetsByFQDNs(Set(record.name + "." + dottedZone.name)) + doReturn(IO.pure(Set())) + .when(mockZoneRepo) + .getZonesByFilters(record.name.split('.').map(x => x + "." + dottedZone.name).toSet) + doReturn(IO.pure(Set(dummyGroup))) + .when(mockGroupRepo) + .getGroupsByName(dottedHostsConfigGroupsAllowed.toSet) + doReturn(IO.pure(ListUsersResults(listOfDummyUsers.toSeq, None))) + .when(mockUserRepo) + .getUsers(dummyGroup.memberIds, None, None) + + // passes as all three properties within dotted hosts config (allowed zones, users and record types) are satisfied + val result: RecordSetChange = rightResultOf( + underTest.addRecordSet(record, xyzAuth).map(_.asInstanceOf[RecordSetChange]).value + ) + + result.recordSet.name shouldBe record.name + } + "succeed if the record is dotted and zone, user in group, record type is in allowed dotted hosts config" in { + val record = + cname.copy(name = "new.name", zoneId = xyzZone.id, status = RecordSetStatus.Active) + + val dottedHostsConfigZonesAllowed: List[String] = VinylDNSTestHelpers.dottedHostsConfig.zoneAuthConfigs.map(x => x.zone) + + val dottedHostsConfigGroupsAllowed: List[String] = getDottedHostsConfigGroupsAllowed(xyzZone, VinylDNSTestHelpers.dottedHostsConfig) + + doReturn(IO.pure(Some(xyzZone))).when(mockZoneRepo).getZone(xyzZone.id) + doReturn(IO.pure(List())) + .when(mockRecordRepo) + .getRecordSets(xyzZone.id, record.name, record.typ) + doReturn(IO.pure(List())) + .when(mockRecordRepo) + .getRecordSetsByName(xyzZone.id, record.name) + doReturn(IO.pure(Set(xyzZone, abcZone, xyzZone))) + .when(mockZoneRepo) + .getZonesByNames(dottedHostsConfigZonesAllowed.toSet) + doReturn(IO.pure(Set())) + .when(mockZoneRepo) + .getZonesByFilters(Set.empty) + doReturn(IO.pure(None)) + .when(mockZoneRepo) + .getZoneByName(record.name + "." + xyzZone.name) + doReturn(IO.pure(List())) + .when(mockRecordRepo) + .getRecordSetsByFQDNs(Set(record.name + "." + xyzZone.name)) + doReturn(IO.pure(Set())) + .when(mockZoneRepo) + .getZonesByFilters(record.name.split('.').map(x => x + "." + xyzZone.name).toSet) + doReturn(IO.pure(Set(xyzGroup))) + .when(mockGroupRepo) + .getGroupsByName(dottedHostsConfigGroupsAllowed.toSet) + doReturn(IO.pure(ListUsersResults(Seq(xyzUser), None))) + .when(mockUserRepo) + .getUsers(xyzGroup.memberIds, None, None) + + // passes as all three properties within dotted hosts config (allowed zones, users and record types) are satisfied + val result: RecordSetChange = rightResultOf( + underTest.addRecordSet(record, xyzAuth).map(_.asInstanceOf[RecordSetChange]).value + ) + + result.recordSet.name shouldBe record.name + } + "fail if the record is dotted and zone, user in group, record type is allowed but record name has dot in the end and is not an apex record" in { + val record = + cname.copy(name = "new.name.", zoneId = xyzZone.id, status = RecordSetStatus.Active) + + val dottedHostsConfigZonesAllowed: List[String] = VinylDNSTestHelpers.dottedHostsConfig.zoneAuthConfigs.map(x => x.zone) + + val dottedHostsConfigGroupsAllowed: List[String] = getDottedHostsConfigGroupsAllowed(xyzZone, VinylDNSTestHelpers.dottedHostsConfig) + + doReturn(IO.pure(Some(xyzZone))).when(mockZoneRepo).getZone(xyzZone.id) + doReturn(IO.pure(List())) + .when(mockRecordRepo) + .getRecordSets(xyzZone.id, record.name, record.typ) + doReturn(IO.pure(List())) + .when(mockRecordRepo) + .getRecordSetsByName(xyzZone.id, record.name) + doReturn(IO.pure(Set(xyzZone, abcZone, xyzZone))) + .when(mockZoneRepo) + .getZonesByNames(dottedHostsConfigZonesAllowed.toSet) + doReturn(IO.pure(Set())) + .when(mockZoneRepo) + .getZonesByFilters(Set.empty) + doReturn(IO.pure(None)) + .when(mockZoneRepo) + .getZoneByName(record.name + "." + xyzZone.name) + doReturn(IO.pure(List())) + .when(mockRecordRepo) + .getRecordSetsByFQDNs(Set(record.name + "." + xyzZone.name)) + doReturn(IO.pure(Set())) + .when(mockZoneRepo) + .getZonesByFilters(record.name.split('.').map(x => x + "." + xyzZone.name).toSet) + doReturn(IO.pure(Set(xyzGroup))) + .when(mockGroupRepo) + .getGroupsByName(dottedHostsConfigGroupsAllowed.toSet) + doReturn(IO.pure(ListUsersResults(Seq(xyzUser), None))) + .when(mockUserRepo) + .getUsers(xyzGroup.memberIds, None, None) + + // fails as dotted host record name has dot at the end and is not an apex record + val result = leftResultOf(underTest.addRecordSet(record, xyzAuth).value) + result shouldBe an[InvalidRequest] + } + "fail if the record is dotted and zone, user, record type is allowed but number of dots allowed in config is 0" in { + val record = + cname.copy(name = "new.name", zoneId = dotZone.id, status = RecordSetStatus.Active) + + val dottedHostsConfigZonesAllowed: List[String] = VinylDNSTestHelpers.dottedHostsConfig.zoneAuthConfigs.map(x => x.zone) + + val dottedHostsConfigGroupsAllowed: List[String] = getDottedHostsConfigGroupsAllowed(dottedZone, VinylDNSTestHelpers.dottedHostsConfig) + + doReturn(IO.pure(Some(dotZone))).when(mockZoneRepo).getZone(dotZone.id) + doReturn(IO.pure(List())) + .when(mockRecordRepo) + .getRecordSets(dotZone.id, record.name, record.typ) + doReturn(IO.pure(List())) + .when(mockRecordRepo) + .getRecordSetsByName(dotZone.id, record.name) + doReturn(IO.pure(Set(dottedZone, abcZone, xyzZone, dotZone))) + .when(mockZoneRepo) + .getZonesByNames(dottedHostsConfigZonesAllowed.toSet) + doReturn(IO.pure(Set())) + .when(mockZoneRepo) + .getZonesByFilters(Set.empty) + doReturn(IO.pure(None)) + .when(mockZoneRepo) + .getZoneByName(record.name + "." + dotZone.name) + doReturn(IO.pure(List())) + .when(mockRecordRepo) + .getRecordSetsByFQDNs(Set(record.name + "." + dotZone.name)) + doReturn(IO.pure(Set())) + .when(mockZoneRepo) + .getZonesByFilters(record.name.split('.').map(x => x + "." + dotZone.name).toSet) + doReturn(IO.pure(Set(dummyGroup))) + .when(mockGroupRepo) + .getGroupsByName(dottedHostsConfigGroupsAllowed.toSet) + doReturn(IO.pure(ListUsersResults(listOfDummyUsers.toSeq, None))) + .when(mockUserRepo) + .getUsers(dummyGroup.memberIds, None, None) + + // fails as no.of.dots allowed for the zone in the config is 0 + val result = leftResultOf(underTest.addRecordSet(record, xyzAuth).value) + result shouldBe an[InvalidRequest] + } + "fail if the record is dotted and user, record type is in allowed dotted hosts config except zone" in { + val record = + cname.copy(name = "new.name", zoneId = okZone.id, status = RecordSetStatus.Active) + + doReturn(IO.pure(List())) + .when(mockRecordRepo) + .getRecordSets(okZone.id, record.name, record.typ) + doReturn(IO.pure(List())) + .when(mockRecordRepo) + .getRecordSetsByName(okZone.id, record.name) + doReturn(IO.pure(Set(dottedZone, abcZone, xyzZone, dotZone))) + .when(mockZoneRepo) + .getZonesByNames(dottedHostsConfigZonesAllowed.toSet) + doReturn(IO.pure(Set())) + .when(mockZoneRepo) + .getZonesByFilters(Set.empty) + doReturn(IO.pure(None)) + .when(mockZoneRepo) + .getZoneByName(record.name + "." + okZone.name) + doReturn(IO.pure(List())) + .when(mockRecordRepo) + .getRecordSetsByFQDNs(Set(record.name + "." + okZone.name)) + doReturn(IO.pure(Set())) + .when(mockZoneRepo) + .getZonesByFilters(record.name.split('.').map(x => x + "." + okZone.name).toSet) + doReturn(IO.pure(Set())) + .when(mockGroupRepo) + .getGroupsByName(dottedHostsConfigGroupsAllowed.toSet) + doReturn(IO.pure(ListUsersResults(Seq(), None))) + .when(mockUserRepo) + .getUsers(Set.empty, None, None) + + // fails as only two properties within dotted hosts config (users and record types) are satisfied while zone is not allowed + val result = leftResultOf(underTest.addRecordSet(record, okAuth).value) + result shouldBe an[InvalidRequest] + } + "fail if the record is dotted and zone, record type is in allowed dotted hosts config except user" in { + val record = + cname.copy(name = "new.name", zoneId = abcZone.id, status = RecordSetStatus.Active) + + val dottedHostsConfigZonesAllowed: List[String] = VinylDNSTestHelpers.dottedHostsConfig.zoneAuthConfigs.map(x => x.zone) + + val dottedHostsConfigGroupsAllowed: List[String] = getDottedHostsConfigGroupsAllowed(abcZone, VinylDNSTestHelpers.dottedHostsConfig) + + doReturn(IO.pure(Some(abcZone))).when(mockZoneRepo).getZone(abcZone.id) + doReturn(IO.pure(List())) + .when(mockRecordRepo) + .getRecordSets(abcZone.id, record.name, record.typ) + doReturn(IO.pure(List())) + .when(mockRecordRepo) + .getRecordSetsByName(abcZone.id, record.name) + doReturn(IO.pure(Set(abcZone, dottedZone, xyzZone))) + .when(mockZoneRepo) + .getZonesByNames(dottedHostsConfigZonesAllowed.toSet) + doReturn(IO.pure(Set())) + .when(mockZoneRepo) + .getZonesByFilters(Set.empty) + doReturn(IO.pure(None)) + .when(mockZoneRepo) + .getZoneByName(record.name + "." + abcZone.name) + doReturn(IO.pure(List())) + .when(mockRecordRepo) + .getRecordSetsByFQDNs(Set(record.name + "." + abcZone.name)) + doReturn(IO.pure(Set())) + .when(mockZoneRepo) + .getZonesByFilters(record.name.split('.').map(x => x + "." + abcZone.name).toSet) + doReturn(IO.pure(Set(dummyGroup))) + .when(mockGroupRepo) + .getGroupsByName(dottedHostsConfigGroupsAllowed.toSet) + doReturn(IO.pure(ListUsersResults(listOfDummyUsers.toSeq, None))) + .when(mockUserRepo) + .getUsers(dummyGroup.memberIds, None, None) + + // fails as only two properties within dotted hosts config (zones and record types) are satisfied while user is not allowed + val result = leftResultOf(underTest.addRecordSet(record, abcAuth).value) + result shouldBe an[InvalidRequest] + } + "fail if the record is dotted and zone, user is in allowed dotted hosts config except record type" in { + val record = + aaaa.copy(name = "new.name", zoneId = dottedZone.id, status = RecordSetStatus.Active) + + val dottedHostsConfigZonesAllowed: List[String] = VinylDNSTestHelpers.dottedHostsConfig.zoneAuthConfigs.map { + case y:ZoneAuthConfigs => y.zone + } + + val dottedHostsConfigGroupsAllowed: List[String] = getDottedHostsConfigGroupsAllowed(dottedZone, VinylDNSTestHelpers.dottedHostsConfig) + + doReturn(IO.pure(Some(dottedZone))).when(mockZoneRepo).getZone(dottedZone.id) + doReturn(IO.pure(List())) + .when(mockRecordRepo) + .getRecordSets(dottedZone.id, record.name, record.typ) + doReturn(IO.pure(List())) + .when(mockRecordRepo) + .getRecordSetsByName(dottedZone.id, record.name) + doReturn(IO.pure(Set(dottedZone, abcZone, xyzZone, dotZone))) + .when(mockZoneRepo) + .getZonesByNames(dottedHostsConfigZonesAllowed.toSet) + doReturn(IO.pure(Set())) + .when(mockZoneRepo) + .getZonesByFilters(Set.empty) + doReturn(IO.pure(None)) + .when(mockZoneRepo) + .getZoneByName(record.name + "." + dottedZone.name) + doReturn(IO.pure(List())) + .when(mockRecordRepo) + .getRecordSetsByFQDNs(Set(record.name + "." + dottedZone.name)) + doReturn(IO.pure(Set())) + .when(mockZoneRepo) + .getZonesByFilters(record.name.split('.').map(x => x + "." + dottedZone.name).toSet) + doReturn(IO.pure(Set(dummyGroup))) + .when(mockGroupRepo) + .getGroupsByName(dottedHostsConfigGroupsAllowed.toSet) + doReturn(IO.pure(ListUsersResults(listOfDummyUsers.toSeq, None))) + .when(mockUserRepo) + .getUsers(dummyGroup.memberIds, None, None) + + // fails as only two properties within dotted hosts config (zone and user) are satisfied while record type is not allowed + val result = leftResultOf(underTest.addRecordSet(record, xyzAuth).value) + result shouldBe an[InvalidRequest] + } "updateRecordSet" should { "return the recordSet change as the result" in { @@ -341,6 +861,27 @@ class RecordSetServiceSpec doReturn(IO.pure(List())) .when(mockRecordRepo) .getRecordSetsByName(okZone.id, newRecord.name) + doReturn(IO.pure(Set(dottedZone, abcZone, xyzZone, dotZone))) + .when(mockZoneRepo) + .getZonesByNames(dottedHostsConfigZonesAllowed.toSet) + doReturn(IO.pure(Set())) + .when(mockZoneRepo) + .getZonesByFilters(Set.empty) + doReturn(IO.pure(None)) + .when(mockZoneRepo) + .getZoneByName(newRecord.name + "." + okZone.name) + doReturn(IO.pure(List())) + .when(mockRecordRepo) + .getRecordSetsByFQDNs(Set(newRecord.name + "." + okZone.name)) + doReturn(IO.pure(Set())) + .when(mockZoneRepo) + .getZonesByFilters(Set.empty) + doReturn(IO.pure(Set())) + .when(mockGroupRepo) + .getGroupsByName(dottedHostsConfigGroupsAllowed.toSet) + doReturn(IO.pure(ListUsersResults(Seq(), None))) + .when(mockUserRepo) + .getUsers(Set.empty, None, None) val result: RecordSetChange = rightResultOf( underTest.updateRecordSet(newRecord, okAuth).map(_.asInstanceOf[RecordSetChange]).value @@ -377,6 +918,27 @@ class RecordSetServiceSpec doReturn(IO.pure(List())) .when(mockRecordRepo) .getRecordSetsByName(okZone.id, newRecord.name) + doReturn(IO.pure(Set(dottedZone, abcZone, xyzZone, dotZone))) + .when(mockZoneRepo) + .getZonesByNames(dottedHostsConfigZonesAllowed.toSet) + doReturn(IO.pure(Set())) + .when(mockZoneRepo) + .getZonesByFilters(Set.empty) + doReturn(IO.pure(None)) + .when(mockZoneRepo) + .getZoneByName(newRecord.name + "." + okZone.name) + doReturn(IO.pure(List())) + .when(mockRecordRepo) + .getRecordSetsByFQDNs(Set(newRecord.name + "." + okZone.name)) + doReturn(IO.pure(Set())) + .when(mockZoneRepo) + .getZonesByFilters(newRecord.name.split('.').map(x => x + "." + okZone.name).toSet) + doReturn(IO.pure(Set())) + .when(mockGroupRepo) + .getGroupsByName(dottedHostsConfigGroupsAllowed.toSet) + doReturn(IO.pure(ListUsersResults(Seq(), None))) + .when(mockUserRepo) + .getUsers(Set.empty, None, None) val result: RecordSetChange = rightResultOf( underTest.updateRecordSet(newRecord, okAuth).map(_.asInstanceOf[RecordSetChange]).value @@ -416,6 +978,27 @@ class RecordSetServiceSpec doReturn(IO.pure(List())) .when(mockRecordRepo) .getRecordSetsByName(okZone.id, newRecord.name) + doReturn(IO.pure(Set(dottedZone, abcZone, xyzZone, dotZone))) + .when(mockZoneRepo) + .getZonesByNames(dottedHostsConfigZonesAllowed.toSet) + doReturn(IO.pure(Set())) + .when(mockZoneRepo) + .getZonesByFilters(Set.empty) + doReturn(IO.pure(None)) + .when(mockZoneRepo) + .getZoneByName(newRecord.name) + doReturn(IO.pure(List())) + .when(mockRecordRepo) + .getRecordSetsByFQDNs(Set(newRecord.name)) + doReturn(IO.pure(Set())) + .when(mockZoneRepo) + .getZonesByFilters(newRecord.name.split('.').map(x => x + "." + okZone.name).toSet) + doReturn(IO.pure(Set())) + .when(mockGroupRepo) + .getGroupsByName(dottedHostsConfigGroupsAllowed.toSet) + doReturn(IO.pure(ListUsersResults(Seq(), None))) + .when(mockUserRepo) + .getUsers(Set.empty, None, None) val result: RecordSetChange = rightResultOf( underTest.updateRecordSet(newRecord, okAuth).map(_.asInstanceOf[RecordSetChange]).value @@ -438,6 +1021,27 @@ class RecordSetServiceSpec doReturn(IO.pure(List())) .when(mockRecordRepo) .getRecordSetsByName(okZone.id, newRecord.name) + doReturn(IO.pure(Set(dottedZone, abcZone, xyzZone, dotZone))) + .when(mockZoneRepo) + .getZonesByNames(dottedHostsConfigZonesAllowed.toSet) + doReturn(IO.pure(Set())) + .when(mockZoneRepo) + .getZonesByFilters(Set.empty) + doReturn(IO.pure(None)) + .when(mockZoneRepo) + .getZoneByName(newRecord.name) + doReturn(IO.pure(List())) + .when(mockRecordRepo) + .getRecordSetsByFQDNs(Set(newRecord.name)) + doReturn(IO.pure(Set())) + .when(mockZoneRepo) + .getZonesByFilters(newRecord.name.split('.').map(x => x + "." + okZone.name).toSet) + doReturn(IO.pure(Set())) + .when(mockGroupRepo) + .getGroupsByName(dottedHostsConfigGroupsAllowed.toSet) + doReturn(IO.pure(ListUsersResults(Seq(), None))) + .when(mockUserRepo) + .getUsers(Set.empty, None, None) val result: RecordSetChange = rightResultOf( underTest.updateRecordSet(newRecord, okAuth).map(_.asInstanceOf[RecordSetChange]).value @@ -460,6 +1064,27 @@ class RecordSetServiceSpec doReturn(IO.pure(List())) .when(mockRecordRepo) .getRecordSetsByName(okZone.id, newRecord.name) + doReturn(IO.pure(Set(dottedZone, abcZone, xyzZone, dotZone))) + .when(mockZoneRepo) + .getZonesByNames(dottedHostsConfigZonesAllowed.toSet) + doReturn(IO.pure(Set())) + .when(mockZoneRepo) + .getZonesByFilters(Set.empty) + doReturn(IO.pure(None)) + .when(mockZoneRepo) + .getZoneByName(newRecord.name) + doReturn(IO.pure(List())) + .when(mockRecordRepo) + .getRecordSetsByFQDNs(Set(newRecord.name)) + doReturn(IO.pure(Set())) + .when(mockZoneRepo) + .getZonesByFilters(newRecord.name.split('.').map(x => x + "." + okZone.name).toSet) + doReturn(IO.pure(Set())) + .when(mockGroupRepo) + .getGroupsByName(dottedHostsConfigGroupsAllowed.toSet) + doReturn(IO.pure(ListUsersResults(Seq(), None))) + .when(mockUserRepo) + .getUsers(Set.empty, None, None) val result: RecordSetChange = rightResultOf( underTest.updateRecordSet(newRecord, okAuth).map(_.asInstanceOf[RecordSetChange]).value @@ -595,6 +1220,27 @@ class RecordSetServiceSpec doReturn(IO.pure(Some(oneUserDummyGroup))) .when(mockGroupRepo) .getGroup(oneUserDummyGroup.id) + doReturn(IO.pure(Set(dottedZone, abcZone, xyzZone, dotZone))) + .when(mockZoneRepo) + .getZonesByNames(dottedHostsConfigZonesAllowed.toSet) + doReturn(IO.pure(Set())) + .when(mockZoneRepo) + .getZonesByFilters(Set.empty) + doReturn(IO.pure(None)) + .when(mockZoneRepo) + .getZoneByName(newRecord.name + "." + okZone.name) + doReturn(IO.pure(List())) + .when(mockRecordRepo) + .getRecordSetsByFQDNs(Set(newRecord.name + "." + okZone.name)) + doReturn(IO.pure(Set())) + .when(mockZoneRepo) + .getZonesByFilters(Set.empty) + doReturn(IO.pure(Set())) + .when(mockGroupRepo) + .getGroupsByName(dottedHostsConfigGroupsAllowed.toSet) + doReturn(IO.pure(ListUsersResults(Seq(), None))) + .when(mockUserRepo) + .getUsers(Set.empty, None, None) val result = rightResultOf( underTest.updateRecordSet(newRecord, auth).map(_.asInstanceOf[RecordSetChange]).value @@ -624,6 +1270,27 @@ class RecordSetServiceSpec doReturn(IO.pure(List(oldRecord))) .when(mockRecordRepo) .getRecordSetsByName(zone.id, newRecord.name) + doReturn(IO.pure(Set(dottedZone, abcZone, xyzZone))) + .when(mockZoneRepo) + .getZonesByNames(dottedHostsConfigZonesAllowed.toSet) + doReturn(IO.pure(Set())) + .when(mockZoneRepo) + .getZonesByFilters(Set.empty) + doReturn(IO.pure(None)) + .when(mockZoneRepo) + .getZoneByName(newRecord.name + "." + okZone.name) + doReturn(IO.pure(List())) + .when(mockRecordRepo) + .getRecordSetsByFQDNs(Set(newRecord.name + "." + okZone.name)) + doReturn(IO.pure(Set())) + .when(mockZoneRepo) + .getZonesByFilters(Set.empty) + doReturn(IO.pure(Set())) + .when(mockGroupRepo) + .getGroupsByName(dottedHostsConfigGroupsAllowed.toSet) + doReturn(IO.pure(ListUsersResults(Seq(), None))) + .when(mockUserRepo) + .getUsers(Set.empty, None, None) val result = rightResultOf( underTest.updateRecordSet(newRecord, auth).map(_.asInstanceOf[RecordSetChange]).value diff --git a/modules/api/src/test/scala/vinyldns/api/domain/record/RecordSetValidationsSpec.scala b/modules/api/src/test/scala/vinyldns/api/domain/record/RecordSetValidationsSpec.scala index 304f65ed4..e0a03905e 100644 --- a/modules/api/src/test/scala/vinyldns/api/domain/record/RecordSetValidationsSpec.scala +++ b/modules/api/src/test/scala/vinyldns/api/domain/record/RecordSetValidationsSpec.scala @@ -44,6 +44,8 @@ class RecordSetValidationsSpec import RecordSetValidations._ + val dottedHostsConfigZonesAllowed: List[String] = VinylDNSTestHelpers.dottedHostsConfig.zoneAuthConfigs.map(x => x.zone) + "RecordSetValidations" should { "validRecordTypes" should { "return invalid request when adding a PTR record to a forward zone" in { @@ -184,24 +186,75 @@ class RecordSetValidationsSpec } } + "isDotted" should { + "return a failure for any record with dotted hosts if it is already present" in { + val test = aaaa.copy(name = "this.is.a.failure.") + leftValue(isDotted(test, okZone, None, false, true)) shouldBe an[InvalidRequest] + } + + "return a failure for any record that is a dotted host if user or record type is not allowed" in { + val test = aaaa.copy(name = "this.is.a.failure." + okZone.name) + leftValue(isDotted(test, okZone, None, true, false)) shouldBe an[InvalidRequest] + } + + "return success for a dotted record if it does not already have a record or zone with same name and user is allowed" in { + val test = aaaa.copy(name = "this.passes") + isDotted(test, okZone, None, true, true) should be(right) + } + + "return success for a new record that has the same name as the existing record" in { + val newRecord = aaaa.copy(name = "dot.ted") + val existingRecord = newRecord.copy(ttl = 330) + + isDotted(newRecord, okZone, Some(existingRecord), true, true) should be(right) + } + } + "typeSpecificValidations" should { "Run dotted hosts checks" should { val dottedARecord = rsOk.copy(name = "this.is.a.failure.") "return a failure for any new record with dotted hosts in forward zones" in { leftValue( - typeSpecificValidations(dottedARecord, List(), okZone, None, Nil) + typeSpecificValidations(dottedARecord, List(), okZone, None, Nil, true, dottedHostsConfigZonesAllowed.toSet, false) ) shouldBe an[InvalidRequest] } "return a failure for any new record with dotted hosts in forward zones (CNAME)" in { leftValue( - typeSpecificValidations(dottedARecord.copy(typ = CNAME), List(), okZone, None, Nil) + typeSpecificValidations(dottedARecord.copy(typ = CNAME), List(), okZone, None, Nil, true, dottedHostsConfigZonesAllowed.toSet, false) ) shouldBe an[InvalidRequest] } "return a failure for any new record with dotted hosts in forward zones (NS)" in { leftValue( - typeSpecificValidations(dottedARecord.copy(typ = NS), List(), okZone, None, Nil) + typeSpecificValidations(dottedARecord.copy(typ = NS), List(), okZone, None, Nil, true, dottedHostsConfigZonesAllowed.toSet, false) + ) shouldBe an[InvalidRequest] + } + + "return a success for any new record with dotted hosts in forward zones if it satisfies dotted hosts configs" in { + // Zone, User, Record Type and Number of dots are all satisfied + val record = typeSpecificValidations(dottedARecord.copy(typ = CNAME, zoneId = dottedZone.id), List(), dottedZone, None, Nil, true, dottedHostsConfigZonesAllowed.toSet, true, 5) + record should be(right) + } + + "return a failure for any new record with dotted hosts if no.of.dots allowed is 0" in { + // Zone, User, Record Type and Number of dots are all satisfied + leftValue( + typeSpecificValidations(dottedARecord.copy(typ = CNAME, zoneId = dottedZone.id), List(), dottedZone, None, Nil, true, dottedHostsConfigZonesAllowed.toSet, true, 0) + ) shouldBe an[InvalidRequest] + } + + "return a failure for any new record with dotted hosts in forward zones (A record) if it doesn't satisfy dotted hosts configs" in { + // 'A' record is not allowed in the config + leftValue( + typeSpecificValidations(dottedARecord.copy(zoneId = dottedZone.id), List(), dottedZone, None, Nil, true, dottedHostsConfigZonesAllowed.toSet, false, 5) + ) shouldBe an[InvalidRequest] + } + + "return a failure for any new record with dotted hosts in forward zones (NS record) if it doesn't satisfy dotted hosts configs" in { + // 'NS' record is not allowed in the config + leftValue( + typeSpecificValidations(dottedARecord.copy(typ = NS, zoneId = dottedZone.id), List(), dottedZone, None, Nil, true, dottedHostsConfigZonesAllowed.toSet, false, 5) ) shouldBe an[InvalidRequest] } @@ -211,7 +264,10 @@ class RecordSetValidationsSpec List(), okZone, Some(dottedARecord.copy(ttl = 300)), - Nil + Nil, + true, + dottedHostsConfigZonesAllowed.toSet, + false ) should be(right) } @@ -222,7 +278,10 @@ class RecordSetValidationsSpec List(), okZone, Some(dottedCNAMERecord.copy(ttl = 300)), - Nil + Nil, + true, + dottedHostsConfigZonesAllowed.toSet, + false ) should be(right) } @@ -234,7 +293,10 @@ class RecordSetValidationsSpec List(), okZone, Some(dottedNSRecord.copy(ttl = 300)), - Nil + Nil, + true, + dottedHostsConfigZonesAllowed.toSet, + false ) ) shouldBe an[InvalidRequest] } @@ -245,35 +307,35 @@ class RecordSetValidationsSpec val test = srv.copy(name = "_sip._tcp.example.com.") val zone = okZone.copy(name = "example.com.") - typeSpecificValidations(test, List(), zone, None, Nil) should be(right) + typeSpecificValidations(test, List(), zone, None, Nil, true, dottedHostsConfigZonesAllowed.toSet, false) should be(right) } "return success for an SRV record following convention without FQDN" in { val test = srv.copy(name = "_sip._tcp") val zone = okZone.copy(name = "example.com.") - typeSpecificValidations(test, List(), zone, None, Nil) should be(right) + typeSpecificValidations(test, List(), zone, None, Nil, true, dottedHostsConfigZonesAllowed.toSet, false) should be(right) } "return success for an SRV record following convention with a record name" in { val test = srv.copy(name = "_sip._tcp.foo.") val zone = okZone.copy(name = "example.com.") - typeSpecificValidations(test, List(), zone, None, Nil) should be(right) + typeSpecificValidations(test, List(), zone, None, Nil, true, dottedHostsConfigZonesAllowed.toSet, false) should be(right) } "return success on a wildcard SRV that follows convention" in { val test = srv.copy(name = "*._tcp.example.com.") val zone = okZone.copy(name = "example.com.") - typeSpecificValidations(test, List(), zone, None, Nil) should be(right) + typeSpecificValidations(test, List(), zone, None, Nil, true, dottedHostsConfigZonesAllowed.toSet, false) should be(right) } "return success on a wildcard in second position SRV that follows convention" in { val test = srv.copy(name = "_sip._*.example.com.") val zone = okZone.copy(name = "example.com.") - typeSpecificValidations(test, List(), zone, None, Nil) should be(right) + typeSpecificValidations(test, List(), zone, None, Nil, true, dottedHostsConfigZonesAllowed.toSet, false) should be(right) } } "Skip dotted checks on NAPTR" should { @@ -281,21 +343,21 @@ class RecordSetValidationsSpec val test = naptr.copy(name = "sub.naptr.example.com.") val zone = okZone.copy(name = "example.com.") - typeSpecificValidations(test, List(), zone, None, Nil) should be(right) + typeSpecificValidations(test, List(), zone, None, Nil, true, dottedHostsConfigZonesAllowed.toSet, false) should be(right) } "return success for an NAPTR record without FQDN" in { val test = naptr.copy(name = "sub.naptr") val zone = okZone.copy(name = "example.com.") - typeSpecificValidations(test, List(), zone, None, Nil) should be(right) + typeSpecificValidations(test, List(), zone, None, Nil, true, dottedHostsConfigZonesAllowed.toSet, false) should be(right) } "return success on a wildcard NAPTR" in { val test = naptr.copy(name = "*.sub.naptr.example.com.") val zone = okZone.copy(name = "example.com.") - typeSpecificValidations(test, List(), zone, None, Nil) should be(right) + typeSpecificValidations(test, List(), zone, None, Nil, true, dottedHostsConfigZonesAllowed.toSet, false) should be(right) } } @@ -304,7 +366,7 @@ class RecordSetValidationsSpec val test = ptrIp4.copy(name = "10.1.2.") val zone = zoneIp4.copy(name = "198.in-addr.arpa.") - typeSpecificValidations(test, List(), zone, None, Nil) should be(right) + typeSpecificValidations(test, List(), zone, None, Nil, true, dottedHostsConfigZonesAllowed.toSet, false) should be(right) } } "Skip dotted checks on TXT" should { @@ -312,7 +374,7 @@ class RecordSetValidationsSpec val test = txt.copy(name = "sub.txt.example.com.") val zone = okZone.copy(name = "example.com.") - typeSpecificValidations(test, List(), zone, None, Nil) should be(right) + typeSpecificValidations(test, List(), zone, None, Nil, true, dottedHostsConfigZonesAllowed.toSet, false) should be(right) } } @@ -329,7 +391,7 @@ class RecordSetValidationsSpec List(SOAData(Fqdn("something"), "other", 1, 2, 3, 5, 6)) ) - typeSpecificValidations(test, List(), zoneIp4, None, Nil) should be(right) + typeSpecificValidations(test, List(), zoneIp4, None, Nil, true, dottedHostsConfigZonesAllowed.toSet, false) should be(right) } } } @@ -342,29 +404,29 @@ class RecordSetValidationsSpec records = List(NSData(Fqdn("some.test.ns."))) ) - nsValidations(valid, okZone, None, List(new Regex(".*"))) should be(right) + nsValidations(valid, okZone, None, List(new Regex(".*")), true, dottedHostsConfigZonesAllowed.toSet, false) should be(right) } "return an InvalidRequest if an NS record is '@'" in { - val error = leftValue(nsValidations(invalidNsApexRs, okZone, None, Nil)) + val error = leftValue(nsValidations(invalidNsApexRs, okZone, None, Nil, true, dottedHostsConfigZonesAllowed.toSet, false)) error shouldBe an[InvalidRequest] } "return an InvalidRequest if an NS record is the same as the zone" in { val invalid = invalidNsApexRs.copy(name = okZone.name) - val error = leftValue(nsValidations(invalid, okZone, None, Nil)) + val error = leftValue(nsValidations(invalid, okZone, None, Nil, true, dottedHostsConfigZonesAllowed.toSet, false)) error shouldBe an[InvalidRequest] } "return an InvalidRequest if the NS record being updated is '@'" in { val valid = invalidNsApexRs.copy(name = "this-is-not-origin-mate") - val error = leftValue(nsValidations(valid, okZone, Some(invalidNsApexRs), Nil)) + val error = leftValue(nsValidations(valid, okZone, Some(invalidNsApexRs), Nil, true, dottedHostsConfigZonesAllowed.toSet, false)) error shouldBe an[InvalidRequest] } "return an InvalidRequest if an NS record data is not in the approved server list" in { val ns = invalidNsApexRs.copy(records = List(NSData(Fqdn("not.approved.")))) - val error = leftValue(nsValidations(ns, okZone, None, List(new Regex("not.*")))) + val error = leftValue(nsValidations(ns, okZone, None, List(new Regex("not.*")), true, dottedHostsConfigZonesAllowed.toSet, false)) error shouldBe an[InvalidRequest] } } @@ -372,25 +434,35 @@ class RecordSetValidationsSpec "DSValidations" should { val matchingNs = ns.copy(zoneId = ds.zoneId, name = ds.name, ttl = ds.ttl) "return ok if the record is non-origin DS with matching NS" in { - dsValidations(ds, List(matchingNs), okZone) should be(right) + dsValidations(ds, List(matchingNs), okZone, true, dottedHostsConfigZonesAllowed.toSet, false) should be(right) } "return an InvalidRequest if a DS record is '@'" in { val apex = ds.copy(name = "@") - val error = leftValue(dsValidations(apex, List(matchingNs), okZone)) + val error = leftValue(dsValidations(apex, List(matchingNs), okZone, true, dottedHostsConfigZonesAllowed.toSet, false)) error shouldBe an[InvalidRequest] } "return an InvalidRequest if a DS record is the same as the zone" in { val apex = ds.copy(name = okZone.name) - val error = leftValue(dsValidations(apex, List(matchingNs), okZone)) + val error = leftValue(dsValidations(apex, List(matchingNs), okZone, true, dottedHostsConfigZonesAllowed.toSet, false)) error shouldBe an[InvalidRequest] } "return an InvalidRequest if there is no NS matching the record" in { - val error = leftValue(dsValidations(ds, List(), okZone)) + val error = leftValue(dsValidations(ds, List(), okZone, true, dottedHostsConfigZonesAllowed.toSet, false)) error shouldBe an[InvalidRequest] } "return an InvalidRequest if the DS is dotted" in { val error = - leftValue(dsValidations(ds.copy(name = "test.dotted"), List(matchingNs), okZone)) + leftValue(dsValidations(ds.copy(name = "test.dotted"), List(matchingNs), okZone, true, dottedHostsConfigZonesAllowed.toSet, false)) + error shouldBe an[InvalidRequest] + } + "return ok if the DS is dotted and zone, user, record type is allowed in dotted hosts config" in { + val record = + dsValidations(ds.copy(name = "dotted.trial", zoneId = dottedZone.id), List(matchingNs), dottedZone, true, dottedHostsConfigZonesAllowed.toSet, true, 5) + record should be(right) + } + "return an InvalidRequest if the DS is dotted and zone, user, record type is allowed in dotted hosts config but has a conflict with existing record or zone" in { + val error = + leftValue(dsValidations(ds.copy(name = "dotted.trial", zoneId = dottedZone.id), List(matchingNs), dottedZone, false, dottedHostsConfigZonesAllowed.toSet, true)) error shouldBe an[InvalidRequest] } } @@ -398,54 +470,64 @@ class RecordSetValidationsSpec "CnameValidations" should { val invalidCnameApexRs: RecordSet = cname.copy(name = "@") "return a RecordSetAlreadyExistsError if a record with the same name exists and creating a cname" in { - val error = leftValue(cnameValidations(cname, List(aaaa), okZone)) + val error = leftValue(cnameValidations(cname, List(aaaa), okZone, None, true, dottedHostsConfigZonesAllowed.toSet, false)) error shouldBe a[RecordSetAlreadyExists] } "return ok if name is not '@'" in { - cnameValidations(cname, List(), okZone) should be(right) + cnameValidations(cname, List(), okZone, None, true, dottedHostsConfigZonesAllowed.toSet, false) should be(right) } "return an InvalidRequest if a cname record set name is '@'" in { - val error = leftValue(cnameValidations(invalidCnameApexRs, List(), okZone)) + val error = leftValue(cnameValidations(invalidCnameApexRs, List(), okZone, None, true, dottedHostsConfigZonesAllowed.toSet, false)) error shouldBe an[InvalidRequest] } "return an InvalidRequest if a cname record set name is same as zone" in { val invalid = invalidCnameApexRs.copy(name = okZone.name) - val error = leftValue(cnameValidations(invalid, List(), okZone)) + val error = leftValue(cnameValidations(invalid, List(), okZone, None, true, dottedHostsConfigZonesAllowed.toSet, false)) error shouldBe an[InvalidRequest] } "return an InvalidRequest if a cname record set name is dotted" in { - val error = leftValue(cnameValidations(cname.copy(name = "dot.ted"), List(), okZone)) + val error = leftValue(cnameValidations(cname.copy(name = "dot.ted"), List(), okZone, None, true, dottedHostsConfigZonesAllowed.toSet, false)) error shouldBe an[InvalidRequest] } "return ok if new recordset name does not contain dot" in { - cnameValidations(cname, List(), okZone, Some(cname.copy(name = "not-dotted"))) should be( + cnameValidations(cname, List(), okZone, Some(cname.copy(name = "not-dotted")), true, dottedHostsConfigZonesAllowed.toSet, false) should be( right ) } "return ok if dotted host name doesn't change" in { val newRecord = cname.copy(name = "dot.ted", ttl = 500) - cnameValidations(newRecord, List(), okZone, Some(newRecord.copy(ttl = 300))) should be( + cnameValidations(newRecord, List(), okZone, Some(newRecord.copy(ttl = 300)), true, dottedHostsConfigZonesAllowed.toSet, false) should be( right ) } "return an InvalidRequest if a cname record set name is updated to '@'" in { - val error = leftValue(cnameValidations(cname.copy(name = "@"), List(), okZone, Some(cname))) + val error = leftValue(cnameValidations(cname.copy(name = "@"), List(), okZone, Some(cname), true, dottedHostsConfigZonesAllowed.toSet, false)) error shouldBe an[InvalidRequest] } "return an InvalidRequest if updated cname record set name is same as zone" in { val error = - leftValue(cnameValidations(cname.copy(name = okZone.name), List(), okZone, Some(cname))) + leftValue(cnameValidations(cname.copy(name = okZone.name), List(), okZone, Some(cname), true, dottedHostsConfigZonesAllowed.toSet, false)) error shouldBe an[InvalidRequest] } "return an RecordSetValidation error if recordset data contain more than one sequential '.'" in { - val error = leftValue(cnameValidations(cname.copy(records = List(CNAMEData(Fqdn("record..zone")))), List(), okZone)) + val error = leftValue(cnameValidations(cname.copy(records = List(CNAMEData(Fqdn("record..zone")))), List(), okZone, None, true, dottedHostsConfigZonesAllowed.toSet, false)) error shouldBe an[RecordSetValidation] } "return ok if recordset data does not contain sequential '.'" in { - cnameValidations(cname.copy(records = List(CNAMEData(Fqdn("record.zone")))), List(), okZone) should be( + cnameValidations(cname.copy(records = List(CNAMEData(Fqdn("record.zone")))), List(), okZone, None, true, dottedHostsConfigZonesAllowed.toSet, false) should be( right ) } + "return ok if the CNAME is dotted and zone, user, record type is allowed in dotted hosts config" in { + val record = + cnameValidations(cname.copy(name = "dot.ted", zoneId = dottedZone.id), List(), dottedZone, None, true, dottedHostsConfigZonesAllowed.toSet, true, 5) + record should be(right) + } + "return an InvalidRequest if the CNAME is dotted and zone, user, record type is allowed in dotted hosts config but has a conflict with existing record or zone" in { + val error = + leftValue(cnameValidations(cname.copy(name = "dot.ted", zoneId = dottedZone.id), List(), dottedZone, None, false, dottedHostsConfigZonesAllowed.toSet, true)) + error shouldBe an[InvalidRequest] + } } "isNotHighValueDomain" should { diff --git a/modules/api/src/test/scala/vinyldns/api/repository/EmptyRepositories.scala b/modules/api/src/test/scala/vinyldns/api/repository/EmptyRepositories.scala index ad3d79310..70893e3de 100644 --- a/modules/api/src/test/scala/vinyldns/api/repository/EmptyRepositories.scala +++ b/modules/api/src/test/scala/vinyldns/api/repository/EmptyRepositories.scala @@ -119,6 +119,8 @@ trait EmptyGroupRepo extends GroupRepository { def getGroups(groupIds: Set[String]): IO[Set[Group]] = IO.pure(Set()) + def getGroupsByName(groupNames: Set[String]): IO[Set[Group]] = IO.pure(Set()) + def getGroupByName(groupName: String): IO[Option[Group]] = IO.pure(None) def getGroupsByName(groupName: String): IO[Set[Group]] = IO.pure(Set()) diff --git a/modules/core/src/main/scala/vinyldns/core/domain/membership/GroupRepository.scala b/modules/core/src/main/scala/vinyldns/core/domain/membership/GroupRepository.scala index 128792a8c..525323ef4 100644 --- a/modules/core/src/main/scala/vinyldns/core/domain/membership/GroupRepository.scala +++ b/modules/core/src/main/scala/vinyldns/core/domain/membership/GroupRepository.scala @@ -31,6 +31,8 @@ trait GroupRepository extends Repository { def getGroups(groupIds: Set[String]): IO[Set[Group]] + def getGroupsByName(groupNames: Set[String]): IO[Set[Group]] + def getGroupByName(groupName: String): IO[Option[Group]] def getGroupsByName(groupName: String): IO[Set[Group]] diff --git a/modules/core/src/test/scala/vinyldns/core/TestMembershipData.scala b/modules/core/src/test/scala/vinyldns/core/TestMembershipData.scala index 4f9480ed5..9f5c7c489 100644 --- a/modules/core/src/test/scala/vinyldns/core/TestMembershipData.scala +++ b/modules/core/src/test/scala/vinyldns/core/TestMembershipData.scala @@ -36,6 +36,7 @@ object TestMembershipData { val dummyUser = User("dummyName", "dummyAccess", "dummySecret") val superUser = User("super", "superAccess", "superSecret", isSuper = true) + val xyzUser = User("xyz", "xyzAccess", "xyzSecret") val supportUser = User("support", "supportAccess", "supportSecret", isSupport = true) val lockedUser = User("locked", "lockedAccess", "lockedSecret", lockStatus = LockStatus.Locked) val sharedZoneUser = User("sharedZoneAdmin", "sharedAccess", "sharedSecret") diff --git a/modules/core/src/test/scala/vinyldns/core/TestZoneData.scala b/modules/core/src/test/scala/vinyldns/core/TestZoneData.scala index 751d44359..df08e8a0a 100644 --- a/modules/core/src/test/scala/vinyldns/core/TestZoneData.scala +++ b/modules/core/src/test/scala/vinyldns/core/TestZoneData.scala @@ -34,6 +34,8 @@ object TestZoneData { adminGroupId = okGroup.id, connection = testConnection ) + val dottedZone: Zone = Zone("dotted.xyz.", "dotted@xyz.com", adminGroupId = xyzGroup.id) + val dotZone: Zone = Zone("dot.xyz.", "dotted@xyz.com", adminGroupId = xyzGroup.id) val abcZone: Zone = Zone("abc.zone.recordsets.", "test@test.com", adminGroupId = abcGroup.id) val xyzZone: Zone = Zone("xyz.", "abc@xyz.com", adminGroupId = xyzGroup.id) val zoneIp4: Zone = Zone("0.162.198.in-addr.arpa.", "test@test.com", adminGroupId = abcGroup.id) diff --git a/modules/docs/src/main/mdoc/operator/config-api.md b/modules/docs/src/main/mdoc/operator/config-api.md index 2d820849a..fa8b69ba3 100644 --- a/modules/docs/src/main/mdoc/operator/config-api.md +++ b/modules/docs/src/main/mdoc/operator/config-api.md @@ -536,7 +536,66 @@ v6-discovery-nibble-boundaries { min = 5 max = 20 } +``` +### Dotted Hosts + +Configuration setting that determines the zones, users (either individual or based on group) and record types that are +allowed to create dotted hosts. If only all the above are satisfied, one can create a dotted host in VinylDNS. + +Note the following: +1. Zones defined in the `zone` must always end with a dot. Eg: `comcast.com.` +2. Wildcard character `*` can be used in `zone` to allow dotted hosts for all zones matching it. +3. Individual users who are allowed to create dotted hosts are added to the `user-list` using their username. +4. A set of users in a group who are allowed to create dotted hosts are added to the `group-list` using group name. +5. If the user is either in `user-list` or `group-list`, they are allowed to create a dotted host. It is +not necessary for the user to be in both `user-list` and `group-list`. +6. The record types which are allowed while creating a dotted host is added to the `record-types`. +7. The number of dots allowed in a record name for a zone is given in `dots-limit`. +8. If `user-list` is left empty (`user-list = []`), no user will be allowed to create dotted hosts unless +they're present in `group-list` and vice-versa. If both `user-list` and `group-list` is left empty +no users will be allowed to create dotted hosts in that zone. +9. If `record-types` is left empty (`record-types = []`), user cannot create dotted hosts of any record type +in that zone. +10. If `dots-limit` is set to 0 (`dots-limit = 0`), we cannot create dotted hosts record in that zone. + +```yaml +# approved zones, individual users, users in groups, record types and no.of.dots that are allowed for dotted hosts +dotted-hosts = { + allowed-settings = [ + { + zone = "dummy." + user-list = ["testuser"] + group-list = ["dummy-group"] + record-types = ["AAAA"] + dots-limit = 3 + }, + { + # for wildcard zones. Settings will be applied to all matching zones + zone = "*ent.com." + user-list = ["professor", "testuser"] + group-list = ["testing-group"] + record-types = ["A", "CNAME"] + dots-limit = 3 + } + ] +} +``` + +In the above, the dotted hosts can be created only in the zone `dummy.` and zones matching `*ent.com.` (parent.com., child.parent.com.) + +Also, it must satisfy the allowed users or group users and record type of the respective zone to create a dotted host. + +For eg, we can't create a dotted host with `CNAME` record type in the zone `dummy.` as it's not in `record-types`. +And the user `professor` can't create a dotted host in the zone `dummy.` as the user is not in `user-list` or +`group-list` (not part of `dummy-group`). + +The config can be left empty as follows if we don't want to use it: + +```yaml +dotted-hosts = { + allowed-settings = [] +} ``` ### Full Example Config @@ -713,6 +772,27 @@ v6-discovery-nibble-boundaries { } } + # approved zones, individual users, users in groups, record types and no.of.dots that are allowed for dotted hosts + dotted-hosts = { + allowed-settings = [ + { + zone = "dummy." + user-list = ["testuser"] + group-list = ["dummy-group"] + record-types = ["AAAA"] + dots-limit = 3 + }, + { + # for wildcard zones. Settings will be applied to all matching zones + zone = "*ent.com." + user-list = ["professor", "testuser"] + group-list = ["testing-group"] + record-types = ["A", "CNAME"] + dots-limit = 3 + } + ] + } + # true if you want to enable manual review for non-fatal errors manual-batch-review-enabled = true diff --git a/modules/mysql/src/it/scala/vinyldns/mysql/repository/MySqlGroupRepositoryIntegrationSpec.scala b/modules/mysql/src/it/scala/vinyldns/mysql/repository/MySqlGroupRepositoryIntegrationSpec.scala index 76b3469b4..d600318c3 100644 --- a/modules/mysql/src/it/scala/vinyldns/mysql/repository/MySqlGroupRepositoryIntegrationSpec.scala +++ b/modules/mysql/src/it/scala/vinyldns/mysql/repository/MySqlGroupRepositoryIntegrationSpec.scala @@ -93,6 +93,24 @@ class MySqlGroupRepositoryIntegrationSpec } } + "MySqlGroupRepository.getGroupsByName" should { + "omits all non existing groups" in { + val result = repo.getGroupsByName(Set("no-existo", groups.head.name)).unsafeRunSync() + result should contain theSameElementsAs Set(groups.head) + } + + "returns correct list of groups" in { + val names = Set(groups(0).name, groups(1).name, groups(2).name) + val result = repo.getGroupsByName(names).unsafeRunSync() + result should contain theSameElementsAs groups.take(3).toSet + } + + "returns empty list when given no names" in { + val result = repo.getGroupsByName(Set[String]()).unsafeRunSync() + result should contain theSameElementsAs Set() + } + } + "MySqlGroupRepository.getGroupByName" should { "retrieve a group" in { repo.getGroupByName(groups.head.name).unsafeRunSync() shouldBe Some(groups.head) diff --git a/modules/mysql/src/main/scala/vinyldns/mysql/repository/MySqlGroupRepository.scala b/modules/mysql/src/main/scala/vinyldns/mysql/repository/MySqlGroupRepository.scala index ae61e201c..02311e6fd 100644 --- a/modules/mysql/src/main/scala/vinyldns/mysql/repository/MySqlGroupRepository.scala +++ b/modules/mysql/src/main/scala/vinyldns/mysql/repository/MySqlGroupRepository.scala @@ -69,6 +69,13 @@ class MySqlGroupRepository extends GroupRepository with GroupProtobufConversions | WHERE id """.stripMargin + private val BASE_GET_GROUPS_BY_NAMES = + """ + |SELECT data + | FROM groups + | WHERE name + """.stripMargin + def save(db: DB, group: Group): IO[Group] = monitor("repo.Group.save") { IO { @@ -141,6 +148,27 @@ class MySqlGroupRepository extends GroupRepository with GroupProtobufConversions } } + def getGroupsByName(groupNames: Set[String]): IO[Set[Group]] = + monitor("repo.Group.getGroups") { + IO { + logger.debug(s"Getting group with names: $groupNames") + if (groupNames.isEmpty) + Set[Group]() + else { + DB.readOnly { implicit s => + val groupNameList = groupNames.toList + val inClause = " IN (" + groupNameList.as("?").mkString(",") + ")" + val query = BASE_GET_GROUPS_BY_NAMES + inClause + SQL(query) + .bind(groupNameList: _*) + .map(toGroup(1)) + .list() + .apply() + }.toSet + } + } + } + def getGroupByName(groupName: String): IO[Option[Group]] = monitor("repo.Group.getGroupByName") { IO { diff --git a/modules/portal/app/views/zones/zoneTabs/manageRecords.scala.html b/modules/portal/app/views/zones/zoneTabs/manageRecords.scala.html index 850bd8a81..61272b245 100644 --- a/modules/portal/app/views/zones/zoneTabs/manageRecords.scala.html +++ b/modules/portal/app/views/zones/zoneTabs/manageRecords.scala.html @@ -122,7 +122,7 @@
+ title="This is a dotted host!"> {{record.name}}