diff --git a/modules/common/src/main/scala/com.snowplowanalytics.snowplow.enrich/common/enrichments/AtomicFields.scala b/modules/common/src/main/scala/com.snowplowanalytics.snowplow.enrich/common/enrichments/AtomicFields.scala index f9721a519..67c950978 100644 --- a/modules/common/src/main/scala/com.snowplowanalytics.snowplow.enrich/common/enrichments/AtomicFields.scala +++ b/modules/common/src/main/scala/com.snowplowanalytics.snowplow.enrich/common/enrichments/AtomicFields.scala @@ -12,6 +12,9 @@ package com.snowplowanalytics.snowplow.enrich.common.enrichments import cats.data.NonEmptyList +import io.circe.Json +import io.circe.syntax._ + import com.snowplowanalytics.snowplow.badrows.FailureDetails import com.snowplowanalytics.iglu.client.ClientError.ValidationError @@ -133,17 +136,19 @@ object AtomicFields { AtomicFields(withLimits) } - def errorsToSchemaViolation(errors: NonEmptyList[ValidatorReport]): FailureEntity.SchemaViolationWithExtraContext = { + def errorsToSchemaViolation(errors: NonEmptyList[ValidatorReport]): FailureEntity.SchemaViolation = { val clientError = ValidationError(ValidatorError.InvalidData(errors), None) - FailureEntity.SchemaViolationWithExtraContext( + val failureData = Json.obj(errors.toList.flatMap(e => e.path.map(p => p := e.keyword)): _*) + + FailureEntity.SchemaViolation( schemaViolation = FailureDetails.SchemaViolation.IgluError( AtomicFields.atomicSchema, clientError ), // Source atomic field and actual value of the field should be already on the ValidatorReport list source = "atomic_field", - data = None + data = failureData ) } } diff --git a/modules/common/src/main/scala/com.snowplowanalytics.snowplow.enrich/common/enrichments/AtomicFieldsLengthValidator.scala b/modules/common/src/main/scala/com.snowplowanalytics.snowplow.enrich/common/enrichments/AtomicFieldsLengthValidator.scala index 9b58f11bf..7771968ba 100644 --- a/modules/common/src/main/scala/com.snowplowanalytics.snowplow.enrich/common/enrichments/AtomicFieldsLengthValidator.scala +++ b/modules/common/src/main/scala/com.snowplowanalytics.snowplow.enrich/common/enrichments/AtomicFieldsLengthValidator.scala @@ -35,7 +35,7 @@ object AtomicFieldsLengthValidator { acceptInvalid: Boolean, invalidCount: F[Unit], atomicFields: AtomicFields - ): IorT[F, FailureEntity.SchemaViolationWithExtraContext, Unit] = + ): IorT[F, FailureEntity.SchemaViolation, Unit] = IorT { atomicFields.value .map(validateField(event, _).toValidatedNel) 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 8143d15f9..861bda787 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 @@ -47,7 +47,12 @@ import com.snowplowanalytics.snowplow.enrich.common.utils.{IgluUtils, Conversion object EnrichmentManager { private type EnrichmentResult[F[_]] = - IorT[F, NonEmptyList[FailureEntity.BadRowWithFailureEntities], (EnrichedEvent, List[SelfDescribingData[Json]])] + IorT[F, NonEmptyList[IntermediateBadRow], (EnrichedEvent, List[SelfDescribingData[Json]])] + + private case class IntermediateBadRow( + failureEntities: NonEmptyList[FailureEntity], + partiallyEnrichedEvent: Payload.PartiallyEnrichedEvent + ) /** * Run the enrichment workflow @@ -75,7 +80,7 @@ object EnrichmentManager { emitIncomplete: Boolean ): IorT[F, BadRow, EnrichedEvent] = { val iorT: EnrichmentResult[F] = for { - enriched <- IorT.pure[F, NonEmptyList[FailureEntity.BadRowWithFailureEntities]](new EnrichedEvent) + enriched <- IorT.pure[F, NonEmptyList[IntermediateBadRow]](new EnrichedEvent) extractResult <- mapAndValidateInput( raw, enriched, @@ -94,7 +99,6 @@ object EnrichmentManager { } enrichmentsContexts <- runEnrichments( registry, - processor, raw, enriched, extractResult.contexts, @@ -105,10 +109,8 @@ object EnrichmentManager { .possiblyExitingEarly(emitIncomplete) validContexts <- validateEnriched( enriched, - raw, enrichmentsContexts, client, - processor, registryLookup, featureFlags.acceptInvalid, invalidCount, @@ -121,17 +123,36 @@ object EnrichmentManager { // derived contexts are set lastly because we want to include failure entities // to derived contexts as well and we can get failure entities only in the end // of the enrichment process - setDerivedContexts(iorT).leftMap(_.head.badRow).map(_._1) + setDerivedContexts(iorT, processor) + .leftMap(createBadRow(_, RawEvent.toRawEvent(raw), processor)) + .map(_._1) + } + + private def createBadRow( + fe: NonEmptyList[IntermediateBadRow], + re: Payload.RawEvent, + processor: Processor + ): BadRow = { + val intermediateBadRow = fe.head + intermediateBadRow.failureEntities.head match { + case h: FailureEntity.SchemaViolation => + val sv = intermediateBadRow.failureEntities.tail.collect { case f: FailureEntity.SchemaViolation => f } + buildSchemaViolationsBadRow(NonEmptyList(h, sv), intermediateBadRow.partiallyEnrichedEvent, re, processor) + case h: FailureEntity.EnrichmentFailure => + val ef = intermediateBadRow.failureEntities.tail.collect { case f: FailureEntity.EnrichmentFailure => f } + buildEnrichmentFailuresBadRow(NonEmptyList(h, ef), intermediateBadRow.partiallyEnrichedEvent, re, processor) + } } - private def setDerivedContexts[F[_]: Sync](enriched: EnrichmentResult[F]): EnrichmentResult[F] = + private def setDerivedContexts[F[_]: Sync](enriched: EnrichmentResult[F], processor: Processor): EnrichmentResult[F] = IorT( enriched.value.flatTap(v => Sync[F].delay { + val now = Instant.now() 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) + case Ior.Left(l) => (convertFailureEntitiesToSDJ(l, now, processor), None) + case Ior.Both(b, (e, l)) => (l ::: convertFailureEntitiesToSDJ(b, now, processor), e.some) } for { c <- ME.formatContexts(derivedContexts) @@ -142,8 +163,12 @@ object EnrichmentManager { ) ) - private def extractFailureEntities(l: NonEmptyList[FailureEntity.BadRowWithFailureEntities]): List[SelfDescribingData[Json]] = - l.toList.flatMap(_.failureEntities).map(FailureEntity.toSDJ) + private def convertFailureEntitiesToSDJ( + l: NonEmptyList[IntermediateBadRow], + timestamp: Instant, + processor: Processor + ): List[SelfDescribingData[Json]] = + l.flatMap(_.failureEntities).map(FailureEntity.toSDJ(_, timestamp, processor)).toList private def mapAndValidateInput[F[_]: Sync]( raw: RawEvent, @@ -152,21 +177,16 @@ object EnrichmentManager { processor: Processor, client: IgluCirceClient[F], registryLookup: RegistryLookup[F] - ): IorT[F, FailureEntity.BadRowWithFailureEntities, IgluUtils.EventExtractResult] = { + ): IorT[F, IntermediateBadRow, IgluUtils.EventExtractResult] = { val iorT = for { _ <- setupEnrichedEvent[F](raw, enrichedEvent, etlTstamp, processor) .leftMap(NonEmptyList.one) - extract <- IgluUtils.extractAndValidateInputJsons(enrichedEvent, client, registryLookup) + extract <- IgluUtils + .extractAndValidateInputJsons(enrichedEvent, client, registryLookup) + .leftMap { l: NonEmptyList[FailureEntity] => l } } yield extract - iorT.leftMap { violations => - buildSchemaViolationsBadRow( - violations, - EnrichedEvent.toPartiallyEnrichedEvent(enrichedEvent), - RawEvent.toRawEvent(raw), - processor - ) - } + iorT.leftMap(v => IntermediateBadRow(v, EnrichedEvent.toPartiallyEnrichedEvent(enrichedEvent))) } /** @@ -177,13 +197,12 @@ object EnrichmentManager { */ private def runEnrichments[F[_]: Monad]( registry: EnrichmentRegistry[F], - processor: Processor, raw: RawEvent, enriched: EnrichedEvent, inputContexts: List[SelfDescribingData[Json]], unstructEvent: Option[SelfDescribingData[Json]], legacyOrder: Boolean - ): IorT[F, FailureEntity.BadRowWithFailureEntities, List[SelfDescribingData[Json]]] = + ): IorT[F, IntermediateBadRow, List[SelfDescribingData[Json]]] = IorT { accState(registry, raw, inputContexts, unstructEvent, legacyOrder) .runS(Accumulation(enriched, Nil, Nil)) @@ -192,12 +211,7 @@ object EnrichmentManager { failures.toNel match { case Some(nel) => Ior.both( - buildEnrichmentFailuresBadRow( - nel, - EnrichedEvent.toPartiallyEnrichedEvent(enriched), - RawEvent.toRawEvent(raw), - processor - ), + IntermediateBadRow(nel.map(FailureEntity.EnrichmentFailure), EnrichedEvent.toPartiallyEnrichedEvent(enriched)), contexts ) case None => @@ -208,30 +222,21 @@ object EnrichmentManager { private def validateEnriched[F[_]: Clock: Monad]( enriched: EnrichedEvent, - raw: RawEvent, enrichmentsContexts: List[SelfDescribingData[Json]], client: IgluCirceClient[F], - processor: Processor, registryLookup: RegistryLookup[F], acceptInvalid: Boolean, invalidCount: F[Unit], atomicFields: AtomicFields - ): IorT[F, FailureEntity.BadRowWithFailureEntities, List[SelfDescribingData[Json]]] = { + ): IorT[F, IntermediateBadRow, List[SelfDescribingData[Json]]] = { val iorT = for { validContexts <- IgluUtils.validateEnrichmentsContexts[F](client, enrichmentsContexts, registryLookup) _ <- AtomicFieldsLengthValidator .validate[F](enriched, acceptInvalid, invalidCount, atomicFields) - .leftMap(NonEmptyList.one) + .leftMap { v: FailureEntity => NonEmptyList.one(v) } } yield validContexts - iorT.leftMap { violations => - buildSchemaViolationsBadRow( - violations, - EnrichedEvent.toPartiallyEnrichedEvent(enriched), - RawEvent.toRawEvent(raw), - processor - ) - } + iorT.leftMap(v => IntermediateBadRow(v, EnrichedEvent.toPartiallyEnrichedEvent(enriched))) } private[enrichments] case class Accumulation( @@ -360,7 +365,7 @@ object EnrichmentManager { e: EnrichedEvent, etlTstamp: DateTime, processor: Processor - ): IorT[F, FailureEntity.SchemaViolationWithExtraContext, Unit] = + ): IorT[F, FailureEntity.SchemaViolation, Unit] = IorT { Sync[F].delay { e.event_id = EE.generateEventId() // May be updated later if we have an `eid` parameter @@ -872,38 +877,30 @@ object EnrichmentManager { } private def buildSchemaViolationsBadRow( - vs: NonEmptyList[FailureEntity.SchemaViolationWithExtraContext], + fe: NonEmptyList[FailureEntity.SchemaViolation], pee: Payload.PartiallyEnrichedEvent, re: Payload.RawEvent, processor: Processor - ): FailureEntity.BadRowWithFailureEntities = { + ): BadRow = { val now = Instant.now() - 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 + BadRow.SchemaViolations( + processor, + Failure.SchemaViolations(now, fe.map(_.schemaViolation)), + Payload.EnrichmentPayload(pee, re) ) } private def buildEnrichmentFailuresBadRow( - fs: NonEmptyList[FailureDetails.EnrichmentFailure], + fe: NonEmptyList[FailureEntity.EnrichmentFailure], pee: Payload.PartiallyEnrichedEvent, re: Payload.RawEvent, processor: Processor - ): FailureEntity.BadRowWithFailureEntities = { + ): BadRow = { val now = Instant.now() - val failureEntities = fs.toList.flatMap(v => FailureEntity.fromEnrichmentFailure(v, now, processor)) - FailureEntity.BadRowWithFailureEntities( - badRow = BadRow.EnrichmentFailures( - processor, - Failure.EnrichmentFailures(now, fs), - Payload.EnrichmentPayload(pee, re) - ), - failureEntities = failureEntities + BadRow.EnrichmentFailures( + processor, + Failure.EnrichmentFailures(now, fe.map(_.enrichmentFailure)), + Payload.EnrichmentPayload(pee, re) ) } 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 fab511a61..6bb01a9e0 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 @@ -21,7 +21,7 @@ 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.client.validator.{ValidatorError, ValidatorReport} import com.snowplowanalytics.iglu.core.{SchemaKey, SchemaVer, SelfDescribingData} import com.snowplowanalytics.iglu.core.circe.implicits.schemaKeyCirceJsonEncoder @@ -30,53 +30,57 @@ import com.snowplowanalytics.iglu.core.circe.implicits.schemaKeyCirceJsonEncoder * Represents a failure encountered during enrichment of the event. * Failure entities will be attached to incomplete events as derived contexts. */ -case class FailureEntity( - failureType: String, - errors: List[Json], - schema: Option[SchemaKey], - data: Option[String], - timestamp: Instant, - componentName: String, - componentVersion: String -) +sealed trait FailureEntity object FailureEntity { 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, - failureEntities: List[FailureEntity] - ) - - /** - * Wrapper for schema violation failure that stores extra information about the failure. - * These extra information will be used while creating the failure entities that will be - * attached to incomplete events. - */ - case class SchemaViolationWithExtraContext( + case class SchemaViolation( schemaViolation: FailureDetails.SchemaViolation, source: String, - data: Option[String] + data: Json + ) extends FailureEntity + + case class EnrichmentFailure( + enrichmentFailure: FailureDetails.EnrichmentFailure + ) extends FailureEntity + + case class FailureEntityContext( + failureType: String, + errors: List[Json], + schema: Option[SchemaKey], + data: Option[Json], + timestamp: Instant, + componentName: String, + componentVersion: String ) - def toSDJ(failure: FailureEntity): SelfDescribingData[Json] = - SelfDescribingData( - failureEntitySchemaKey, - failure.asJson - ) + object FailureEntityContext { + implicit val failureEntityContextEncoder: Encoder[FailureEntityContext] = deriveEncoder[FailureEntityContext] + } + + def toSDJ( + fe: FailureEntity, + timestamp: Instant, + processor: Processor + ): SelfDescribingData[Json] = { + val feJson = fe match { + case sv: SchemaViolation => fromSchemaViolation(sv, timestamp, processor) + case ef: EnrichmentFailure => fromEnrichmentFailure(ef, timestamp, processor) + } + SelfDescribingData(failureEntitySchemaKey, feJson.asJson) + } def fromEnrichmentFailure( - ef: FailureDetails.EnrichmentFailure, + ef: EnrichmentFailure, timestamp: Instant, processor: Processor - ): Option[FailureEntity] = { - val failureType = s"EnrichmentError: ${ef.enrichment.map(_.identifier).getOrElse("")}" - ef.message match { + ): FailureEntityContext = { + val failureType = s"EnrichmentError: ${ef.enrichmentFailure.enrichment.map(_.identifier).getOrElse("")}" + val createContext = ef.enrichmentFailure.message match { case m: FailureDetails.EnrichmentFailureMessage.InputData => - FailureEntity( + FailureEntityContext( failureType = failureType, errors = List( Json.obj( @@ -84,76 +88,96 @@ object FailureEntity { "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 + schema = ef.enrichmentFailure.enrichment.map(_.schemaKey), + data = Json.obj(m.field := m.value).some, + _, + _, + _ + ) case m: FailureDetails.EnrichmentFailureMessage.Simple => - FailureEntity( + FailureEntityContext( + failureType = failureType, + errors = List( + Json.obj( + "message" := m.error + ) + ), + schema = ef.enrichmentFailure.enrichment.map(_.schemaKey), + data = None, + _, + _, + _ + ) + case m: FailureDetails.EnrichmentFailureMessage.IgluError => + // EnrichmentFailureMessage.IgluError isn't used anywhere in the project. + // We are return this value for completeness. + FailureEntityContext( failureType = failureType, errors = List( Json.obj( "message" := m.error ) ), - schema = ef.enrichment.map(_.schemaKey), + schema = ef.enrichmentFailure.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 + _, + _, + _ + ) } + createContext(timestamp, processor.artifact, processor.version) } def fromSchemaViolation( - v: SchemaViolationWithExtraContext, + v: SchemaViolation, timestamp: Instant, processor: Processor - ): FailureEntity = - v.schemaViolation match { + ): FailureEntityContext = { + val createContext = v.schemaViolation match { case f: FailureDetails.SchemaViolation.NotJson => - FailureEntity( + val error = Json.obj( + "message" := f.error, + "source" := v.source + ) + FailureEntityContext( failureType = "NotJSON", - errors = List(Json.obj("message" := f.error.asJson, "source" := v.source.asJson)), + errors = List(error), schema = None, - data = v.data, - timestamp = timestamp, - componentName = processor.artifact, - componentVersion = processor.version + data = Json.obj(v.source := v.data).some, + _, + _, + _ ) case f: FailureDetails.SchemaViolation.NotIglu => val message = f.error.message("").split(":").headOption - FailureEntity( + val error = Json.obj( + "message" := message, + "source" := v.source + ) + FailureEntityContext( failureType = "NotIglu", - errors = List(Json.obj("message" := message.asJson, "source" := v.source.asJson)), + errors = List(error), schema = None, - data = v.data, - timestamp = timestamp, - componentName = processor.artifact, - componentVersion = processor.version + data = v.data.some, + _, + _, + _ ) case f: FailureDetails.SchemaViolation.CriterionMismatch => val message = s"Unexpected schema: ${f.schemaKey.toSchemaUri} does not match the criterion" - FailureEntity( + val error = Json.obj( + "message" := message, + "source" := v.source, + "criterion" := f.schemaCriterion.asString + ) + FailureEntityContext( failureType = "CriterionMismatch", - errors = List( - Json.obj( - "message" := message, - "source" := v.source, - "criterion" := f.schemaCriterion.asString - ) - ), + errors = List(error), schema = f.schemaKey.some, - data = v.data, - timestamp = timestamp, - componentName = processor.artifact, - componentVersion = processor.version + data = v.data.some, + _, + _, + _ ) case FailureDetails.SchemaViolation.IgluError(schemaKey, ClientError.ResolutionError(lh)) => val message = s"Resolution error: schema ${schemaKey.toSchemaUri} not found" @@ -162,39 +186,42 @@ object FailureEntity { case (repo, lookups) => lookups.asJson.deepMerge(Json.obj("repository" := repo.asJson)) } - FailureEntity( + val error = Json.obj( + "message" := message, + "source" := v.source, + "lookupHistory" := lookupHistory + ) + FailureEntityContext( failureType = "ResolutionError", - errors = List( - Json.obj( - "message" := message, - "source" := v.source, - "lookupHistory" := lookupHistory - ) - ), + errors = List(error), schema = schemaKey.some, - data = v.data, - timestamp = timestamp, - componentName = processor.artifact, - componentVersion = processor.version + data = v.data.some, + _, + _, + _ ) case FailureDetails.SchemaViolation.IgluError(schemaKey, ClientError.ValidationError(ValidatorError.InvalidData(e), _)) => + val isAtomicField = schemaKey == AtomicFields.atomicSchema + // If error is for atomic field, we want to set the source to atomic field name. Since ValidatorReport.path + // is set to atomic field name, we are using path as source. + def source(r: ValidatorReport) = if (isAtomicField) r.path.getOrElse(v.source) else v.source val errors = e.toList.map { r => Json.obj( "message" := r.message, - "source" := v.source, + "source" := source(r), "path" := r.path, "keyword" := r.keyword, "targets" := r.targets ) } - FailureEntity( + FailureEntityContext( failureType = "ValidationError", errors = errors, schema = schemaKey.some, - data = v.data, - timestamp = timestamp, - componentName = processor.artifact, - componentVersion = processor.version + data = v.data.some, + _, + _, + _ ) case FailureDetails.SchemaViolation.IgluError(schemaKey, ClientError.ValidationError(ValidatorError.InvalidSchema(e), _)) => val errors = e.toList.map { r => @@ -204,14 +231,16 @@ object FailureEntity { "path" := r.path ) } - FailureEntity( + FailureEntityContext( failureType = "ValidationError", errors = errors, schema = schemaKey.some, - data = v.data, - timestamp = timestamp, - componentName = processor.artifact, - componentVersion = processor.version + data = v.data.some, + _, + _, + _ ) } + createContext(timestamp, processor.artifact, 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 96586e2ba..7f0d92868 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 @@ -25,7 +25,7 @@ import com.snowplowanalytics.iglu.client.resolver.registries.RegistryLookup import com.snowplowanalytics.iglu.core.{SchemaCriterion, SchemaKey, SchemaVer, SelfDescribingData} import com.snowplowanalytics.iglu.core.circe.implicits._ -import com.snowplowanalytics.snowplow.enrich.common.enrichments.FailureEntity._ +import com.snowplowanalytics.snowplow.enrich.common.enrichments.FailureEntity import com.snowplowanalytics.snowplow.badrows._ @@ -53,7 +53,7 @@ object IgluUtils { enriched: EnrichedEvent, client: IgluCirceClient[F], registryLookup: RegistryLookup[F] - ): IorT[F, NonEmptyList[SchemaViolationWithExtraContext], EventExtractResult] = + ): IorT[F, NonEmptyList[FailureEntity.SchemaViolation], EventExtractResult] = for { contexts <- extractAndValidateInputContexts(enriched, client, registryLookup) unstruct <- extractAndValidateUnstructEvent(enriched, client, registryLookup) @@ -81,7 +81,7 @@ object IgluUtils { registryLookup: RegistryLookup[F], field: String = "ue_properties", criterion: SchemaCriterion = SchemaCriterion("com.snowplowanalytics.snowplow", "unstruct_event", "jsonschema", 1, 0) - ): IorT[F, NonEmptyList[SchemaViolationWithExtraContext], Option[SdjExtractResult]] = + ): IorT[F, NonEmptyList[FailureEntity.SchemaViolation], Option[SdjExtractResult]] = Option(enriched.unstruct_event) match { case Some(rawUnstructEvent) => val iorT = for { @@ -94,7 +94,7 @@ object IgluUtils { } yield unstructSDJ.some iorT.recoverWith { case errors => IorT.fromIor[F](Ior.Both(errors, None)) } case None => - IorT.rightT[F, NonEmptyList[SchemaViolationWithExtraContext]](none[SdjExtractResult]) + IorT.rightT[F, NonEmptyList[FailureEntity.SchemaViolation]](none[SdjExtractResult]) } /** @@ -112,7 +112,7 @@ object IgluUtils { registryLookup: RegistryLookup[F], field: String = "contexts", criterion: SchemaCriterion = SchemaCriterion("com.snowplowanalytics.snowplow", "contexts", "jsonschema", 1, 0) - ): IorT[F, NonEmptyList[SchemaViolationWithExtraContext], List[SdjExtractResult]] = + ): IorT[F, NonEmptyList[FailureEntity.SchemaViolation], List[SdjExtractResult]] = Option(enriched.contexts) match { case Some(rawContexts) => val iorT = for { @@ -132,7 +132,7 @@ object IgluUtils { } yield contextsSdj iorT.recoverWith { case errors => IorT.fromIor[F](Ior.Both(errors, Nil)) } case None => - IorT.rightT[F, NonEmptyList[SchemaViolationWithExtraContext]](Nil) + IorT.rightT[F, NonEmptyList[FailureEntity.SchemaViolation]](Nil) } /** @@ -148,15 +148,15 @@ object IgluUtils { client: IgluCirceClient[F], sdjs: List[SelfDescribingData[Json]], registryLookup: RegistryLookup[F] - ): IorT[F, NonEmptyList[SchemaViolationWithExtraContext], List[SelfDescribingData[Json]]] = + ): IorT[F, NonEmptyList[FailureEntity.SchemaViolation], List[SelfDescribingData[Json]]] = checkList(client, sdjs, registryLookup) .leftMap( _.map { case (sdj, clientError) => - SchemaViolationWithExtraContext( + FailureEntity.SchemaViolation( schemaViolation = FailureDetails.SchemaViolation.IgluError(sdj.schema, clientError), source = "derived_contexts", - data = sdj.asString.some + data = sdj.asJson ) } ) @@ -168,16 +168,16 @@ object IgluUtils { expectedCriterion: SchemaCriterion, client: IgluCirceClient[F], registryLookup: RegistryLookup[F] - ): EitherT[F, SchemaViolationWithExtraContext, Json] = + ): EitherT[F, FailureEntity.SchemaViolation, Json] = for { // Parse Json string with the SDJ json <- JsonUtils .extractJson(rawJson) .leftMap(e => - SchemaViolationWithExtraContext( + FailureEntity.SchemaViolation( schemaViolation = FailureDetails.SchemaViolation.NotJson(field, rawJson.some, e), source = field, - data = s"""{"$field" : "$rawJson"}""".some + data = rawJson.asJson ) ) .toEitherT[F] @@ -185,37 +185,37 @@ object IgluUtils { sdj <- SelfDescribingData .parse(json) .leftMap(e => - SchemaViolationWithExtraContext( + FailureEntity.SchemaViolation( schemaViolation = FailureDetails.SchemaViolation.NotIglu(json, e), source = field, - data = json.noSpaces.some + data = json ) ) .toEitherT[F] // Check that the schema of SelfDescribingData[Json] is the expected one _ <- if (validateCriterion(sdj, expectedCriterion)) - EitherT.rightT[F, SchemaViolationWithExtraContext](sdj) + EitherT.rightT[F, FailureEntity.SchemaViolation](sdj) else EitherT .leftT[F, SelfDescribingData[Json]]( - SchemaViolationWithExtraContext( + FailureEntity.SchemaViolation( schemaViolation = FailureDetails.SchemaViolation.CriterionMismatch(sdj.schema, expectedCriterion), source = field, - data = sdj.asString.some + data = sdj.asJson ) ) // Check that the SDJ holding the .data is valid _ <- check(client, sdj, registryLookup) .leftMap { case (schemaKey, clientError) => - SchemaViolationWithExtraContext( + FailureEntity.SchemaViolation( schemaViolation = FailureDetails.SchemaViolation.IgluError(schemaKey, clientError), source = field, - data = sdj.asString.some + data = sdj.asJson ) } // Extract .data of SelfDescribingData[Json] - data <- EitherT.rightT[F, SchemaViolationWithExtraContext](sdj.data) + data <- EitherT.rightT[F, FailureEntity.SchemaViolation](sdj.data) } yield data /** Check that the schema of a SDJ matches the expected one */ @@ -257,26 +257,26 @@ object IgluUtils { client: IgluCirceClient[F], registryLookup: RegistryLookup[F], field: String - ): IorT[F, NonEmptyList[SchemaViolationWithExtraContext], SdjExtractResult] = + ): IorT[F, NonEmptyList[FailureEntity.SchemaViolation], SdjExtractResult] = for { sdj <- IorT .fromEither[F](SelfDescribingData.parse(json)) - .leftMap[SchemaViolationWithExtraContext](e => - SchemaViolationWithExtraContext( + .leftMap[FailureEntity.SchemaViolation](e => + FailureEntity.SchemaViolation( schemaViolation = FailureDetails.SchemaViolation.NotIglu(json, e), source = field, - data = json.noSpaces.some + data = json.asJson ) ) .leftMap(NonEmptyList.one) supersedingSchema <- check(client, sdj, registryLookup) .leftMap { case (schemaKey, clientError) => - SchemaViolationWithExtraContext( + FailureEntity.SchemaViolation( schemaViolation = FailureDetails.SchemaViolation .IgluError(schemaKey, clientError): FailureDetails.SchemaViolation, source = field, - data = json.noSpaces.some + data = json.asJson ) } .leftMap(NonEmptyList.one) 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 5a9335823..44f14f407 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 @@ -25,9 +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.badrows.FailureDetails -import com.snowplowanalytics.snowplow.enrich.common.enrichments.FailureEntity.SchemaViolationWithExtraContext +import com.snowplowanalytics.snowplow.enrich.common.enrichments.FailureEntity import com.snowplowanalytics.snowplow.enrich.common.outputs.EnrichedEvent import com.snowplowanalytics.snowplow.enrich.common.SpecHelpers import com.snowplowanalytics.snowplow.enrich.common.adapters.RawEvent @@ -149,7 +149,7 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect } } - "return a SchemaViolation.NotJson if unstruct_event does not contain a properly formatted JSON string" >> { + "return a FailureDetails.FailureDetails.SchemaViolation.NotJson if unstruct_event does not contain a properly formatted JSON string" >> { val input = new EnrichedEvent input.setUnstruct_event(notJson) @@ -157,12 +157,16 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect .extractAndValidateUnstructEvent(input, SpecHelpers.client, SpecHelpers.registryLookup) .value .map { - case Ior.Both(NonEmptyList(SchemaViolationWithExtraContext(_: SchemaViolation.NotJson, `ueProperties`, `jsonNotJson`), _), None) => ok + case Ior.Both( + NonEmptyList(FailureEntity.SchemaViolation(_: FailureDetails.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" >> { + "return a FailureDetails.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) @@ -171,12 +175,15 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect .extractAndValidateUnstructEvent(input, SpecHelpers.client, SpecHelpers.registryLookup) .value .map { - case Ior.Both(NonEmptyList(SchemaViolationWithExtraContext(_: SchemaViolation.NotIglu, `ueProperties`, `json`), _), None) => ok + case Ior.Both(NonEmptyList(FailureEntity.SchemaViolation(_: FailureDetails.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" >> { + "return a FailureDetails.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) @@ -185,12 +192,16 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect .extractAndValidateUnstructEvent(input, SpecHelpers.client, SpecHelpers.registryLookup) .value .map { - case Ior.Both(NonEmptyList(SchemaViolationWithExtraContext(_: SchemaViolation.CriterionMismatch, `ueProperties`, `json`), _), None) => ok + case Ior.Both( + NonEmptyList(FailureEntity.SchemaViolation(_: FailureDetails.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" >> { + "return a FailureDetails.SchemaViolation.NotJson if the JSON in .data is not a JSON" >> { val input = new EnrichedEvent val ue = buildUnstruct(notJson) val ueJson = ue.asJson @@ -200,12 +211,15 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect .extractAndValidateUnstructEvent(input, SpecHelpers.client, SpecHelpers.registryLookup) .value .map { - case Ior.Both(NonEmptyList(SchemaViolationWithExtraContext(_: SchemaViolation.NotJson, `ueProperties`, `ueJson`), _), None) => ok + case Ior.Both(NonEmptyList(FailureEntity.SchemaViolation(_: FailureDetails.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" >> { + "return a FailureDetails.SchemaViolation.IgluError containing a ValidationError if the JSON in .data is not self-describing" >> { val input = new EnrichedEvent val ue = buildUnstruct(notIglu) val ueJson = parse(ue).toOption.get @@ -215,13 +229,20 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect .extractAndValidateUnstructEvent(input, SpecHelpers.client, SpecHelpers.registryLookup) .value .map { - case Ior.Both(NonEmptyList(SchemaViolationWithExtraContext(SchemaViolation.IgluError(_, _: ValidationError), `ueProperties`, `ueJson`), _), None) => + case Ior.Both(NonEmptyList(FailureEntity.SchemaViolation(FailureDetails.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" >> { + "return a FailureDetails.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)) @@ -230,13 +251,20 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect .extractAndValidateUnstructEvent(input, SpecHelpers.client, SpecHelpers.registryLookup) .value .map { - case Ior.Both(NonEmptyList(SchemaViolationWithExtraContext(SchemaViolation.IgluError(_, _: ValidationError), `ueProperties`, `json`), _), None) => + case Ior.Both(NonEmptyList(FailureEntity.SchemaViolation(FailureDetails.SchemaViolation.IgluError(_, _: ValidationError), + `ueProperties`, + `json` + ), + _ + ), + None + ) => ok case other => ko(s"[$other] is not expected one") } } - "return a SchemaViolation.IgluError containing a ResolutionError if the schema of the SDJ in .data can't be resolved" >> { + "return a FailureDetails.SchemaViolation.IgluError containing a ResolutionError if the schema of the SDJ in .data can't be resolved" >> { val input = new EnrichedEvent input.setUnstruct_event(buildUnstruct(noSchema)) @@ -244,9 +272,12 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect .extractAndValidateUnstructEvent(input, SpecHelpers.client, SpecHelpers.registryLookup) .value .map { - case Ior.Both(NonEmptyList(SchemaViolationWithExtraContext(SchemaViolation.IgluError(_, _: ResolutionError), _, _), _), None) => + case Ior.Both( + NonEmptyList(FailureEntity.SchemaViolation(FailureDetails.SchemaViolation.IgluError(_, _: ResolutionError), _, _), _), + None + ) => ok - case Ior.Both(NonEmptyList(SchemaViolationWithExtraContext(ie: SchemaViolation.IgluError, _, _), _), None) => + case Ior.Both(NonEmptyList(FailureEntity.SchemaViolation(ie: FailureDetails.SchemaViolation.IgluError, _, _), _), None) => ko(s"IgluError [$ie] is not a ResolutionError") case other => ko(s"[$other] is not an error with IgluError") } @@ -320,7 +351,7 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect } } - "return a SchemaViolation.NotJson if .contexts does not contain a properly formatted JSON string" >> { + "return a FailureDetails.SchemaViolation.NotJson if .contexts does not contain a properly formatted JSON string" >> { val input = new EnrichedEvent input.setContexts(notJson) @@ -328,12 +359,16 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect .extractAndValidateInputContexts(input, SpecHelpers.client, SpecHelpers.registryLookup) .value .map { - case Ior.Both(NonEmptyList(SchemaViolationWithExtraContext(_: SchemaViolation.NotJson, `contexts`, `jsonNotJson`), Nil), Nil) => ok + case Ior.Both( + NonEmptyList(FailureEntity.SchemaViolation(_: FailureDetails.SchemaViolation.NotJson, `contexts`, `jsonNotJson`), Nil), + Nil + ) => + ok case other => ko(s"[$other] is not an error with NotJson") } } - "return a SchemaViolation.NotIglu if .contexts contains a properly formatted JSON string that is not self-describing" >> { + "return a FailureDetails.SchemaViolation.NotIglu if .contexts contains a properly formatted JSON string that is not self-describing" >> { val input = new EnrichedEvent input.setContexts(notIglu) @@ -341,12 +376,12 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect .extractAndValidateInputContexts(input, SpecHelpers.client, SpecHelpers.registryLookup) .value .map { - case Ior.Both(NonEmptyList(SchemaViolationWithExtraContext(_: SchemaViolation.NotIglu, _, _), Nil), Nil) => ok + case Ior.Both(NonEmptyList(FailureEntity.SchemaViolation(_: FailureDetails.SchemaViolation.NotIglu, _, _), Nil), Nil) => ok case other => ko(s"[$other] is not an error with NotIglu") } } - "return a SchemaViolation.CriterionMismatch if .contexts contains a self-describing JSON but not with the right schema" >> { + "return a FailureDetails.SchemaViolation.CriterionMismatch if .contexts contains a self-describing JSON but not with the right schema" >> { val input = new EnrichedEvent input.setContexts(noSchema) @@ -354,12 +389,13 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect .extractAndValidateInputContexts(input, SpecHelpers.client, SpecHelpers.registryLookup) .value .map { - case Ior.Both(NonEmptyList(SchemaViolationWithExtraContext(_: SchemaViolation.CriterionMismatch, _, _), Nil), Nil) => ok + case Ior.Both(NonEmptyList(FailureEntity.SchemaViolation(_: FailureDetails.SchemaViolation.CriterionMismatch, _, _), Nil), Nil) => + ok case other => ko(s"[$other] is not an error with CriterionMismatch") } } - "return a SchemaViolation.IgluError containing a ValidationError if .data does not contain an array of JSON objects" >> { + "return a FailureDetails.SchemaViolation.IgluError containing a ValidationError if .data does not contain an array of JSON objects" >> { val input = new EnrichedEvent val notArrayContexts = s"""{"schema": "${inputContextsSchema.toSchemaUri}", "data": ${emailSent1}}""" @@ -369,15 +405,18 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect .extractAndValidateInputContexts(input, SpecHelpers.client, SpecHelpers.registryLookup) .value .map { - case Ior.Both(NonEmptyList(SchemaViolationWithExtraContext(SchemaViolation.IgluError(_, _: ValidationError), _, _), Nil), Nil) => + case Ior.Both( + NonEmptyList(FailureEntity.SchemaViolation(FailureDetails.SchemaViolation.IgluError(_, _: ValidationError), _, _), Nil), + Nil + ) => ok - case Ior.Both(NonEmptyList(SchemaViolationWithExtraContext(ie: SchemaViolation.IgluError, _, _), Nil), Nil) => + case Ior.Both(NonEmptyList(FailureEntity.SchemaViolation(ie: FailureDetails.SchemaViolation.IgluError, _, _), Nil), Nil) => ko(s"IgluError [$ie] is not ValidationError") case other => ko(s"[$other] is not an error with IgluError") } } - "return a SchemaViolation.IgluError containing a ValidationError if .data contains one invalid context" >> { + "return a FailureDetails.SchemaViolation.IgluError containing a ValidationError if .data contains one invalid context" >> { val input = new EnrichedEvent input.setContexts(buildInputContexts(List(invalidEmailSent))) @@ -385,15 +424,18 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect .extractAndValidateInputContexts(input, SpecHelpers.client, SpecHelpers.registryLookup) .value .map { - case Ior.Both(NonEmptyList(SchemaViolationWithExtraContext(SchemaViolation.IgluError(_, _: ValidationError), _, _), Nil), Nil) => + case Ior.Both( + NonEmptyList(FailureEntity.SchemaViolation(FailureDetails.SchemaViolation.IgluError(_, _: ValidationError), _, _), Nil), + Nil + ) => ok - case Ior.Both(NonEmptyList(SchemaViolationWithExtraContext(ie: SchemaViolation.IgluError, _, _), Nil), Nil) => + case Ior.Both(NonEmptyList(FailureEntity.SchemaViolation(ie: FailureDetails.SchemaViolation.IgluError, _, _), Nil), Nil) => ko(s"IgluError [$ie] is not ValidationError") case other => ko(s"[$other] is not an error with IgluError") } } - "return a SchemaViolation.IgluError containing a ResolutionError if .data contains one context whose schema can't be resolved" >> { + "return a FailureDetails.SchemaViolation.IgluError containing a ResolutionError if .data contains one context whose schema can't be resolved" >> { val input = new EnrichedEvent input.setContexts(buildInputContexts(List(noSchema))) @@ -401,9 +443,12 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect .extractAndValidateInputContexts(input, SpecHelpers.client, SpecHelpers.registryLookup) .value .map { - case Ior.Both(NonEmptyList(SchemaViolationWithExtraContext(SchemaViolation.IgluError(_, _: ResolutionError), _, _), Nil), Nil) => + case Ior.Both( + NonEmptyList(FailureEntity.SchemaViolation(FailureDetails.SchemaViolation.IgluError(_, _: ResolutionError), _, _), Nil), + Nil + ) => ok - case Ior.Both(NonEmptyList(SchemaViolationWithExtraContext(ie: SchemaViolation.IgluError, _, _), Nil), Nil) => + case Ior.Both(NonEmptyList(FailureEntity.SchemaViolation(ie: FailureDetails.SchemaViolation.IgluError, _, _), Nil), Nil) => ko(s"IgluError [$ie] is not ResolutionError") case other => ko(s"[$other] is not an error with IgluError") } @@ -418,8 +463,8 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect .value .map { case Ior.Both(NonEmptyList( - SchemaViolationWithExtraContext(SchemaViolation.IgluError(_, _: ValidationError), _, _), - List(SchemaViolationWithExtraContext(SchemaViolation.IgluError(_, _: ResolutionError), _, _)) + FailureEntity.SchemaViolation(FailureDetails.SchemaViolation.IgluError(_, _: ValidationError), _, _), + List(FailureEntity.SchemaViolation(FailureDetails.SchemaViolation.IgluError(_, _: ResolutionError), _, _)) ), Nil ) => @@ -436,7 +481,7 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect .extractAndValidateInputContexts(input, SpecHelpers.client, SpecHelpers.registryLookup) .value .map { - case Ior.Both(NonEmptyList(SchemaViolationWithExtraContext(_: SchemaViolation.IgluError, _, _), Nil), List(extract)) + case Ior.Both(NonEmptyList(FailureEntity.SchemaViolation(_: FailureDetails.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") @@ -499,7 +544,10 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect .validateEnrichmentsContexts(SpecHelpers.client, contexts, SpecHelpers.registryLookup) .value .map { - case Ior.Both(NonEmptyList(SchemaViolationWithExtraContext(SchemaViolation.IgluError(_, _: ValidationError), _, _), Nil), Nil) => + case Ior.Both( + NonEmptyList(FailureEntity.SchemaViolation(FailureDetails.SchemaViolation.IgluError(_, _: ValidationError), _, _), Nil), + Nil + ) => ok case other => ko(s"[$other] is not one ValidationError") } @@ -515,8 +563,9 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect .validateEnrichmentsContexts(SpecHelpers.client, contexts, SpecHelpers.registryLookup) .value .map { - case Ior.Both(NonEmptyList(SchemaViolationWithExtraContext(SchemaViolation.IgluError(_, _: ValidationError), _, _), - List(SchemaViolationWithExtraContext(SchemaViolation.IgluError(_, _: ResolutionError), _, _)) + case Ior.Both(NonEmptyList( + FailureEntity.SchemaViolation(FailureDetails.SchemaViolation.IgluError(_, _: ValidationError), _, _), + List(FailureEntity.SchemaViolation(FailureDetails.SchemaViolation.IgluError(_, _: ResolutionError), _, _)) ), Nil ) => @@ -536,7 +585,7 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect .value .map { case Ior.Both(NonEmptyList( - SchemaViolationWithExtraContext(SchemaViolation.IgluError(_, _: ValidationError), _, _), + FailureEntity.SchemaViolation(FailureDetails.SchemaViolation.IgluError(_, _: ValidationError), _, _), Nil ), List(sdj) @@ -577,7 +626,7 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect .map { case Ior.Both( NonEmptyList( - _: SchemaViolationWithExtraContext, + _: FailureEntity.SchemaViolation, Nil ), IgluUtils.EventExtractResult(Nil, None, Nil) @@ -601,7 +650,7 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect .map { case Ior.Both( NonEmptyList( - _: SchemaViolationWithExtraContext, + _: FailureEntity.SchemaViolation, Nil ), IgluUtils.EventExtractResult(Nil, None, Nil) @@ -626,8 +675,8 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect .map { case Ior.Both( NonEmptyList( - _: SchemaViolationWithExtraContext, - List(_: SchemaViolationWithExtraContext) + _: FailureEntity.SchemaViolation, + List(_: FailureEntity.SchemaViolation) ), IgluUtils.EventExtractResult(Nil, None, Nil) ) => @@ -675,7 +724,7 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect .value .map { case Ior.Both( - NonEmptyList(SchemaViolationWithExtraContext(SchemaViolation.IgluError(_, _: ValidationError), _, _), _), + NonEmptyList(FailureEntity.SchemaViolation(FailureDetails.SchemaViolation.IgluError(_, _: ValidationError), _, _), _), extract ) if extract.contexts.isEmpty && extract.unstructEvent.isDefined && extract.unstructEvent.get.schema == emailSentSchema => ok @@ -700,7 +749,7 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect .value .map { case Ior.Both( - NonEmptyList(SchemaViolationWithExtraContext(SchemaViolation.IgluError(_, _: ValidationError), _, _), _), + NonEmptyList(FailureEntity.SchemaViolation(FailureDetails.SchemaViolation.IgluError(_, _: ValidationError), _, _), _), extract ) if extract.contexts.size == 1 && extract.contexts.head.schema == emailSentSchema && extract.unstructEvent.isEmpty => ok