From 22a978b8c068bbb7aed8b48b157ee2e2283de4cb Mon Sep 17 00:00:00 2001 From: spenes Date: Tue, 26 Mar 2024 15:37:58 +0300 Subject: [PATCH] [WIP] Add failure entity for incomplete events With this commit, failure entities are started to be attached to incomplete events as derived contexts. There are some cases that we need more information than in the bad rows to create failure entities therefore it isn't possible to create failure entities from bad rows directly. For this purpose, we created wrapper classes to attach extra information about failure entities. --- .../common/enrichments/AtomicFields.scala | 13 ++- .../AtomicFieldsLengthValidator.scala | 4 +- .../enrichments/EnrichmentManager.scala | 94 ++++++++++++------ .../common/enrichments/FailureEntity.scala | 96 +++++++++++++++++++ .../common/utils/IgluUtils.scala | 87 ++++++++++++----- 5 files changed, 231 insertions(+), 63 deletions(-) create mode 100644 modules/common/src/main/scala/com.snowplowanalytics.snowplow.enrich/common/enrichments/FailureEntity.scala 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 26e513760..f9721a519 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 @@ -133,12 +133,17 @@ object AtomicFields { AtomicFields(withLimits) } - def errorsToSchemaViolation(errors: NonEmptyList[ValidatorReport]): FailureDetails.SchemaViolation = { + def errorsToSchemaViolation(errors: NonEmptyList[ValidatorReport]): FailureEntity.SchemaViolationWithExtraContext = { val clientError = ValidationError(ValidatorError.InvalidData(errors), None) - FailureDetails.SchemaViolation.IgluError( - AtomicFields.atomicSchema, - clientError + FailureEntity.SchemaViolationWithExtraContext( + 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 ) } } 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 c14a7a6d2..9b58f11bf 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 @@ -19,8 +19,6 @@ import cats.implicits._ import com.snowplowanalytics.iglu.client.validator.ValidatorReport -import com.snowplowanalytics.snowplow.badrows.FailureDetails - import com.snowplowanalytics.snowplow.enrich.common.enrichments.AtomicFields.LimitedAtomicField import com.snowplowanalytics.snowplow.enrich.common.outputs.EnrichedEvent @@ -37,7 +35,7 @@ object AtomicFieldsLengthValidator { acceptInvalid: Boolean, invalidCount: F[Unit], atomicFields: AtomicFields - ): IorT[F, FailureDetails.SchemaViolation, Unit] = + ): IorT[F, FailureEntity.SchemaViolationWithExtraContext, 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 54be85e71..f14fdd137 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,6 +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]])] + /** * Run the enrichment workflow * @param registry Contain configuration for all enrichments to apply @@ -71,8 +73,8 @@ object EnrichmentManager { atomicFields: AtomicFields, emitIncomplete: Boolean ): IorT[F, BadRow, EnrichedEvent] = { - val iorT: IorT[F, NonEmptyList[BadRow], EnrichedEvent] = for { - enriched <- IorT.pure[F, NonEmptyList[BadRow]](new EnrichedEvent) + val iorT: EnrichmentResult[F] = for { + enriched <- IorT.pure[F, NonEmptyList[FailureEntity.BadRowWithFailureEntities]](new EnrichedEvent) extractResult <- mapAndValidateInput( raw, enriched, @@ -100,11 +102,10 @@ object EnrichmentManager { ) .leftMap(NonEmptyList.one) .possiblyExitingEarly(emitIncomplete) - _ <- validateEnriched( + validContexts <- validateEnriched( enriched, raw, enrichmentsContexts, - extractResult.validationInfoContexts, client, processor, registryLookup, @@ -114,11 +115,33 @@ object EnrichmentManager { ) .leftMap(NonEmptyList.one) .possiblyExitingEarly(emitIncomplete) - } yield enriched + } yield (enriched, validContexts ::: extractResult.validationInfoContexts) - iorT.leftMap(_.head) + // 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) } + 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) + } + for { + c <- ME.formatContexts(derivedContexts) + e <- enriched + _ = e.derived_contexts = c + } yield () + }) + ) + + private def extractFailureEntities(l: NonEmptyList[FailureEntity.BadRowWithFailureEntities]): List[SelfDescribingData[Json]] = + l.toList.flatMap(_.failureEntities).map(FailureEntity.toSDJ) + private def mapAndValidateInput[F[_]: Sync]( raw: RawEvent, enrichedEvent: EnrichedEvent, @@ -126,7 +149,7 @@ object EnrichmentManager { processor: Processor, client: IgluCirceClient[F], registryLookup: RegistryLookup[F] - ): IorT[F, BadRow, IgluUtils.EventExtractResult] = { + ): IorT[F, FailureEntity.BadRowWithFailureEntities, IgluUtils.EventExtractResult] = { val iorT = for { _ <- setupEnrichedEvent[F](raw, enrichedEvent, etlTstamp, processor) .leftMap(NonEmptyList.one) @@ -134,12 +157,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 + ) } } @@ -157,7 +180,7 @@ object EnrichmentManager { inputContexts: List[SelfDescribingData[Json]], unstructEvent: Option[SelfDescribingData[Json]], legacyOrder: Boolean - ): IorT[F, BadRow, List[SelfDescribingData[Json]]] = + ): IorT[F, FailureEntity.BadRowWithFailureEntities, List[SelfDescribingData[Json]]] = IorT { accState(registry, raw, inputContexts, unstructEvent, legacyOrder) .runS(Accumulation(enriched, Nil, Nil)) @@ -184,21 +207,19 @@ object EnrichmentManager { enriched: EnrichedEvent, raw: RawEvent, enrichmentsContexts: List[SelfDescribingData[Json]], - validationInfoContexts: List[SelfDescribingData[Json]], client: IgluCirceClient[F], processor: Processor, registryLookup: RegistryLookup[F], acceptInvalid: Boolean, invalidCount: F[Unit], atomicFields: AtomicFields - ): IorT[F, BadRow, Unit] = { + ): IorT[F, FailureEntity.BadRowWithFailureEntities, List[SelfDescribingData[Json]]] = { val iorT = for { validContexts <- IgluUtils.validateEnrichmentsContexts[F](client, enrichmentsContexts, registryLookup) - _ = ME.formatContexts(validContexts ::: validationInfoContexts).foreach(enriched.derived_contexts = _) _ <- AtomicFieldsLengthValidator .validate[F](enriched, acceptInvalid, invalidCount, atomicFields) .leftMap(NonEmptyList.one) - } yield () + } yield validContexts iorT.leftMap { violations => buildSchemaViolationsBadRow( @@ -336,7 +357,7 @@ object EnrichmentManager { e: EnrichedEvent, etlTstamp: DateTime, processor: Processor - ): IorT[F, FailureDetails.SchemaViolation, Unit] = + ): IorT[F, FailureEntity.SchemaViolationWithExtraContext, Unit] = IorT { Sync[F].delay { e.event_id = EE.generateEventId() // May be updated later if we have an `eid` parameter @@ -848,28 +869,41 @@ object EnrichmentManager { } private def buildSchemaViolationsBadRow( - vs: NonEmptyList[FailureDetails.SchemaViolation], + vs: NonEmptyList[FailureEntity.SchemaViolationWithExtraContext], pee: Payload.PartiallyEnrichedEvent, re: Payload.RawEvent, processor: Processor - ): BadRow.SchemaViolations = - BadRow.SchemaViolations( - processor, - Failure.SchemaViolations(Instant.now(), vs), - Payload.EnrichmentPayload(pee, re) + ): FailureEntity.BadRowWithFailureEntities = { + val now = Instant.now() + val failureEntities = vs.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 ) + } private def buildEnrichmentFailuresBadRow( fs: NonEmptyList[FailureDetails.EnrichmentFailure], pee: Payload.PartiallyEnrichedEvent, re: Payload.RawEvent, processor: Processor - ) = - BadRow.EnrichmentFailures( - processor, - Failure.EnrichmentFailures(Instant.now(), fs), - Payload.EnrichmentPayload(pee, re) + ): FailureEntity.BadRowWithFailureEntities = { + val now = Instant.now() + // TODO: Fill it correctly + val failureEntities = Nil + FailureEntity.BadRowWithFailureEntities( + badRow = BadRow.EnrichmentFailures( + processor, + Failure.EnrichmentFailures(now, fs), + Payload.EnrichmentPayload(pee, re) + ), + failureEntities = failureEntities ) + } private implicit class IorTOps[F[_], A, B](val iorT: IorT[F, A, B]) extends AnyVal { 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 new file mode 100644 index 000000000..a96a277fd --- /dev/null +++ b/modules/common/src/main/scala/com.snowplowanalytics.snowplow.enrich/common/enrichments/FailureEntity.scala @@ -0,0 +1,96 @@ +/* + * Copyright (c) 2024-present Snowplow Analytics Ltd. + * All rights reserved. + * + * This software is made available by Snowplow Analytics, Ltd., + * under the terms of the Snowplow Limited Use License Agreement, Version 1.0 + * located at https://docs.snowplow.io/limited-use-license-1.0 + * BY INSTALLING, DOWNLOADING, ACCESSING, USING OR DISTRIBUTING ANY PORTION + * OF THE SOFTWARE, YOU AGREE TO THE TERMS OF SUCH LICENSE AGREEMENT. + */ +package com.snowplowanalytics.snowplow.enrich.common.enrichments + +import java.time.Instant + +import io.circe.{Encoder, Json} +import io.circe.generic.semiauto._ +import io.circe.syntax._ + +import com.snowplowanalytics.snowplow.badrows._ + +import com.snowplowanalytics.iglu.core.{SchemaKey, SchemaVer, SelfDescribingData} + +/** + * 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[String], + data: Option[String], + timestamp: Instant, + componentName: String, + componentVersion: String +) + +object FailureEntity { + + val schemaKey = SchemaKey("com.snowplowanalytics.snowplow", "failure", "jsonschema", SchemaVer.Full(1, 0, 0)) + + 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( + schemaViolation: FailureDetails.SchemaViolation, + source: String, + data: Option[String] + ) + + def toSDJ(failure: FailureEntity): SelfDescribingData[Json] = + SelfDescribingData( + schemaKey, + failure.asJson + ) + + implicit val instantEncoder: Encoder[Instant] = Encoder.encodeString.contramap[Instant](_.toString) + implicit val encoder: Encoder[FailureEntity] = deriveEncoder[FailureEntity] + + def fromSchemaViolation( + v: SchemaViolationWithExtraContext, + timestamp: Instant, + processor: Processor + ): FailureEntity = + v.schemaViolation match { + case f: FailureDetails.SchemaViolation.NotJson => + FailureEntity( + failureType = "NotJson", + errors = List(Json.obj("message" -> f.error.asJson, "source" -> v.source.asJson)), + schema = None, + data = v.data, + timestamp = timestamp, + componentName = processor.artifact, + componentVersion = processor.version + ) + case f: FailureDetails.SchemaViolation.NotIglu => + val message = f.error.message("").split(":").headOption + FailureEntity( + failureType = "NotIglu", + 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("") + } +} 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 8483b2cca..7c425066b 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,6 +25,8 @@ 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.badrows._ import com.snowplowanalytics.snowplow.enrich.common.outputs.EnrichedEvent @@ -51,7 +53,7 @@ object IgluUtils { enriched: EnrichedEvent, client: IgluCirceClient[F], registryLookup: RegistryLookup[F] - ): IorT[F, NonEmptyList[FailureDetails.SchemaViolation], EventExtractResult] = + ): IorT[F, NonEmptyList[SchemaViolationWithExtraContext], EventExtractResult] = for { contexts <- extractAndValidateInputContexts(enriched, client, registryLookup) unstruct <- extractAndValidateUnstructEvent(enriched, client, registryLookup) @@ -79,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[FailureDetails.SchemaViolation], Option[SdjExtractResult]] = + ): IorT[F, NonEmptyList[SchemaViolationWithExtraContext], Option[SdjExtractResult]] = Option(enriched.unstruct_event) match { case Some(rawUnstructEvent) => val iorT = for { @@ -88,11 +90,11 @@ object IgluUtils { .leftMap(NonEmptyList.one) .toIor // Parse Json unstructured event as SelfDescribingData[Json] - unstructSDJ <- parseAndValidateSDJ(unstruct, client, registryLookup) + unstructSDJ <- parseAndValidateSDJ(unstruct, client, registryLookup, field) } yield unstructSDJ.some iorT.recoverWith { case errors => IorT.fromIor[F](Ior.Both(errors, None)) } case None => - IorT.rightT[F, NonEmptyList[FailureDetails.SchemaViolation]](none[SdjExtractResult]) + IorT.rightT[F, NonEmptyList[SchemaViolationWithExtraContext]](none[SdjExtractResult]) } /** @@ -110,7 +112,7 @@ object IgluUtils { registryLookup: RegistryLookup[F], field: String = "contexts", criterion: SchemaCriterion = SchemaCriterion("com.snowplowanalytics.snowplow", "contexts", "jsonschema", 1, 0) - ): IorT[F, NonEmptyList[FailureDetails.SchemaViolation], List[SdjExtractResult]] = + ): IorT[F, NonEmptyList[SchemaViolationWithExtraContext], List[SdjExtractResult]] = Option(enriched.contexts) match { case Some(rawContexts) => val iorT = for { @@ -122,7 +124,7 @@ object IgluUtils { // Parse and validate each SDJ and merge the errors contextsSdj <- contexts .traverse( - parseAndValidateSDJ(_, client, registryLookup) + parseAndValidateSDJ(_, client, registryLookup, field) .map(sdj => List(sdj)) .recoverWith { case errors => IorT.fromIor[F](Ior.Both(errors, Nil)) } ) @@ -130,7 +132,7 @@ object IgluUtils { } yield contextsSdj iorT.recoverWith { case errors => IorT.fromIor[F](Ior.Both(errors, Nil)) } case None => - IorT.rightT[F, NonEmptyList[FailureDetails.SchemaViolation]](Nil) + IorT.rightT[F, NonEmptyList[SchemaViolationWithExtraContext]](Nil) } /** @@ -146,13 +148,16 @@ object IgluUtils { client: IgluCirceClient[F], sdjs: List[SelfDescribingData[Json]], registryLookup: RegistryLookup[F] - ): IorT[F, NonEmptyList[FailureDetails.SchemaViolation], List[SelfDescribingData[Json]]] = + ): IorT[F, NonEmptyList[SchemaViolationWithExtraContext], List[SelfDescribingData[Json]]] = checkList(client, sdjs, registryLookup) .leftMap( _.map { - case (schemaKey, clientError) => - val f: FailureDetails.SchemaViolation = FailureDetails.SchemaViolation.IgluError(schemaKey, clientError) - f + case (sdj, clientError) => + SchemaViolationWithExtraContext( + schemaViolation = FailureDetails.SchemaViolation.IgluError(sdj.schema, clientError), + source = "derived_contexts", + data = sdj.asString.some + ) } ) @@ -163,34 +168,54 @@ object IgluUtils { expectedCriterion: SchemaCriterion, client: IgluCirceClient[F], registryLookup: RegistryLookup[F] - ): EitherT[F, FailureDetails.SchemaViolation, Json] = + ): EitherT[F, SchemaViolationWithExtraContext, Json] = for { // Parse Json string with the SDJ json <- JsonUtils .extractJson(rawJson) - .leftMap(e => FailureDetails.SchemaViolation.NotJson(field, rawJson.some, e)) + .leftMap(e => + SchemaViolationWithExtraContext( + schemaViolation = FailureDetails.SchemaViolation.NotJson(field, rawJson.some, e), + source = field, + data = s"{$field : $rawJson".some + ) + ) .toEitherT[F] // Parse Json as SelfDescribingData[Json] (which contains the .data that we want) sdj <- SelfDescribingData .parse(json) - .leftMap(FailureDetails.SchemaViolation.NotIglu(json, _)) + .leftMap(e => + SchemaViolationWithExtraContext( + schemaViolation = FailureDetails.SchemaViolation.NotIglu(json, e), + source = field, + data = json.noSpaces.some + ) + ) .toEitherT[F] // Check that the schema of SelfDescribingData[Json] is the expected one _ <- if (validateCriterion(sdj, expectedCriterion)) - EitherT.rightT[F, FailureDetails.SchemaViolation](sdj) + EitherT.rightT[F, SchemaViolationWithExtraContext](sdj) else EitherT .leftT[F, SelfDescribingData[Json]]( - FailureDetails.SchemaViolation.CriterionMismatch(sdj.schema, expectedCriterion) + SchemaViolationWithExtraContext( + schemaViolation = FailureDetails.SchemaViolation.CriterionMismatch(sdj.schema, expectedCriterion), + source = field, + data = sdj.asString.some + ) ) // Check that the SDJ holding the .data is valid _ <- check(client, sdj, registryLookup) .leftMap { case (schemaKey, clientError) => - FailureDetails.SchemaViolation.IgluError(schemaKey, clientError) + SchemaViolationWithExtraContext( + schemaViolation = FailureDetails.SchemaViolation.IgluError(schemaKey, clientError), + source = field, + data = sdj.asString.some + ) } // Extract .data of SelfDescribingData[Json] - data <- EitherT.rightT[F, FailureDetails.SchemaViolation](sdj.data) + data <- EitherT.rightT[F, SchemaViolationWithExtraContext](sdj.data) } yield data /** Check that the schema of a SDJ matches the expected one */ @@ -217,11 +242,11 @@ object IgluUtils { client: IgluCirceClient[F], sdjs: List[SelfDescribingData[Json]], registryLookup: RegistryLookup[F] - ): IorT[F, NonEmptyList[(SchemaKey, ClientError)], List[SelfDescribingData[Json]]] = + ): IorT[F, NonEmptyList[(SelfDescribingData[Json], ClientError)], List[SelfDescribingData[Json]]] = sdjs.map { sdj => check(client, sdj, registryLookup) .map(_ => List(sdj)) - .leftMap(NonEmptyList.one) + .leftMap(e => NonEmptyList.one((sdj, e._2))) .toIor .recoverWith { case errors => IorT.fromIor[F](Ior.Both(errors, Nil)) } }.foldA @@ -230,19 +255,29 @@ object IgluUtils { private def parseAndValidateSDJ[F[_]: Monad: Clock]( json: Json, client: IgluCirceClient[F], - registryLookup: RegistryLookup[F] - ): IorT[F, NonEmptyList[FailureDetails.SchemaViolation], SdjExtractResult] = + registryLookup: RegistryLookup[F], + field: String + ): IorT[F, NonEmptyList[SchemaViolationWithExtraContext], SdjExtractResult] = for { sdj <- IorT .fromEither[F](SelfDescribingData.parse(json)) - .leftMap[FailureDetails.SchemaViolation](FailureDetails.SchemaViolation.NotIglu(json, _)) + .leftMap[SchemaViolationWithExtraContext](e => + SchemaViolationWithExtraContext( + schemaViolation = FailureDetails.SchemaViolation.NotIglu(json, e), + source = field, + data = json.noSpaces.some + ) + ) .leftMap(NonEmptyList.one) supersedingSchema <- check(client, sdj, registryLookup) .leftMap { case (schemaKey, clientError) => - FailureDetails.SchemaViolation - .IgluError(schemaKey, clientError): FailureDetails.SchemaViolation - + SchemaViolationWithExtraContext( + schemaViolation = FailureDetails.SchemaViolation + .IgluError(schemaKey, clientError): FailureDetails.SchemaViolation, + source = field, + data = json.noSpaces.some + ) } .leftMap(NonEmptyList.one) .toIor