diff --git a/modules/common/src/main/scala/com.snowplowanalytics.snowplow.enrich/common/enrichments/EnrichmentManager.scala b/modules/common/src/main/scala/com.snowplowanalytics.snowplow.enrich/common/enrichments/EnrichmentManager.scala index f14fdd137..8143d15f9 100644 --- a/modules/common/src/main/scala/com.snowplowanalytics.snowplow.enrich/common/enrichments/EnrichmentManager.scala +++ b/modules/common/src/main/scala/com.snowplowanalytics.snowplow.enrich/common/enrichments/EnrichmentManager.scala @@ -46,7 +46,8 @@ import com.snowplowanalytics.snowplow.enrich.common.utils.{IgluUtils, Conversion object EnrichmentManager { - private type EnrichmentResult[F[_]] = IorT[F, NonEmptyList[FailureEntity.BadRowWithFailureEntities], (EnrichedEvent, List[SelfDescribingData[Json]])] + private type EnrichmentResult[F[_]] = + IorT[F, NonEmptyList[FailureEntity.BadRowWithFailureEntities], (EnrichedEvent, List[SelfDescribingData[Json]])] /** * Run the enrichment workflow @@ -103,18 +104,18 @@ object EnrichmentManager { .leftMap(NonEmptyList.one) .possiblyExitingEarly(emitIncomplete) validContexts <- validateEnriched( - enriched, - raw, - enrichmentsContexts, - client, - processor, - registryLookup, - featureFlags.acceptInvalid, - invalidCount, - atomicFields - ) - .leftMap(NonEmptyList.one) - .possiblyExitingEarly(emitIncomplete) + enriched, + raw, + enrichmentsContexts, + client, + processor, + registryLookup, + featureFlags.acceptInvalid, + invalidCount, + atomicFields + ) + .leftMap(NonEmptyList.one) + .possiblyExitingEarly(emitIncomplete) } yield (enriched, validContexts ::: extractResult.validationInfoContexts) // derived contexts are set lastly because we want to include failure entities @@ -125,18 +126,20 @@ object EnrichmentManager { private def setDerivedContexts[F[_]: Sync](enriched: EnrichmentResult[F]): EnrichmentResult[F] = IorT( - enriched.value.flatTap(v => Sync[F].delay { - val (derivedContexts, enriched) = v match { - case Ior.Right((e, l)) => (l, e.some) - case Ior.Left(l) => (extractFailureEntities(l), None) - case Ior.Both(b, (e, l)) => (l ::: extractFailureEntities(b), e.some) + enriched.value.flatTap(v => + Sync[F].delay { + val (derivedContexts, enriched) = v match { + case Ior.Right((e, l)) => (l, e.some) + case Ior.Left(l) => (extractFailureEntities(l), None) + case Ior.Both(b, (e, l)) => (l ::: extractFailureEntities(b), e.some) + } + for { + c <- ME.formatContexts(derivedContexts) + e <- enriched + _ = e.derived_contexts = c + } yield () } - for { - c <- ME.formatContexts(derivedContexts) - e <- enriched - _ = e.derived_contexts = c - } yield () - }) + ) ) private def extractFailureEntities(l: NonEmptyList[FailureEntity.BadRowWithFailureEntities]): List[SelfDescribingData[Json]] = @@ -157,12 +160,12 @@ object EnrichmentManager { } yield extract iorT.leftMap { violations => - buildSchemaViolationsBadRow( - violations, - EnrichedEvent.toPartiallyEnrichedEvent(enrichedEvent), - RawEvent.toRawEvent(raw), - processor - ) + buildSchemaViolationsBadRow( + violations, + EnrichedEvent.toPartiallyEnrichedEvent(enrichedEvent), + RawEvent.toRawEvent(raw), + processor + ) } } @@ -875,14 +878,14 @@ object EnrichmentManager { processor: Processor ): FailureEntity.BadRowWithFailureEntities = { val now = Instant.now() - val failureEntities = vs.map(v => FailureEntity.fromSchemaViolation(v, now, processor)) + val failureEntities = vs.toList.map(v => FailureEntity.fromSchemaViolation(v, now, processor)) FailureEntity.BadRowWithFailureEntities( badRow = BadRow.SchemaViolations( processor, Failure.SchemaViolations(now, vs.map(_.schemaViolation)), Payload.EnrichmentPayload(pee, re) ), - failureEntities = failureEntities.toList + failureEntities = failureEntities ) } @@ -893,8 +896,7 @@ object EnrichmentManager { processor: Processor ): FailureEntity.BadRowWithFailureEntities = { val now = Instant.now() - // TODO: Fill it correctly - val failureEntities = Nil + val failureEntities = fs.toList.flatMap(v => FailureEntity.fromEnrichmentFailure(v, now, processor)) FailureEntity.BadRowWithFailureEntities( badRow = BadRow.EnrichmentFailures( processor, diff --git a/modules/common/src/main/scala/com.snowplowanalytics.snowplow.enrich/common/enrichments/FailureEntity.scala b/modules/common/src/main/scala/com.snowplowanalytics.snowplow.enrich/common/enrichments/FailureEntity.scala index a96a277fd..fab511a61 100644 --- a/modules/common/src/main/scala/com.snowplowanalytics.snowplow.enrich/common/enrichments/FailureEntity.scala +++ b/modules/common/src/main/scala/com.snowplowanalytics.snowplow.enrich/common/enrichments/FailureEntity.scala @@ -12,13 +12,19 @@ package com.snowplowanalytics.snowplow.enrich.common.enrichments import java.time.Instant +import cats.syntax.option._ + import io.circe.{Encoder, Json} import io.circe.generic.semiauto._ import io.circe.syntax._ import com.snowplowanalytics.snowplow.badrows._ +import com.snowplowanalytics.iglu.client.ClientError +import com.snowplowanalytics.iglu.client.validator.ValidatorError + import com.snowplowanalytics.iglu.core.{SchemaKey, SchemaVer, SelfDescribingData} +import com.snowplowanalytics.iglu.core.circe.implicits.schemaKeyCirceJsonEncoder /** * Represents a failure encountered during enrichment of the event. @@ -27,7 +33,7 @@ import com.snowplowanalytics.iglu.core.{SchemaKey, SchemaVer, SelfDescribingData case class FailureEntity( failureType: String, errors: List[Json], - schema: Option[String], + schema: Option[SchemaKey], data: Option[String], timestamp: Instant, componentName: String, @@ -36,7 +42,9 @@ case class FailureEntity( object FailureEntity { - val schemaKey = SchemaKey("com.snowplowanalytics.snowplow", "failure", "jsonschema", SchemaVer.Full(1, 0, 0)) + val failureEntitySchemaKey = SchemaKey("com.snowplowanalytics.snowplow", "failure", "jsonschema", SchemaVer.Full(1, 0, 0)) + + implicit val failureEntityEncoder: Encoder[FailureEntity] = deriveEncoder[FailureEntity] case class BadRowWithFailureEntities( badRow: BadRow, @@ -56,12 +64,52 @@ object FailureEntity { def toSDJ(failure: FailureEntity): SelfDescribingData[Json] = SelfDescribingData( - schemaKey, + failureEntitySchemaKey, failure.asJson ) - implicit val instantEncoder: Encoder[Instant] = Encoder.encodeString.contramap[Instant](_.toString) - implicit val encoder: Encoder[FailureEntity] = deriveEncoder[FailureEntity] + def fromEnrichmentFailure( + ef: FailureDetails.EnrichmentFailure, + timestamp: Instant, + processor: Processor + ): Option[FailureEntity] = { + val failureType = s"EnrichmentError: ${ef.enrichment.map(_.identifier).getOrElse("")}" + ef.message match { + case m: FailureDetails.EnrichmentFailureMessage.InputData => + FailureEntity( + failureType = failureType, + errors = List( + Json.obj( + "message" := s"${m.field} - ${m.expectation}", + "source" := m.field + ) + ), + schema = ef.enrichment.map(_.schemaKey), + data = s"""{"${m.field}" : "${m.value.getOrElse("")}"}""".some, + timestamp = timestamp, + componentName = processor.artifact, + componentVersion = processor.version + ).some + case m: FailureDetails.EnrichmentFailureMessage.Simple => + FailureEntity( + failureType = failureType, + errors = List( + Json.obj( + "message" := m.error + ) + ), + schema = ef.enrichment.map(_.schemaKey), + data = None, + timestamp = timestamp, + componentName = processor.artifact, + componentVersion = processor.version + ).some + case _: FailureDetails.EnrichmentFailureMessage.IgluError => + // EnrichmentFailureMessage.IgluError isn't used anywhere in the project + // therefore we don't expect it in here + None + } + } def fromSchemaViolation( v: SchemaViolationWithExtraContext, @@ -71,8 +119,8 @@ object FailureEntity { v.schemaViolation match { case f: FailureDetails.SchemaViolation.NotJson => FailureEntity( - failureType = "NotJson", - errors = List(Json.obj("message" -> f.error.asJson, "source" -> v.source.asJson)), + failureType = "NotJSON", + errors = List(Json.obj("message" := f.error.asJson, "source" := v.source.asJson)), schema = None, data = v.data, timestamp = timestamp, @@ -83,14 +131,87 @@ object FailureEntity { val message = f.error.message("").split(":").headOption FailureEntity( failureType = "NotIglu", - errors = List(Json.obj("message" -> message.asJson, "source" -> v.source.asJson)), + errors = List(Json.obj("message" := message.asJson, "source" := v.source.asJson)), schema = None, data = v.data, timestamp = timestamp, componentName = processor.artifact, componentVersion = processor.version ) - // TODO: Implement remaining cases - case _ => throw new Exception("") + case f: FailureDetails.SchemaViolation.CriterionMismatch => + val message = s"Unexpected schema: ${f.schemaKey.toSchemaUri} does not match the criterion" + FailureEntity( + failureType = "CriterionMismatch", + errors = List( + Json.obj( + "message" := message, + "source" := v.source, + "criterion" := f.schemaCriterion.asString + ) + ), + schema = f.schemaKey.some, + data = v.data, + timestamp = timestamp, + componentName = processor.artifact, + componentVersion = processor.version + ) + case FailureDetails.SchemaViolation.IgluError(schemaKey, ClientError.ResolutionError(lh)) => + val message = s"Resolution error: schema ${schemaKey.toSchemaUri} not found" + val lookupHistory = lh.toList + .map { + case (repo, lookups) => + lookups.asJson.deepMerge(Json.obj("repository" := repo.asJson)) + } + FailureEntity( + failureType = "ResolutionError", + errors = List( + Json.obj( + "message" := message, + "source" := v.source, + "lookupHistory" := lookupHistory + ) + ), + schema = schemaKey.some, + data = v.data, + timestamp = timestamp, + componentName = processor.artifact, + componentVersion = processor.version + ) + case FailureDetails.SchemaViolation.IgluError(schemaKey, ClientError.ValidationError(ValidatorError.InvalidData(e), _)) => + val errors = e.toList.map { r => + Json.obj( + "message" := r.message, + "source" := v.source, + "path" := r.path, + "keyword" := r.keyword, + "targets" := r.targets + ) + } + FailureEntity( + failureType = "ValidationError", + errors = errors, + schema = schemaKey.some, + data = v.data, + timestamp = timestamp, + componentName = processor.artifact, + componentVersion = processor.version + ) + case FailureDetails.SchemaViolation.IgluError(schemaKey, ClientError.ValidationError(ValidatorError.InvalidSchema(e), _)) => + val errors = e.toList.map { r => + Json.obj( + "message" := s"Invalid schema: $schemaKey - ${r.message}", + "source" := v.source, + "path" := r.path + ) + } + FailureEntity( + failureType = "ValidationError", + errors = errors, + schema = schemaKey.some, + data = v.data, + timestamp = timestamp, + componentName = processor.artifact, + componentVersion = processor.version + ) } } diff --git a/modules/common/src/main/scala/com.snowplowanalytics.snowplow.enrich/common/utils/IgluUtils.scala b/modules/common/src/main/scala/com.snowplowanalytics.snowplow.enrich/common/utils/IgluUtils.scala index 7c425066b..96586e2ba 100644 --- a/modules/common/src/main/scala/com.snowplowanalytics.snowplow.enrich/common/utils/IgluUtils.scala +++ b/modules/common/src/main/scala/com.snowplowanalytics.snowplow.enrich/common/utils/IgluUtils.scala @@ -177,7 +177,7 @@ object IgluUtils { SchemaViolationWithExtraContext( schemaViolation = FailureDetails.SchemaViolation.NotJson(field, rawJson.some, e), source = field, - data = s"{$field : $rawJson".some + data = s"""{"$field" : "$rawJson"}""".some ) ) .toEitherT[F] diff --git a/modules/common/src/test/scala/com.snowplowanalytics.snowplow.enrich.common/utils/IgluUtilsSpec.scala b/modules/common/src/test/scala/com.snowplowanalytics.snowplow.enrich.common/utils/IgluUtilsSpec.scala index 607b3a5c3..5a9335823 100644 --- a/modules/common/src/test/scala/com.snowplowanalytics.snowplow.enrich.common/utils/IgluUtilsSpec.scala +++ b/modules/common/src/test/scala/com.snowplowanalytics.snowplow.enrich.common/utils/IgluUtilsSpec.scala @@ -16,6 +16,7 @@ import org.specs2.matcher.ValidatedMatchers import cats.effect.testing.specs2.CatsEffect import io.circe.parser.parse +import io.circe.syntax._ import cats.data.{Ior, NonEmptyList} @@ -24,7 +25,9 @@ import com.snowplowanalytics.iglu.core.{SchemaKey, SchemaVer} import com.snowplowanalytics.iglu.client.ClientError.{ResolutionError, ValidationError} import com.snowplowanalytics.snowplow.badrows._ +import com.snowplowanalytics.snowplow.badrows.FailureDetails.SchemaViolation +import com.snowplowanalytics.snowplow.enrich.common.enrichments.FailureEntity.SchemaViolationWithExtraContext import com.snowplowanalytics.snowplow.enrich.common.outputs.EnrichedEvent import com.snowplowanalytics.snowplow.enrich.common.SpecHelpers import com.snowplowanalytics.snowplow.enrich.common.adapters.RawEvent @@ -42,7 +45,11 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect val processor = Processor("unit tests SCE", "v42") val enriched = new EnrichedEvent() + val ueProperties = "ue_properties" + val contexts = "contexts" + val notJson = "foo" + val jsonNotJson = notJson.asJson // Just jsonized version of the string val notIglu = """{"foo":"bar"}""" val unstructSchema = SchemaKey( @@ -150,12 +157,13 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect .extractAndValidateUnstructEvent(input, SpecHelpers.client, SpecHelpers.registryLookup) .value .map { - case Ior.Both(NonEmptyList(_: FailureDetails.SchemaViolation.NotJson, _), None) => ok + case Ior.Both(NonEmptyList(SchemaViolationWithExtraContext(_: SchemaViolation.NotJson, `ueProperties`, `jsonNotJson`), _), None) => ok case other => ko(s"[$other] is not an error with NotJson") } } "return a SchemaViolation.NotIglu if unstruct_event contains a properly formatted JSON string that is not self-describing" >> { + val json = parse(notIglu).toOption.get val input = new EnrichedEvent input.setUnstruct_event(notIglu) @@ -163,12 +171,13 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect .extractAndValidateUnstructEvent(input, SpecHelpers.client, SpecHelpers.registryLookup) .value .map { - case Ior.Both(NonEmptyList(_: FailureDetails.SchemaViolation.NotIglu, _), None) => ok + case Ior.Both(NonEmptyList(SchemaViolationWithExtraContext(_: SchemaViolation.NotIglu, `ueProperties`, `json`), _), None) => ok case other => ko(s"[$other] is not an error with NotIglu") } } "return a SchemaViolation.CriterionMismatch if unstruct_event contains a self-describing JSON but not with the expected schema for unstructured events" >> { + val json = parse(noSchema).toOption.get val input = new EnrichedEvent input.setUnstruct_event(noSchema) @@ -176,51 +185,54 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect .extractAndValidateUnstructEvent(input, SpecHelpers.client, SpecHelpers.registryLookup) .value .map { - case Ior.Both(NonEmptyList(_: FailureDetails.SchemaViolation.CriterionMismatch, _), None) => ok + case Ior.Both(NonEmptyList(SchemaViolationWithExtraContext(_: SchemaViolation.CriterionMismatch, `ueProperties`, `json`), _), None) => ok case other => ko(s"[$other] is not an error with CriterionMismatch") } } "return a SchemaViolation.NotJson if the JSON in .data is not a JSON" >> { val input = new EnrichedEvent - input.setUnstruct_event(buildUnstruct(notJson)) + val ue = buildUnstruct(notJson) + val ueJson = ue.asJson + input.setUnstruct_event(ue) IgluUtils .extractAndValidateUnstructEvent(input, SpecHelpers.client, SpecHelpers.registryLookup) .value .map { - case Ior.Both(NonEmptyList(_: FailureDetails.SchemaViolation.NotJson, _), None) => ok + case Ior.Both(NonEmptyList(SchemaViolationWithExtraContext(_: SchemaViolation.NotJson, `ueProperties`, `ueJson`), _), None) => ok case other => ko(s"[$other] is not an error with NotJson") } } "return a SchemaViolation.IgluError containing a ValidationError if the JSON in .data is not self-describing" >> { val input = new EnrichedEvent - input.setUnstruct_event(buildUnstruct(notIglu)) + val ue = buildUnstruct(notIglu) + val ueJson = parse(ue).toOption.get + input.setUnstruct_event(ue) IgluUtils .extractAndValidateUnstructEvent(input, SpecHelpers.client, SpecHelpers.registryLookup) .value .map { - case Ior.Both(NonEmptyList(FailureDetails.SchemaViolation.IgluError(_, ValidationError(_, _)), _), None) => ok - case Ior.Both(NonEmptyList(ie: FailureDetails.SchemaViolation.IgluError, _), None) => - ko(s"IgluError [$ie] is not ValidationError") - case other => ko(s"[$other] is not an error with IgluError") + case Ior.Both(NonEmptyList(SchemaViolationWithExtraContext(SchemaViolation.IgluError(_, _: ValidationError), `ueProperties`, `ueJson`), _), None) => + ok + case other => ko(s"[$other] is not expected one") } } "return a SchemaViolation.IgluError containing a ValidationError if the JSON in .data is not a valid SDJ" >> { val input = new EnrichedEvent + val json = parse(invalidEmailSent).toOption.get input.setUnstruct_event(buildUnstruct(invalidEmailSent)) IgluUtils .extractAndValidateUnstructEvent(input, SpecHelpers.client, SpecHelpers.registryLookup) .value .map { - case Ior.Both(NonEmptyList(FailureDetails.SchemaViolation.IgluError(_, ValidationError(_, _)), _), None) => ok - case Ior.Both(NonEmptyList(ie: FailureDetails.SchemaViolation.IgluError, _), None) => - ko(s"IgluError [$ie] is not ValidationError") - case other => ko(s"[$other] is not an error with IgluError") + case Ior.Both(NonEmptyList(SchemaViolationWithExtraContext(SchemaViolation.IgluError(_, _: ValidationError), `ueProperties`, `json`), _), None) => + ok + case other => ko(s"[$other] is not expected one") } } @@ -232,8 +244,9 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect .extractAndValidateUnstructEvent(input, SpecHelpers.client, SpecHelpers.registryLookup) .value .map { - case Ior.Both(NonEmptyList(FailureDetails.SchemaViolation.IgluError(_, ResolutionError(_)), _), None) => ok - case Ior.Both(NonEmptyList(ie: FailureDetails.SchemaViolation.IgluError, _), None) => + case Ior.Both(NonEmptyList(SchemaViolationWithExtraContext(SchemaViolation.IgluError(_, _: ResolutionError), _, _), _), None) => + ok + case Ior.Both(NonEmptyList(SchemaViolationWithExtraContext(ie: SchemaViolation.IgluError, _, _), _), None) => ko(s"IgluError [$ie] is not a ResolutionError") case other => ko(s"[$other] is not an error with IgluError") } @@ -315,7 +328,7 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect .extractAndValidateInputContexts(input, SpecHelpers.client, SpecHelpers.registryLookup) .value .map { - case Ior.Both(NonEmptyList(_: FailureDetails.SchemaViolation.NotJson, Nil), Nil) => ok + case Ior.Both(NonEmptyList(SchemaViolationWithExtraContext(_: SchemaViolation.NotJson, `contexts`, `jsonNotJson`), Nil), Nil) => ok case other => ko(s"[$other] is not an error with NotJson") } } @@ -328,7 +341,7 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect .extractAndValidateInputContexts(input, SpecHelpers.client, SpecHelpers.registryLookup) .value .map { - case Ior.Both(NonEmptyList(_: FailureDetails.SchemaViolation.NotIglu, Nil), Nil) => ok + case Ior.Both(NonEmptyList(SchemaViolationWithExtraContext(_: SchemaViolation.NotIglu, _, _), Nil), Nil) => ok case other => ko(s"[$other] is not an error with NotIglu") } } @@ -341,7 +354,7 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect .extractAndValidateInputContexts(input, SpecHelpers.client, SpecHelpers.registryLookup) .value .map { - case Ior.Both(NonEmptyList(_: FailureDetails.SchemaViolation.CriterionMismatch, Nil), Nil) => ok + case Ior.Both(NonEmptyList(SchemaViolationWithExtraContext(_: SchemaViolation.CriterionMismatch, _, _), Nil), Nil) => ok case other => ko(s"[$other] is not an error with CriterionMismatch") } } @@ -356,9 +369,9 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect .extractAndValidateInputContexts(input, SpecHelpers.client, SpecHelpers.registryLookup) .value .map { - case Ior.Both(NonEmptyList(FailureDetails.SchemaViolation.IgluError(_, ValidationError(_, _)), Nil), Nil) => + case Ior.Both(NonEmptyList(SchemaViolationWithExtraContext(SchemaViolation.IgluError(_, _: ValidationError), _, _), Nil), Nil) => ok - case Ior.Both(NonEmptyList(ie: FailureDetails.SchemaViolation.IgluError, Nil), Nil) => + case Ior.Both(NonEmptyList(SchemaViolationWithExtraContext(ie: SchemaViolation.IgluError, _, _), Nil), Nil) => ko(s"IgluError [$ie] is not ValidationError") case other => ko(s"[$other] is not an error with IgluError") } @@ -372,9 +385,9 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect .extractAndValidateInputContexts(input, SpecHelpers.client, SpecHelpers.registryLookup) .value .map { - case Ior.Both(NonEmptyList(FailureDetails.SchemaViolation.IgluError(_, ValidationError(_, _)), Nil), Nil) => + case Ior.Both(NonEmptyList(SchemaViolationWithExtraContext(SchemaViolation.IgluError(_, _: ValidationError), _, _), Nil), Nil) => ok - case Ior.Both(NonEmptyList(ie: FailureDetails.SchemaViolation.IgluError, Nil), Nil) => + case Ior.Both(NonEmptyList(SchemaViolationWithExtraContext(ie: SchemaViolation.IgluError, _, _), Nil), Nil) => ko(s"IgluError [$ie] is not ValidationError") case other => ko(s"[$other] is not an error with IgluError") } @@ -388,9 +401,9 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect .extractAndValidateInputContexts(input, SpecHelpers.client, SpecHelpers.registryLookup) .value .map { - case Ior.Both(NonEmptyList(FailureDetails.SchemaViolation.IgluError(_, ResolutionError(_)), Nil), Nil) => + case Ior.Both(NonEmptyList(SchemaViolationWithExtraContext(SchemaViolation.IgluError(_, _: ResolutionError), _, _), Nil), Nil) => ok - case Ior.Both(NonEmptyList(ie: FailureDetails.SchemaViolation.IgluError, Nil), Nil) => + case Ior.Both(NonEmptyList(SchemaViolationWithExtraContext(ie: SchemaViolation.IgluError, _, _), Nil), Nil) => ko(s"IgluError [$ie] is not ResolutionError") case other => ko(s"[$other] is not an error with IgluError") } @@ -405,8 +418,8 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect .value .map { case Ior.Both(NonEmptyList( - FailureDetails.SchemaViolation.IgluError(_, ValidationError(_, _)), - List(FailureDetails.SchemaViolation.IgluError(_, ResolutionError(_))) + SchemaViolationWithExtraContext(SchemaViolation.IgluError(_, _: ValidationError), _, _), + List(SchemaViolationWithExtraContext(SchemaViolation.IgluError(_, _: ResolutionError), _, _)) ), Nil ) => @@ -423,7 +436,7 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect .extractAndValidateInputContexts(input, SpecHelpers.client, SpecHelpers.registryLookup) .value .map { - case Ior.Both(NonEmptyList(_: FailureDetails.SchemaViolation.IgluError, Nil), List(extract)) + case Ior.Both(NonEmptyList(SchemaViolationWithExtraContext(_: SchemaViolation.IgluError, _, _), Nil), List(extract)) if extract.sdj.schema == emailSentSchema => ok case other => ko(s"[$other] is not one IgluError and one SDJ with schema $emailSentSchema") @@ -486,7 +499,8 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect .validateEnrichmentsContexts(SpecHelpers.client, contexts, SpecHelpers.registryLookup) .value .map { - case Ior.Both(NonEmptyList(FailureDetails.SchemaViolation.IgluError(_, ValidationError(_, _)), Nil), Nil) => ok + case Ior.Both(NonEmptyList(SchemaViolationWithExtraContext(SchemaViolation.IgluError(_, _: ValidationError), _, _), Nil), Nil) => + ok case other => ko(s"[$other] is not one ValidationError") } } @@ -501,8 +515,8 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect .validateEnrichmentsContexts(SpecHelpers.client, contexts, SpecHelpers.registryLookup) .value .map { - case Ior.Both(NonEmptyList(FailureDetails.SchemaViolation.IgluError(_, ValidationError(_, _)), - List(FailureDetails.SchemaViolation.IgluError(_, ResolutionError(_))) + case Ior.Both(NonEmptyList(SchemaViolationWithExtraContext(SchemaViolation.IgluError(_, _: ValidationError), _, _), + List(SchemaViolationWithExtraContext(SchemaViolation.IgluError(_, _: ResolutionError), _, _)) ), Nil ) => @@ -522,7 +536,7 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect .value .map { case Ior.Both(NonEmptyList( - FailureDetails.SchemaViolation.IgluError(_, ValidationError(_, _)), + SchemaViolationWithExtraContext(SchemaViolation.IgluError(_, _: ValidationError), _, _), Nil ), List(sdj) @@ -563,7 +577,7 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect .map { case Ior.Both( NonEmptyList( - _: FailureDetails.SchemaViolation, + _: SchemaViolationWithExtraContext, Nil ), IgluUtils.EventExtractResult(Nil, None, Nil) @@ -587,7 +601,7 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect .map { case Ior.Both( NonEmptyList( - _: FailureDetails.SchemaViolation, + _: SchemaViolationWithExtraContext, Nil ), IgluUtils.EventExtractResult(Nil, None, Nil) @@ -612,8 +626,8 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect .map { case Ior.Both( NonEmptyList( - _: FailureDetails.SchemaViolation, - List(_: FailureDetails.SchemaViolation) + _: SchemaViolationWithExtraContext, + List(_: SchemaViolationWithExtraContext) ), IgluUtils.EventExtractResult(Nil, None, Nil) ) => @@ -661,7 +675,7 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect .value .map { case Ior.Both( - NonEmptyList(FailureDetails.SchemaViolation.IgluError(_, ValidationError(_, _)), _), + NonEmptyList(SchemaViolationWithExtraContext(SchemaViolation.IgluError(_, _: ValidationError), _, _), _), extract ) if extract.contexts.isEmpty && extract.unstructEvent.isDefined && extract.unstructEvent.get.schema == emailSentSchema => ok @@ -686,7 +700,7 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect .value .map { case Ior.Both( - NonEmptyList(FailureDetails.SchemaViolation.IgluError(_, ValidationError(_, _)), _), + NonEmptyList(SchemaViolationWithExtraContext(SchemaViolation.IgluError(_, _: ValidationError), _, _), _), extract ) if extract.contexts.size == 1 && extract.contexts.head.schema == emailSentSchema && extract.unstructEvent.isEmpty => ok