2
0
mirror of https://github.com/VinylDNS/vinyldns synced 2025-09-01 14:55:22 +00:00

[DeleteRecord] Refactor batch change converter (#834)

* Refactor BatchChangeConverter to use ChangeForValidationMap.
* Update unit tests.
This commit is contained in:
Michael Ly
2019-09-20 14:56:06 -04:00
committed by GitHub
parent 1bc054440c
commit 8dd07df858
5 changed files with 274 additions and 169 deletions

View File

@@ -22,16 +22,13 @@ import cats.syntax.functor._
import org.joda.time.DateTime import org.joda.time.DateTime
import org.slf4j.LoggerFactory import org.slf4j.LoggerFactory
import vinyldns.api.domain.batch.BatchChangeInterfaces._ import vinyldns.api.domain.batch.BatchChangeInterfaces._
import vinyldns.api.domain.batch.BatchTransformations.{ import vinyldns.api.domain.batch.BatchTransformations._
BatchConversionOutput, import vinyldns.api.domain.batch.BatchTransformations.LogicalChangeType._
ExistingRecordSets,
ExistingZones
}
import vinyldns.api.domain.record.RecordSetChangeGenerator import vinyldns.api.domain.record.RecordSetChangeGenerator
import vinyldns.core.domain.record
import vinyldns.core.domain.record._ import vinyldns.core.domain.record._
import vinyldns.core.domain.zone.Zone import vinyldns.core.domain.zone.Zone
import vinyldns.core.domain.batch._ import vinyldns.core.domain.batch._
import vinyldns.core.domain.record.RecordType.RecordType
import vinyldns.core.queue.MessageQueue import vinyldns.core.queue.MessageQueue
class BatchChangeConverter(batchChangeRepo: BatchChangeRepository, messageQueue: MessageQueue) class BatchChangeConverter(batchChangeRepo: BatchChangeRepository, messageQueue: MessageQueue)
@@ -42,7 +39,7 @@ class BatchChangeConverter(batchChangeRepo: BatchChangeRepository, messageQueue:
def sendBatchForProcessing( def sendBatchForProcessing(
batchChange: BatchChange, batchChange: BatchChange,
existingZones: ExistingZones, existingZones: ExistingZones,
existingRecordSets: ExistingRecordSets, groupedChanges: ChangeForValidationMap,
ownerGroupId: Option[String]): BatchResult[BatchConversionOutput] = { ownerGroupId: Option[String]): BatchResult[BatchConversionOutput] = {
logger.info( logger.info(
s"Converting BatchChange [${batchChange.id}] with SingleChanges [${batchChange.changes.map(_.id)}]") s"Converting BatchChange [${batchChange.id}] with SingleChanges [${batchChange.changes.map(_.id)}]")
@@ -50,7 +47,7 @@ class BatchChangeConverter(batchChangeRepo: BatchChangeRepository, messageQueue:
recordSetChanges <- createRecordSetChangesForBatch( recordSetChanges <- createRecordSetChangesForBatch(
batchChange.changes, batchChange.changes,
existingZones, existingZones,
existingRecordSets, groupedChanges,
batchChange.userId, batchChange.userId,
ownerGroupId).toRightBatchResult ownerGroupId).toRightBatchResult
_ <- allChangesWereConverted(batchChange.changes, recordSetChanges) _ <- allChangesWereConverted(batchChange.changes, recordSetChanges)
@@ -123,140 +120,115 @@ class BatchChangeConverter(batchChangeRepo: BatchChangeRepository, messageQueue:
def createRecordSetChangesForBatch( def createRecordSetChangesForBatch(
changes: List[SingleChange], changes: List[SingleChange],
existingZones: ExistingZones, existingZones: ExistingZones,
existingRecordSets: ExistingRecordSets, groupedChanges: ChangeForValidationMap,
userId: String, userId: String,
ownerGroupId: Option[String]): List[RecordSetChange] = { ownerGroupId: Option[String]): List[RecordSetChange] = {
// NOTE: this also assumes we are past approval and know the zone/record split at this point // NOTE: this also assumes we are past approval and know the zone/record split at this point
val changesByKey = for { val supportedChangesByKey = changes
c <- changes .filter(sc => SupportedBatchChangeRecordTypes.get.contains(sc.typ))
rk <- c.recordKey.toList .groupBy(_.recordKey)
} yield (rk, c) .map {
case (recordKey, singleChangeList) => (recordKey, singleChangeList.toNel)
}
val groupedChangeTuples = supportedChangesByKey
changesByKey.groupBy { case (recordKey, _) => recordKey }.values.toList .collect {
groupedChangeTuples.flatMap { gc => case (Some(recordKey), Some(singleChangeNel)) =>
combineChanges( val existingRecordSet = groupedChanges.getExistingRecordSet(recordKey)
gc.map { case (_, singleChange) => singleChange }, val proposedRecordData = groupedChanges.getProposedRecordData(recordKey)
existingZones,
existingRecordSets, for {
userId, zoneName <- singleChangeNel.head.zoneName
ownerGroupId) zone <- existingZones.getByName(zoneName)
} logicalChangeType <- groupedChanges.getLogicalChangeType(recordKey)
recordSetChange <- generateRecordSetChange(
logicalChangeType,
singleChangeNel,
zone,
recordKey.recordType,
proposedRecordData,
userId,
existingRecordSet,
ownerGroupId)
} yield recordSetChange
}
.toList
.flatten
} }
def combineChanges( def generateRecordSetChange(
changes: List[SingleChange], logicalChangeType: LogicalChangeType.LogicalChangeType,
existingZones: ExistingZones, singleChangeNel: NonEmptyList[SingleChange],
existingRecordSets: ExistingRecordSets, zone: Zone,
recordType: RecordType,
proposedRecordData: Set[RecordData],
userId: String, userId: String,
existingRecordSet: Option[RecordSet],
ownerGroupId: Option[String]): Option[RecordSetChange] = { ownerGroupId: Option[String]): Option[RecordSetChange] = {
val adds = NonEmptyList.fromList {
changes.collect {
case add: SingleAddChange if SupportedBatchChangeRecordTypes.get.contains(add.typ) => add
}
}
val deletes = NonEmptyList.fromList {
changes.collect {
case del: SingleDeleteRRSetChange
if SupportedBatchChangeRecordTypes.get.contains(del.typ) =>
del
}
}
// Note: deletes are applied before adds by this logic val singleChangeIds = singleChangeNel.map(_.id).toList
(deletes, adds) match {
case (None, Some(a)) => generateAddChange(a, existingZones, userId, ownerGroupId)
case (Some(d), None) => generateDeleteChange(d, existingZones, existingRecordSets, userId)
case (Some(d), Some(a)) =>
generateUpdateChange(d, a, existingZones, existingRecordSets, userId, ownerGroupId)
case _ => None
}
}
def generateUpdateChange( // Determine owner group for add/update
deleteChanges: NonEmptyList[SingleDeleteRRSetChange], lazy val setOwnerGroupId = existingRecordSet match {
addChanges: NonEmptyList[SingleAddChange], // Update
existingZones: ExistingZones, case Some(existingRs) =>
existingRecordSets: ExistingRecordSets, if (zone.shared && existingRs.ownerGroupId.isEmpty) {
userId: String,
ownerGroupId: Option[String]): Option[RecordSetChange] =
for {
deleteChange <- Some(deleteChanges.head)
zoneName <- deleteChange.zoneName
key <- deleteChange.recordKey
zone <- existingZones.getByName(zoneName)
existingRecordSet <- existingRecordSets.get(key)
newRecordSet <- {
val setOwnerGroupId = if (zone.shared && existingRecordSet.ownerGroupId.isEmpty) {
ownerGroupId ownerGroupId
} else { } else {
existingRecordSet.ownerGroupId existingRs.ownerGroupId
} }
combineAddChanges(addChanges, zone, setOwnerGroupId) // Add
case None =>
if (zone.shared) {
ownerGroupId
} else {
None
}
}
// New record set for add/update or single delete
lazy val newRecordSet = {
val firstAddChange = singleChangeNel.collect {
case sac: SingleAddChange => sac
}.headOption
// For adds, grab the first ttl; for updates via single DeleteRecord, use existing TTL
val newTtlRecordNameTuple = firstAddChange
.map(add => (Some(add.ttl), add.recordName))
.orElse(existingRecordSet.map(rs => (Some(rs.ttl), Some(rs.name))))
newTtlRecordNameTuple.collect {
case (Some(ttl), Some(recordName)) =>
RecordSet(
zone.id,
recordName,
recordType,
ttl,
RecordSetStatus.Pending,
DateTime.now,
None,
proposedRecordData.toList,
ownerGroupId = setOwnerGroupId
)
} }
changeIds = deleteChanges.map(_.id) ++ addChanges.map(_.id).toList }
} yield
RecordSetChangeGenerator.forUpdate(
existingRecordSet,
newRecordSet,
zone,
userId,
changeIds.toList)
def generateDeleteChange( // Generate RecordSetChange based on logical type
deleteChanges: NonEmptyList[SingleDeleteRRSetChange], logicalChangeType match {
existingZones: ExistingZones, case Add =>
existingRecordSets: ExistingRecordSets, newRecordSet.map { newRs =>
userId: String): Option[RecordSetChange] = RecordSetChangeGenerator.forAdd(newRs, zone, userId, singleChangeIds)
for { }
deleteChange <- Some(deleteChanges.head) case FullDelete =>
zoneName <- deleteChange.zoneName existingRecordSet.map { existingRs =>
key <- deleteChange.recordKey RecordSetChangeGenerator.forDelete(existingRs, zone, userId, singleChangeIds)
zone <- existingZones.getByName(zoneName) }
existingRecordSet <- existingRecordSets.get(key) case Update =>
} yield for {
RecordSetChangeGenerator.forDelete( existingRs <- existingRecordSet
existingRecordSet, newRs <- newRecordSet
zone, } yield RecordSetChangeGenerator.forUpdate(existingRs, newRs, zone, userId, singleChangeIds)
userId, case _ => None // This case should never happen
deleteChanges.map(_.id).toList)
def generateAddChange(
addChanges: NonEmptyList[SingleAddChange],
existingZones: ExistingZones,
userId: String,
ownerGroupId: Option[String]): Option[RecordSetChange] =
for {
zoneName <- addChanges.head.zoneName
zone <- existingZones.getByName(zoneName)
newRecordSet <- {
val setOwnerGroupId = if (zone.shared) ownerGroupId else None
combineAddChanges(addChanges, zone, setOwnerGroupId)
}
ids = addChanges.map(_.id)
} yield RecordSetChangeGenerator.forAdd(newRecordSet, zone, userId, ids.toList)
// Combines changes where the RecordData can just be appended to list (A, AAAA, CNAME, PTR)
// NOTE: CNAME & PTR will only have one data field due to validations, so the combination is fine
def combineAddChanges(
changes: NonEmptyList[SingleAddChange],
zone: Zone,
ownerGroupId: Option[String]): Option[RecordSet] = {
val combinedData =
changes.foldLeft(List[RecordData]())((acc, ch) => ch.recordData :: acc).distinct
// recordName and typ are shared by all changes passed into this function, can pull those from any change
// TTL choice is arbitrary here; this is taking the 1st
changes.head.recordName.map { recordName =>
record.RecordSet(
zone.id,
recordName,
changes.head.typ,
changes.head.ttl,
RecordSetStatus.Pending,
DateTime.now,
None,
combinedData,
ownerGroupId = ownerGroupId)
} }
} }
} }

View File

@@ -19,7 +19,7 @@ package vinyldns.api.domain.batch
import vinyldns.api.domain.batch.BatchChangeInterfaces.BatchResult import vinyldns.api.domain.batch.BatchChangeInterfaces.BatchResult
import vinyldns.api.domain.batch.BatchTransformations.{ import vinyldns.api.domain.batch.BatchTransformations.{
BatchConversionOutput, BatchConversionOutput,
ExistingRecordSets, ChangeForValidationMap,
ExistingZones ExistingZones
} }
import vinyldns.core.domain.batch.BatchChange import vinyldns.core.domain.batch.BatchChange
@@ -29,6 +29,6 @@ trait BatchChangeConverterAlgebra {
def sendBatchForProcessing( def sendBatchForProcessing(
batchChange: BatchChange, batchChange: BatchChange,
existingZones: ExistingZones, existingZones: ExistingZones,
existingRecordSets: ExistingRecordSets, groupedChanges: ChangeForValidationMap,
ownerGroupId: Option[String]): BatchResult[BatchConversionOutput] ownerGroupId: Option[String]): BatchResult[BatchConversionOutput]
} }

View File

@@ -104,7 +104,7 @@ class BatchChangeService(
serviceCompleteBatch <- convertOrSave( serviceCompleteBatch <- convertOrSave(
changeForConversion, changeForConversion,
validationOutput.existingZones, validationOutput.existingZones,
validationOutput.groupedChanges.existingRecordSets, validationOutput.groupedChanges,
batchChangeInput.ownerGroupId) batchChangeInput.ownerGroupId)
} yield serviceCompleteBatch } yield serviceCompleteBatch
@@ -166,7 +166,7 @@ class BatchChangeService(
serviceCompleteBatch <- convertOrSave( serviceCompleteBatch <- convertOrSave(
changeForConversion, changeForConversion,
validationOutput.existingZones, validationOutput.existingZones,
validationOutput.groupedChanges.existingRecordSets, validationOutput.groupedChanges,
batchChange.ownerGroupId) batchChange.ownerGroupId)
response <- buildResponseForApprover(serviceCompleteBatch).toBatchResult response <- buildResponseForApprover(serviceCompleteBatch).toBatchResult
} yield response } yield response
@@ -470,17 +470,17 @@ class BatchChangeService(
def convertOrSave( def convertOrSave(
batchChange: BatchChange, batchChange: BatchChange,
existingZones: ExistingZones, existingZones: ExistingZones,
existingRecordSets: ExistingRecordSets, groupedChanges: ChangeForValidationMap,
ownerGroupId: Option[String]): BatchResult[BatchChange] = batchChange.approvalStatus match { ownerGroupId: Option[String]): BatchResult[BatchChange] = batchChange.approvalStatus match {
case AutoApproved => case AutoApproved =>
// send on to the converter, it will be saved there // send on to the converter, it will be saved there
batchChangeConverter batchChangeConverter
.sendBatchForProcessing(batchChange, existingZones, existingRecordSets, ownerGroupId) .sendBatchForProcessing(batchChange, existingZones, groupedChanges, ownerGroupId)
.map(_.batchChange) .map(_.batchChange)
case ManuallyApproved if manualReviewEnabled => case ManuallyApproved if manualReviewEnabled =>
// send on to the converter, it will be saved there // send on to the converter, it will be saved there
batchChangeConverter batchChangeConverter
.sendBatchForProcessing(batchChange, existingZones, existingRecordSets, ownerGroupId) .sendBatchForProcessing(batchChange, existingZones, groupedChanges, ownerGroupId)
.map(_.batchChange) .map(_.batchChange)
case PendingReview if manualReviewEnabled => case PendingReview if manualReviewEnabled =>
// save the change, will need to return to it later on approval // save the change, will need to return to it later on approval

View File

@@ -21,7 +21,8 @@ import cats.implicits._
import org.joda.time.DateTime import org.joda.time.DateTime
import org.scalatest.{Matchers, WordSpec} import org.scalatest.{Matchers, WordSpec}
import vinyldns.api.CatsHelpers import vinyldns.api.CatsHelpers
import vinyldns.api.domain.batch.BatchTransformations.{ExistingRecordSets, ExistingZones} import vinyldns.api.domain.batch.BatchTransformations._
import vinyldns.api.domain.batch.BatchTransformations.LogicalChangeType._
import vinyldns.api.engine.TestMessageQueue import vinyldns.api.engine.TestMessageQueue
import vinyldns.api.repository._ import vinyldns.api.repository._
import vinyldns.core.TestMembershipData.okUser import vinyldns.core.TestMembershipData.okUser
@@ -69,6 +70,23 @@ class BatchChangeConverterSpec extends WordSpec with Matchers with CatsHelpers {
None) None)
} }
private def makeAddChangeForValidation(
recordName: String,
recordData: RecordData,
typ: RecordType = RecordType.A): AddChangeForValidation =
AddChangeForValidation(
okZone,
s"$recordName",
AddChangeInput(s"$recordName.ok.", typ, Some(123), recordData))
private def makeDeleteRRSetChangeForValidation(
recordName: String,
typ: RecordType = RecordType.A): DeleteRRSetChangeForValidation =
DeleteRRSetChangeForValidation(
okZone,
s"$recordName",
DeleteRRSetChangeInput(s"$recordName.ok.", typ))
private val addSingleChangesGood = List( private val addSingleChangesGood = List(
makeSingleAddChange("one", AData("1.1.1.1")), makeSingleAddChange("one", AData("1.1.1.1")),
makeSingleAddChange("two", AData("1.1.1.2")), makeSingleAddChange("two", AData("1.1.1.2")),
@@ -81,6 +99,18 @@ class BatchChangeConverterSpec extends WordSpec with Matchers with CatsHelpers {
makeSingleAddChange("mxRecord", MXData(1, "foo.bar."), MX) makeSingleAddChange("mxRecord", MXData(1, "foo.bar."), MX)
) )
private val addChangeForValidationGood = List(
makeAddChangeForValidation("one", AData("1.1.1.1")),
makeAddChangeForValidation("two", AData("1.1.1.2")),
makeAddChangeForValidation("repeat", AData("1.1.1.3")),
makeAddChangeForValidation("repeat", AData("1.1.1.4")),
makeAddChangeForValidation("aaaaRecord", AAAAData("1::1"), AAAA),
makeAddChangeForValidation("cnameRecord", CNAMEData("cname.com."), CNAME),
makeAddChangeForValidation("10.1.1.1", PTRData("ptrData"), PTR),
makeAddChangeForValidation("txtRecord", TXTData("text"), TXT),
makeAddChangeForValidation("mxRecord", MXData(1, "foo.bar."), MX)
)
private val deleteSingleChangesGood = List( private val deleteSingleChangesGood = List(
makeSingleDeleteRRSetChange("aToDelete", A), makeSingleDeleteRRSetChange("aToDelete", A),
makeSingleDeleteRRSetChange("cnameToDelete", CNAME), makeSingleDeleteRRSetChange("cnameToDelete", CNAME),
@@ -89,6 +119,14 @@ class BatchChangeConverterSpec extends WordSpec with Matchers with CatsHelpers {
makeSingleDeleteRRSetChange("mxToDelete", MX) makeSingleDeleteRRSetChange("mxToDelete", MX)
) )
private val deleteRRSetChangeForValidationGood = List(
makeDeleteRRSetChangeForValidation("aToDelete"),
makeDeleteRRSetChangeForValidation("cnameToDelete", CNAME),
makeDeleteRRSetChangeForValidation("cnameToDelete", CNAME), // duplicate should basically be ignored
makeDeleteRRSetChangeForValidation("txtToDelete", TXT),
makeDeleteRRSetChangeForValidation("mxToDelete", MX)
)
private val updateSingleChangesGood = List( private val updateSingleChangesGood = List(
makeSingleDeleteRRSetChange("aToUpdate", A), makeSingleDeleteRRSetChange("aToUpdate", A),
makeSingleAddChange("aToUpdate", AData("1.1.1.1")), makeSingleAddChange("aToUpdate", AData("1.1.1.1")),
@@ -100,18 +138,41 @@ class BatchChangeConverterSpec extends WordSpec with Matchers with CatsHelpers {
makeSingleAddChange("mxToUpdate", MXData(1, "update.com."), MX) makeSingleAddChange("mxToUpdate", MXData(1, "update.com."), MX)
) )
private val updateChangeForValidationGood = List(
makeDeleteRRSetChangeForValidation("aToUpdate", A),
makeAddChangeForValidation("aToUpdate", AData("1.1.1.1")),
makeDeleteRRSetChangeForValidation("cnameToUpdate", CNAME),
makeAddChangeForValidation("cnameToUpdate", CNAMEData("newcname.com."), CNAME),
makeDeleteRRSetChangeForValidation("txtToUpdate", TXT),
makeAddChangeForValidation("txtToUpdate", TXTData("update"), TXT),
makeDeleteRRSetChangeForValidation("mxToUpdate", MX),
makeAddChangeForValidation("mxToUpdate", MXData(1, "update.com."), MX)
)
private val singleChangesOneBad = List( private val singleChangesOneBad = List(
makeSingleAddChange("one", AData("1.1.1.1")), makeSingleAddChange("one", AData("1.1.1.1")),
makeSingleAddChange("two", AData("1.1.1.2")), makeSingleAddChange("two", AData("1.1.1.2")),
makeSingleAddChange("bad", AData("1.1.1.1")) makeSingleAddChange("bad", AData("1.1.1.1"))
) )
private val changeForValidationOneBad = List(
makeAddChangeForValidation("one", AData("1.1.1.1")),
makeAddChangeForValidation("two", AData("1.1.1.2")),
makeAddChangeForValidation("bad", AData("1.1.1.1"))
)
private val singleChangesOneUnsupported = List( private val singleChangesOneUnsupported = List(
makeSingleAddChange("one", AData("1.1.1.1")), makeSingleAddChange("one", AData("1.1.1.1")),
makeSingleAddChange("two", AData("1.1.1.2")), makeSingleAddChange("two", AData("1.1.1.2")),
makeSingleAddChange("wrongType", TXTData("Unsupported!"), UNKNOWN) makeSingleAddChange("wrongType", TXTData("Unsupported!"), UNKNOWN)
) )
private val changeForValidationOneUnsupported = List(
makeAddChangeForValidation("one", AData("1.1.1.1")),
makeAddChangeForValidation("two", AData("1.1.1.2")),
makeAddChangeForValidation("wrongType", TXTData("Unsupported!"), UNKNOWN)
)
private def existingZones = ExistingZones(Set(okZone, sharedZone)) private def existingZones = ExistingZones(Set(okZone, sharedZone))
private val aToDelete = RecordSet( private val aToDelete = RecordSet(
@@ -201,7 +262,7 @@ class BatchChangeConverterSpec extends WordSpec with Matchers with CatsHelpers {
private val batchChangeRepo = new InMemoryBatchChangeRepository private val batchChangeRepo = new InMemoryBatchChangeRepository
private val underTest = new BatchChangeConverter(batchChangeRepo, TestMessageQueue) private val underTest = new BatchChangeConverter(batchChangeRepo, TestMessageQueue)
"convertAndSendBatchForProcessing" should { "sendBatchForProcessing" should {
"successfully generate add RecordSetChange and map IDs for all adds" in { "successfully generate add RecordSetChange and map IDs for all adds" in {
val batchChange = val batchChange =
BatchChange( BatchChange(
@@ -213,7 +274,11 @@ class BatchChangeConverterSpec extends WordSpec with Matchers with CatsHelpers {
approvalStatus = BatchChangeApprovalStatus.AutoApproved) approvalStatus = BatchChangeApprovalStatus.AutoApproved)
val result = rightResultOf( val result = rightResultOf(
underTest underTest
.sendBatchForProcessing(batchChange, existingZones, existingRecordSets, None) .sendBatchForProcessing(
batchChange,
existingZones,
ChangeForValidationMap(addChangeForValidationGood.map(_.validNel), existingRecordSets),
None)
.value) .value)
val rsChanges = result.recordSetChanges val rsChanges = result.recordSetChanges
@@ -240,7 +305,13 @@ class BatchChangeConverterSpec extends WordSpec with Matchers with CatsHelpers {
approvalStatus = BatchChangeApprovalStatus.AutoApproved) approvalStatus = BatchChangeApprovalStatus.AutoApproved)
val result = rightResultOf( val result = rightResultOf(
underTest underTest
.sendBatchForProcessing(batchChange, existingZones, existingRecordSets, None) .sendBatchForProcessing(
batchChange,
existingZones,
ChangeForValidationMap(
deleteRRSetChangeForValidationGood.map(_.validNel),
existingRecordSets),
None)
.value) .value)
val rsChanges = result.recordSetChanges val rsChanges = result.recordSetChanges
@@ -277,7 +348,13 @@ class BatchChangeConverterSpec extends WordSpec with Matchers with CatsHelpers {
approvalStatus = BatchChangeApprovalStatus.AutoApproved) approvalStatus = BatchChangeApprovalStatus.AutoApproved)
val result = rightResultOf( val result = rightResultOf(
underTest underTest
.sendBatchForProcessing(batchChange, existingZones, existingRecordSets, None) .sendBatchForProcessing(
batchChange,
existingZones,
ChangeForValidationMap(
updateChangeForValidationGood.map(_.validNel),
existingRecordSets),
None)
.value) .value)
val rsChanges = result.recordSetChanges val rsChanges = result.recordSetChanges
@@ -301,6 +378,8 @@ class BatchChangeConverterSpec extends WordSpec with Matchers with CatsHelpers {
"successfully handle a combination of adds, updates, and deletes" in { "successfully handle a combination of adds, updates, and deletes" in {
val changes = addSingleChangesGood ++ deleteSingleChangesGood ++ updateSingleChangesGood val changes = addSingleChangesGood ++ deleteSingleChangesGood ++ updateSingleChangesGood
val changeForValidation = addChangeForValidationGood ++ deleteRRSetChangeForValidationGood ++
updateChangeForValidationGood
val batchChange = val batchChange =
BatchChange( BatchChange(
okUser.id, okUser.id,
@@ -311,7 +390,11 @@ class BatchChangeConverterSpec extends WordSpec with Matchers with CatsHelpers {
approvalStatus = BatchChangeApprovalStatus.AutoApproved) approvalStatus = BatchChangeApprovalStatus.AutoApproved)
val result = rightResultOf( val result = rightResultOf(
underTest underTest
.sendBatchForProcessing(batchChange, existingZones, existingRecordSets, None) .sendBatchForProcessing(
batchChange,
existingZones,
ChangeForValidationMap(changeForValidation.map(_.validNel), existingRecordSets),
None)
.value) .value)
val rsChanges = result.recordSetChanges val rsChanges = result.recordSetChanges
@@ -355,7 +438,11 @@ class BatchChangeConverterSpec extends WordSpec with Matchers with CatsHelpers {
approvalStatus = BatchChangeApprovalStatus.AutoApproved) approvalStatus = BatchChangeApprovalStatus.AutoApproved)
val result = rightResultOf( val result = rightResultOf(
underTest underTest
.sendBatchForProcessing(batchChange, existingZones, existingRecordSets, None) .sendBatchForProcessing(
batchChange,
existingZones,
ChangeForValidationMap(List(), existingRecordSets),
None)
.value) .value)
result.batchChange shouldBe batchChange result.batchChange shouldBe batchChange
@@ -372,7 +459,11 @@ class BatchChangeConverterSpec extends WordSpec with Matchers with CatsHelpers {
approvalStatus = BatchChangeApprovalStatus.AutoApproved) approvalStatus = BatchChangeApprovalStatus.AutoApproved)
val result = rightResultOf( val result = rightResultOf(
underTest underTest
.sendBatchForProcessing(batchWithBadChange, existingZones, existingRecordSets, None) .sendBatchForProcessing(
batchWithBadChange,
existingZones,
ChangeForValidationMap(changeForValidationOneBad.map(_.validNel), existingRecordSets),
None)
.value) .value)
val rsChanges = result.recordSetChanges val rsChanges = result.recordSetChanges
@@ -411,7 +502,13 @@ class BatchChangeConverterSpec extends WordSpec with Matchers with CatsHelpers {
approvalStatus = BatchChangeApprovalStatus.AutoApproved) approvalStatus = BatchChangeApprovalStatus.AutoApproved)
val result = leftResultOf( val result = leftResultOf(
underTest underTest
.sendBatchForProcessing(batchChangeUnsupported, existingZones, existingRecordSets, None) .sendBatchForProcessing(
batchChangeUnsupported,
existingZones,
ChangeForValidationMap(
changeForValidationOneUnsupported.map(_.validNel),
existingRecordSets),
None)
.value) .value)
result shouldBe an[BatchConversionError] result shouldBe an[BatchConversionError]
@@ -427,10 +524,14 @@ class BatchChangeConverterSpec extends WordSpec with Matchers with CatsHelpers {
"generate record set changes for shared zone without owner group ID if not provided" in { "generate record set changes for shared zone without owner group ID if not provided" in {
val result = val result =
underTest.generateAddChange( underTest.generateRecordSetChange(
Add,
NonEmptyList.of(singleAddChange), NonEmptyList.of(singleAddChange),
existingZones, sharedZone,
singleAddChange.typ,
Set(singleAddChange.recordData),
okUser.id, okUser.id,
None,
None) None)
result shouldBe defined result shouldBe defined
result.foreach(_.recordSet.ownerGroupId shouldBe None) result.foreach(_.recordSet.ownerGroupId shouldBe None)
@@ -438,10 +539,14 @@ class BatchChangeConverterSpec extends WordSpec with Matchers with CatsHelpers {
"generate record set changes for shared zone with owner group ID if provided" in { "generate record set changes for shared zone with owner group ID if provided" in {
val result = val result =
underTest.generateAddChange( underTest.generateRecordSetChange(
Add,
NonEmptyList.of(singleAddChange), NonEmptyList.of(singleAddChange),
existingZones, sharedZone,
singleAddChange.typ,
Set(singleAddChange.recordData),
okUser.id, okUser.id,
None,
ownerGroupId) ownerGroupId)
result shouldBe defined result shouldBe defined
result.foreach(_.recordSet.ownerGroupId shouldBe ownerGroupId) result.foreach(_.recordSet.ownerGroupId shouldBe ownerGroupId)
@@ -449,10 +554,14 @@ class BatchChangeConverterSpec extends WordSpec with Matchers with CatsHelpers {
"generate record set changes for non-shared zone without owner group ID" in { "generate record set changes for non-shared zone without owner group ID" in {
val result = val result =
underTest.generateAddChange( underTest.generateRecordSetChange(
Add,
NonEmptyList.fromListUnsafe(addSingleChangesGood), NonEmptyList.fromListUnsafe(addSingleChangesGood),
existingZones, okZone,
singleAddChange.typ,
Set(singleAddChange.recordData),
okUser.id, okUser.id,
None,
ownerGroupId) ownerGroupId)
result shouldBe defined result shouldBe defined
result.foreach(_.recordSet.ownerGroupId shouldBe None) result.foreach(_.recordSet.ownerGroupId shouldBe None)
@@ -465,12 +574,14 @@ class BatchChangeConverterSpec extends WordSpec with Matchers with CatsHelpers {
"not overwrite existing owner group ID for existing record set in shared zone" in { "not overwrite existing owner group ID for existing record set in shared zone" in {
val result = val result =
underTest.generateUpdateChange( underTest.generateRecordSetChange(
NonEmptyList.of(deleteChange), Update,
NonEmptyList.of(addChange), NonEmptyList.of(deleteChange, addChange),
existingZones, sharedZone,
ExistingRecordSets(List(sharedZoneRecord)), deleteChange.typ,
Set(addChange.recordData),
okUser.id, okUser.id,
Some(sharedZoneRecord),
Some("new-owner-group-id") Some("new-owner-group-id")
) )
result shouldBe defined result shouldBe defined
@@ -480,12 +591,14 @@ class BatchChangeConverterSpec extends WordSpec with Matchers with CatsHelpers {
"use specified owner group ID if undefined for existing record set in shared zone" in { "use specified owner group ID if undefined for existing record set in shared zone" in {
val ownerGroupId = Some("new-owner-group-id") val ownerGroupId = Some("new-owner-group-id")
val result = val result =
underTest.generateUpdateChange( underTest.generateRecordSetChange(
NonEmptyList.of(deleteChange), Update,
NonEmptyList.of(addChange), NonEmptyList.of(deleteChange, addChange),
existingZones, sharedZone,
ExistingRecordSets(List(sharedZoneRecord.copy(ownerGroupId = None))), addChange.typ,
Set(addChange.recordData),
okUser.id, okUser.id,
Some(sharedZoneRecord.copy(ownerGroupId = None)),
ownerGroupId ownerGroupId
) )
result shouldBe defined result shouldBe defined
@@ -494,13 +607,17 @@ class BatchChangeConverterSpec extends WordSpec with Matchers with CatsHelpers {
"generate record set without updating owner group ID for record set in unshared zone" in { "generate record set without updating owner group ID for record set in unshared zone" in {
val result = val result =
underTest.generateUpdateChange( underTest.generateRecordSetChange(
Update,
NonEmptyList.of( NonEmptyList.of(
deleteChange.copy(zoneId = Some(okZone.id), zoneName = Some(okZone.name))), deleteChange.copy(zoneId = Some(okZone.id), zoneName = Some(okZone.name)),
NonEmptyList.of(addChange.copy(zoneId = Some(okZone.id), zoneName = Some(okZone.name))), addChange.copy(zoneId = Some(okZone.id), zoneName = Some(okZone.name))
existingZones, ),
ExistingRecordSets(List(sharedZoneRecord.copy(ownerGroupId = None, zoneId = okZone.id))), okZone,
addChange.typ,
Set(addChange.recordData),
okUser.id, okUser.id,
Some(sharedZoneRecord.copy(ownerGroupId = None, zoneId = okZone.id)),
Some("new-owner-group-id") Some("new-owner-group-id")
) )
result shouldBe defined result shouldBe defined

View File

@@ -183,7 +183,7 @@ class BatchChangeServiceSpec
def sendBatchForProcessing( def sendBatchForProcessing(
batchChange: BatchChange, batchChange: BatchChange,
existingZones: ExistingZones, existingZones: ExistingZones,
existingRecordSets: ExistingRecordSets, groupedChanges: ChangeForValidationMap,
ownerGroupId: Option[String]): BatchResult[BatchConversionOutput] = ownerGroupId: Option[String]): BatchResult[BatchConversionOutput] =
batchChange.comments match { batchChange.comments match {
case Some("conversionError") => BatchConversionError(pendingChange).toLeftBatchResult case Some("conversionError") => BatchConversionError(pendingChange).toLeftBatchResult
@@ -2201,7 +2201,11 @@ class BatchChangeServiceSpec
val result = rightResultOf( val result = rightResultOf(
underTestManualEnabled underTestManualEnabled
.convertOrSave(batchChange, ExistingZones(Set()), ExistingRecordSets(List()), None) .convertOrSave(
batchChange,
ExistingZones(Set()),
ChangeForValidationMap(List(), ExistingRecordSets(List())),
None)
.value) .value)
result.reviewComment shouldBe Some("batchSentToConverter") result.reviewComment shouldBe Some("batchSentToConverter")
} }
@@ -2217,7 +2221,11 @@ class BatchChangeServiceSpec
val result = rightResultOf( val result = rightResultOf(
underTestManualEnabled underTestManualEnabled
.convertOrSave(batchChange, ExistingZones(Set()), ExistingRecordSets(List()), None) .convertOrSave(
batchChange,
ExistingZones(Set()),
ChangeForValidationMap(List(), ExistingRecordSets(List())),
None)
.value) .value)
// not sent to converter // not sent to converter
@@ -2237,7 +2245,11 @@ class BatchChangeServiceSpec
val result = leftResultOf( val result = leftResultOf(
underTest underTest
.convertOrSave(batchChange, ExistingZones(Set()), ExistingRecordSets(List()), None) .convertOrSave(
batchChange,
ExistingZones(Set()),
ChangeForValidationMap(List(), ExistingRecordSets(List())),
None)
.value) .value)
result shouldBe an[UnknownConversionError] result shouldBe an[UnknownConversionError]
@@ -2254,7 +2266,11 @@ class BatchChangeServiceSpec
val result = leftResultOf( val result = leftResultOf(
underTest underTest
.convertOrSave(batchChange, ExistingZones(Set()), ExistingRecordSets(List()), None) .convertOrSave(
batchChange,
ExistingZones(Set()),
ChangeForValidationMap(List(), ExistingRecordSets(List())),
None)
.value) .value)
result shouldBe an[UnknownConversionError] result shouldBe an[UnknownConversionError]
} }