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 6bb01a9e0..7b69a4d59 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 @@ -226,7 +226,7 @@ object FailureEntity { 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}", + "message" := s"Invalid schema: ${schemaKey.toSchemaUri} - ${r.message}", "source" := v.source, "path" := r.path ) diff --git a/modules/common/src/test/scala/com.snowplowanalytics.snowplow.enrich.common/SpecHelpers.scala b/modules/common/src/test/scala/com.snowplowanalytics.snowplow.enrich.common/SpecHelpers.scala index b7953076a..c8a0b2fe4 100644 --- a/modules/common/src/test/scala/com.snowplowanalytics.snowplow.enrich.common/SpecHelpers.scala +++ b/modules/common/src/test/scala/com.snowplowanalytics.snowplow.enrich.common/SpecHelpers.scala @@ -139,15 +139,17 @@ object SpecHelpers extends CatsEffect { .flatMap(SelfDescribingData.parse[Json]) .leftMap(err => s"Can't parse Json [$rawJson] as as SelfDescribingData, error: [$err]") - def listContextsSchemas(rawContexts: String): List[SchemaKey] = + def listContexts(rawContexts: String): List[SelfDescribingData[Json]] = jsonStringToSDJ(rawContexts) .map(_.data.asArray.get.toList) - .flatMap(contexts => contexts.traverse(c => SelfDescribingData.parse[Json](c).map(_.schema))) match { + .flatMap(contexts => contexts.traverse(c => SelfDescribingData.parse[Json](c))) match { case Left(err) => throw new IllegalArgumentException(s"Couldn't list contexts schemas. Error: [$err]") - case Right(schemas) => schemas + case Right(sdjs) => sdjs } + def listContextsSchemas(rawContexts: String): List[SchemaKey] = listContexts(rawContexts).map(_.schema) + def getUnstructSchema(rawUnstruct: String): SchemaKey = jsonStringToSDJ(rawUnstruct) .map(_.data) diff --git a/modules/common/src/test/scala/com.snowplowanalytics.snowplow.enrich.common/enrichments/AtomicFieldsSpec.scala b/modules/common/src/test/scala/com.snowplowanalytics.snowplow.enrich.common/enrichments/AtomicFieldsSpec.scala new file mode 100644 index 000000000..590004f50 --- /dev/null +++ b/modules/common/src/test/scala/com.snowplowanalytics.snowplow.enrich.common/enrichments/AtomicFieldsSpec.scala @@ -0,0 +1,53 @@ +/* + * Copyright (c) 2022-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 cats.data.NonEmptyList +import cats.syntax.option._ + +import io.circe.Json +import io.circe.syntax._ + +import com.snowplowanalytics.iglu.client.ClientError.ValidationError +import com.snowplowanalytics.iglu.client.validator.{ValidatorError, ValidatorReport} +import com.snowplowanalytics.snowplow.badrows.FailureDetails + +import org.specs2.mutable.Specification + +class AtomicFieldsSpec extends Specification { + + "errorsToSchemaViolation" should { + "convert ValidatorReports to SchemaViolation correctly" >> { + val vrList = NonEmptyList( + ValidatorReport(message = "testMessage", path = "testPath1".some, targets = List("t1, t2"), keyword = "testKeyword1".some), + List( + ValidatorReport(message = "testMessage", path = None, targets = List.empty, keyword = "testKeyword2".some), + ValidatorReport(message = "testMessage", path = "testPath3".some, targets = List("t1", "t2"), keyword = None), + ValidatorReport(message = "testMessage", path = "testPath4".some, targets = List.empty, keyword = "testKeyword4".some) + ) + ) + val expected = FailureEntity.SchemaViolation( + schemaViolation = FailureDetails.SchemaViolation.IgluError( + schemaKey = AtomicFields.atomicSchema, + error = ValidationError(ValidatorError.InvalidData(vrList), None) + ), + source = "atomic_field", + data = Json.obj( + "testPath1" := "testKeyword1", + "testPath3" := Json.Null, + "testPath4" := "testKeyword4" + ) + ) + val result = AtomicFields.errorsToSchemaViolation(vrList) + result must beEqualTo(expected) + } + } +} diff --git a/modules/common/src/test/scala/com.snowplowanalytics.snowplow.enrich.common/enrichments/EnrichmentManagerSpec.scala b/modules/common/src/test/scala/com.snowplowanalytics.snowplow.enrich.common/enrichments/EnrichmentManagerSpec.scala index fd1aff96c..73eea3bf6 100644 --- a/modules/common/src/test/scala/com.snowplowanalytics.snowplow.enrich.common/enrichments/EnrichmentManagerSpec.scala +++ b/modules/common/src/test/scala/com.snowplowanalytics.snowplow.enrich.common/enrichments/EnrichmentManagerSpec.scala @@ -31,6 +31,7 @@ import org.joda.time.DateTime import com.snowplowanalytics.snowplow.badrows._ import com.snowplowanalytics.iglu.core.{SchemaCriterion, SchemaKey, SchemaVer, SelfDescribingData} +import com.snowplowanalytics.iglu.core.circe.implicits._ import com.snowplowanalytics.snowplow.enrich.common.QueryStringParameters import com.snowplowanalytics.snowplow.enrich.common.loaders._ @@ -1237,6 +1238,15 @@ class EnrichmentManagerSpec extends Specification with EitherMatchers with CatsE JavascriptScriptEnrichment(schemaKey, script) ) ) + val invalidUe = + """{ + "schema":"iglu:com.acme/email_sent/jsonschema/1-0-0", + "data": { + "emailAddress": "hello@world.com", + "emailAddress2": "foo@bar.org", + "unallowedAdditionalField": "foo@bar.org" + } + }""" val parameters = Map( "e" -> "pp", @@ -1252,20 +1262,34 @@ class EnrichmentManagerSpec extends Specification with EitherMatchers with CatsE } """, "ue_pr" -> - """ + s""" { "schema":"iglu:com.snowplowanalytics.snowplow/unstruct_event/jsonschema/1-0-0", - "data":{ - "schema":"iglu:com.acme/email_sent/jsonschema/1-0-0", - "data": { - "emailAddress": "hello@world.com", - "emailAddress2": "foo@bar.org", - "unallowedAdditionalField": "foo@bar.org" - } - } + "data":$invalidUe }""" ).toOpt val rawEvent = RawEvent(api, parameters, None, source, context) + def expectedDerivedContexts(enriched: EnrichedEvent): Boolean = { + val emailSentSDJ = SelfDescribingData.parse[Json](jparse(emailSent).toOption.get).toOption.get + SpecHelpers.listContexts(enriched.derived_contexts) match { + case List(`emailSentSDJ`, SelfDescribingData(FailureEntity.`failureEntitySchemaKey`, feJson)) + if feJson.field("failureType") == "ValidationError".asJson && + feJson.field("errors") == Json.arr( + Json.obj( + "message" := "$.unallowedAdditionalField: is not defined in the schema and the schema does not allow additional properties", + "source" := "ue_properties", + "path" := "$", + "keyword" := "additionalProperties", + "targets" := List("unallowedAdditionalField") + ) + ) && + feJson.field("schema") == emailSentSchema.asJson && + feJson.field("data") == jparse(invalidUe).toOption.get => + true + case _ => false + } + } + val enriched = EnrichmentManager.enrichEvent[IO]( enrichmentReg, client, @@ -1278,11 +1302,12 @@ class EnrichmentManagerSpec extends Specification with EitherMatchers with CatsE atomicFieldLimits, emitIncomplete = true ) + enriched.value.map { case Ior.Both(_: BadRow.SchemaViolations, enriched) if Option(enriched.unstruct_event).isEmpty && SpecHelpers.listContextsSchemas(enriched.contexts) == List(clientSessionSchema) && - SpecHelpers.listContextsSchemas(enriched.derived_contexts).contains(emailSentSchema) => + expectedDerivedContexts(enriched) => ok case other => ko(s"[$other] is not a SchemaViolations bad row and an enriched event without the unstructured event") } @@ -1305,24 +1330,24 @@ class EnrichmentManagerSpec extends Specification with EitherMatchers with CatsE JavascriptScriptEnrichment(schemaKey, script) ) ) + val invalidContext = + """{ + "schema":"iglu:com.acme/email_sent/jsonschema/1-0-0", + "data": { + "foo": "hello@world.com", + "emailAddress2": "foo@bar.org" + } + }""" val parameters = Map( "e" -> "pp", "tv" -> "js-0.13.1", "p" -> "web", "co" -> - """ + s""" { "schema": "iglu:com.snowplowanalytics.snowplow/contexts/jsonschema/1-0-0", - "data": [ - { - "schema":"iglu:com.acme/email_sent/jsonschema/1-0-0", - "data": { - "foo": "hello@world.com", - "emailAddress2": "foo@bar.org" - } - } - ] + "data": [$invalidContext] } """, "ue_pr" -> @@ -1333,6 +1358,34 @@ class EnrichmentManagerSpec extends Specification with EitherMatchers with CatsE }""" ).toOpt val rawEvent = RawEvent(api, parameters, None, source, context) + def expectedDerivedContexts(enriched: EnrichedEvent): Boolean = { + val emailSentSDJ = SelfDescribingData.parse[Json](jparse(emailSent).toOption.get).toOption.get + SpecHelpers.listContexts(enriched.derived_contexts) match { + case List(`emailSentSDJ`, SelfDescribingData(FailureEntity.`failureEntitySchemaKey`, feJson)) + if feJson.field("failureType") == "ValidationError".asJson && + feJson.field("errors") == Json.arr( + Json.obj( + "message" := "$.emailAddress: is missing but it is required", + "source" := "contexts", + "path" := "$", + "keyword" := "required", + "targets" := List("emailAddress") + ), + Json.obj( + "message" := "$.foo: is not defined in the schema and the schema does not allow additional properties", + "source" := "contexts", + "path" := "$", + "keyword" := "additionalProperties", + "targets" := List("foo") + ) + ) && + feJson.field("schema") == emailSentSchema.asJson && + feJson.field("data") == jparse(invalidContext).toOption.get => + true + case _ => false + } + } + val enriched = EnrichmentManager.enrichEvent[IO]( enrichmentReg, client, @@ -1349,7 +1402,7 @@ class EnrichmentManagerSpec extends Specification with EitherMatchers with CatsE case Ior.Both(_: BadRow.SchemaViolations, enriched) if Option(enriched.contexts).isEmpty && SpecHelpers.getUnstructSchema(enriched.unstruct_event) == clientSessionSchema && - SpecHelpers.listContextsSchemas(enriched.derived_contexts).contains(emailSentSchema) => + expectedDerivedContexts(enriched) => ok case other => ko(s"[$other] is not a SchemaViolations bad row and an enriched event with no input contexts") } @@ -1372,6 +1425,15 @@ class EnrichmentManagerSpec extends Specification with EitherMatchers with CatsE JavascriptScriptEnrichment(schemaKey, script) ) ) + val invalidContext = + """ + { + "schema":"iglu:com.acme/email_sent/jsonschema/1-0-0", + "data": { + "foo": "hello@world.com", + "emailAddress2": "foo@bar.org" + } + }""" val parameters = Map( "e" -> "pp", @@ -1382,13 +1444,7 @@ class EnrichmentManagerSpec extends Specification with EitherMatchers with CatsE { "schema": "iglu:com.snowplowanalytics.snowplow/contexts/jsonschema/1-0-0", "data": [ - { - "schema":"iglu:com.acme/email_sent/jsonschema/1-0-0", - "data": { - "foo": "hello@world.com", - "emailAddress2": "foo@bar.org" - } - }, + $invalidContext, $clientSession ] } @@ -1401,6 +1457,34 @@ class EnrichmentManagerSpec extends Specification with EitherMatchers with CatsE }""" ).toOpt val rawEvent = RawEvent(api, parameters, None, source, context) + def expectedDerivedContexts(enriched: EnrichedEvent): Boolean = { + val emailSentSDJ = SelfDescribingData.parse[Json](jparse(emailSent).toOption.get).toOption.get + SpecHelpers.listContexts(enriched.derived_contexts) match { + case List(`emailSentSDJ`, SelfDescribingData(FailureEntity.`failureEntitySchemaKey`, feJson)) + if feJson.field("failureType") == "ValidationError".asJson && + feJson.field("errors") == Json.arr( + Json.obj( + "message" := "$.emailAddress: is missing but it is required", + "source" := "contexts", + "path" := "$", + "keyword" := "required", + "targets" := List("emailAddress") + ), + Json.obj( + "message" := "$.foo: is not defined in the schema and the schema does not allow additional properties", + "source" := "contexts", + "path" := "$", + "keyword" := "additionalProperties", + "targets" := List("foo") + ) + ) && + feJson.field("schema") == emailSentSchema.asJson && + feJson.field("data") == jparse(invalidContext).toOption.get => + true + case _ => false + } + } + val enriched = EnrichmentManager.enrichEvent[IO]( enrichmentReg, client, @@ -1417,7 +1501,7 @@ class EnrichmentManagerSpec extends Specification with EitherMatchers with CatsE case Ior.Both(_: BadRow.SchemaViolations, enriched) if SpecHelpers.getUnstructSchema(enriched.unstruct_event) == clientSessionSchema && SpecHelpers.listContextsSchemas(enriched.contexts) == List(clientSessionSchema) && - SpecHelpers.listContextsSchemas(enriched.derived_contexts).contains(emailSentSchema) => + expectedDerivedContexts(enriched) => ok case other => ko(s"[$other] is not a SchemaViolations bad row and an enriched event with 1 input context") } @@ -1455,6 +1539,24 @@ class EnrichmentManagerSpec extends Specification with EitherMatchers with CatsE }""" ).toOpt val rawEvent = RawEvent(api, parameters, None, source, context) + def expectedDerivedContexts(enriched: EnrichedEvent): Boolean = + SpecHelpers.listContexts(enriched.derived_contexts) match { + case List( + SelfDescribingData(SchemaKey("nl.basjes", "yauaa_context", "jsonschema", _), _), + SelfDescribingData(FailureEntity.`failureEntitySchemaKey`, feJson) + ) + if feJson.field("failureType") == "EnrichmentError: Javascript enrichment".asJson && + feJson.field("errors") == Json.arr( + Json.obj( + "message" := "Error during execution of JavaScript function: [Javascript exception in at line number 3 at column number 10]" + ) + ) && + feJson.field("schema") == JavascriptScriptEnrichment.supportedSchema.copy(addition = 0.some).asString.asJson && + feJson.field("data") == Json.Null => + true + case _ => false + } + val enriched = EnrichmentManager.enrichEvent[IO]( enrichmentReg, client, @@ -1470,7 +1572,7 @@ class EnrichmentManagerSpec extends Specification with EitherMatchers with CatsE enriched.value.map { case Ior.Both(_: BadRow.EnrichmentFailures, enriched) if SpecHelpers.getUnstructSchema(enriched.unstruct_event) == clientSessionSchema && - !SpecHelpers.listContextsSchemas(enriched.derived_contexts).contains(emailSentSchema) => + expectedDerivedContexts(enriched) => ok case other => ko(s"[$other] is not an EnrichmentFailures bad row and an enriched event") } @@ -1508,6 +1610,36 @@ class EnrichmentManagerSpec extends Specification with EitherMatchers with CatsE }""" ).toOpt val rawEvent = RawEvent(api, parameters, None, source, context) + def expectedDerivedContexts(enriched: EnrichedEvent): Boolean = + SpecHelpers.listContexts(enriched.derived_contexts) match { + case List( + SelfDescribingData(FailureEntity.`failureEntitySchemaKey`, validationError), + SelfDescribingData(FailureEntity.`failureEntitySchemaKey`, enrichmentError) + ) + if validationError.field("failureType") == "ValidationError".asJson && + validationError.field("errors") == Json.arr( + Json.obj( + "message" := "Cannot be converted to java.math.BigDecimal. Error : Character f is neither a decimal digit number, decimal point, nor \"e\" notation exponential mark.", + "source" := "tr_tt", + "path" := "tr_tt", + "keyword" := "foo", + "targets" := Json.arr() + ) + ) && + validationError.field("schema") == AtomicFields.atomicSchema.asJson && + validationError.field("data") == Json.obj("tr_tt" := "foo") && + enrichmentError.field("failureType") == "EnrichmentError: Javascript enrichment".asJson && + enrichmentError.field("errors") == Json.arr( + Json.obj( + "message" := "Error during execution of JavaScript function: [Javascript exception in at line number 3 at column number 10]" + ) + ) && + enrichmentError.field("schema") == JavascriptScriptEnrichment.supportedSchema.copy(addition = 0.some).asString.asJson && + enrichmentError.field("data") == Json.Null => + true + case _ => false + } + val enriched = EnrichmentManager.enrichEvent[IO]( enrichmentReg, client, @@ -1521,23 +1653,26 @@ class EnrichmentManagerSpec extends Specification with EitherMatchers with CatsE emitIncomplete = true ) enriched.value.map { - case Ior.Both(_: BadRow.SchemaViolations, _) => ok + case Ior.Both(_: BadRow.SchemaViolations, enriched) if expectedDerivedContexts(enriched) => ok case other => ko(s"[$other] doesn't have a SchemaViolations bad row in the Left") } } "remove an invalid enrichment context and return the enriched event if emitIncomplete is set to true" >> { + val invalidContext = + """ + { + "schema":"iglu:com.acme/email_sent/jsonschema/1-0-0", + "data": { + "foo": "hello@world.com", + "emailAddress2": "foo@bar.org" + } + }""" val script = s""" function process(event) { return [ - { - "schema":"iglu:com.acme/email_sent/jsonschema/1-0-0", - "data": { - "foo": "hello@world.com", - "emailAddress2": "foo@bar.org" - } - } + $invalidContext ]; }""" val schemaKey = SchemaKey( @@ -1565,6 +1700,35 @@ class EnrichmentManagerSpec extends Specification with EitherMatchers with CatsE }""" ).toOpt val rawEvent = RawEvent(api, parameters, None, source, context) + def expectedDerivedContexts(enriched: EnrichedEvent): Boolean = + SpecHelpers.listContexts(enriched.derived_contexts) match { + case List( + SelfDescribingData(SchemaKey("nl.basjes", "yauaa_context", "jsonschema", _), _), + SelfDescribingData(FailureEntity.`failureEntitySchemaKey`, feJson) + ) + if feJson.field("failureType") == "ValidationError".asJson && + feJson.field("errors") == Json.arr( + Json.obj( + "message" := "$.emailAddress: is missing but it is required", + "source" := "derived_contexts", + "path" := "$", + "keyword" := "required", + "targets" := List("emailAddress") + ), + Json.obj( + "message" := "$.foo: is not defined in the schema and the schema does not allow additional properties", + "source" := "derived_contexts", + "path" := "$", + "keyword" := "additionalProperties", + "targets" := List("foo") + ) + ) && + feJson.field("schema") == emailSentSchema.asJson && + feJson.field("data") == jparse(invalidContext).toOption.get => + true + case _ => false + } + val enriched = EnrichmentManager.enrichEvent[IO]( enrichmentReg, client, @@ -1580,7 +1744,7 @@ class EnrichmentManagerSpec extends Specification with EitherMatchers with CatsE enriched.value.map { case Ior.Both(_: BadRow.SchemaViolations, enriched) if SpecHelpers.getUnstructSchema(enriched.unstruct_event) == clientSessionSchema && - !SpecHelpers.listContextsSchemas(enriched.derived_contexts).contains(emailSentSchema) => + expectedDerivedContexts(enriched) => ok case other => ko(s"[$other] is not a SchemaViolations bad row and an enriched event without the faulty enrichment context") } @@ -2267,4 +2431,9 @@ object EnrichmentManagerSpec { "userId": "20d631b8-7837-49df-a73e-6da73154e6fd" } }""" + + implicit class JsonFieldGetter(json: Json) { + def field(f: String): Json = + json.hcursor.downField(f).as[Json].toOption.get + } } diff --git a/modules/common/src/test/scala/com.snowplowanalytics.snowplow.enrich.common/enrichments/FailureEntitySpec.scala b/modules/common/src/test/scala/com.snowplowanalytics.snowplow.enrich.common/enrichments/FailureEntitySpec.scala new file mode 100644 index 000000000..a83fecebc --- /dev/null +++ b/modules/common/src/test/scala/com.snowplowanalytics.snowplow.enrich.common/enrichments/FailureEntitySpec.scala @@ -0,0 +1,392 @@ +/* + * Copyright (c) 2014-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 scala.collection.immutable.SortedMap + +import cats.effect.testing.specs2.CatsEffect +import cats.effect.unsafe.implicits.global +import cats.effect.IO + +import cats.data.NonEmptyList +import cats.syntax.option._ + +import io.circe.syntax._ +import io.circe.Json + +import org.specs2.mutable.Specification +import org.specs2.matcher.ValidatedMatchers +import org.specs2.ScalaCheck + +import org.scalacheck.{Gen, Prop} + +import com.snowplowanalytics.snowplow.enrich.common.SpecHelpers +import com.snowplowanalytics.snowplow.badrows.{FailureDetails, Processor} +import com.snowplowanalytics.iglu.core.{ParseError, SchemaCriterion, SchemaKey, SchemaVer, SelfDescribingData} +import com.snowplowanalytics.iglu.client.ClientError +import com.snowplowanalytics.iglu.client.validator.{ValidatorError, ValidatorReport} +import com.snowplowanalytics.iglu.client.resolver.LookupHistory +import com.snowplowanalytics.iglu.client.resolver.registries.RegistryLookup + +class FailureEntitySpec extends Specification with ValidatedMatchers with CatsEffect with ScalaCheck { + + val timestamp = Instant.now() + val processor = Processor("unit tests SCE", "v42") + val schemaKey = SchemaKey("com.snowplowanalytics", "test", "jsonschema", SchemaVer.Full(1, 0, 0)) + val schemaCriterion = SchemaCriterion.apply("com.snowplowanalytics", "test", "jsonschema", 1) + + "FailureEntityContext should be valid against its schema" >> { + implicit val registryLookup: RegistryLookup[IO] = SpecHelpers.registryLookup + + val genFeContext = for { + failureType <- Gen.alphaNumStr + jsonGen = Gen.oneOf( + Json.obj(), + Json.obj("test1" := "value1"), + Json.obj("test1" := "value1", "test2" := "value2"), + Json.obj("test1" := "value1", "test2" := "value2", "test3" := "value3") + ) + errors <- Gen.listOf(jsonGen) + data <- Gen.option(jsonGen) + schema <- Gen.option(Gen.const(schemaKey)) + } yield FailureEntity.FailureEntityContext( + failureType = failureType, + errors = errors, + schema = schema, + data = data, + timestamp = timestamp, + componentName = processor.artifact, + componentVersion = processor.version + ) + + Prop.forAll(genFeContext) { feContext: FailureEntity.FailureEntityContext => + val sdj = SelfDescribingData(schema = FailureEntity.failureEntitySchemaKey, data = feContext.asJson) + SpecHelpers.client + .check(sdj) + .value + .map(_ must beRight) + .unsafeRunSync() + } + } + + "fromEnrichmentFailure" should { + "convert InputData correctly" >> { + val ef = FailureEntity.EnrichmentFailure( + enrichmentFailure = FailureDetails.EnrichmentFailure( + enrichment = FailureDetails + .EnrichmentInformation( + schemaKey = schemaKey, + identifier = "enrichmentId" + ) + .some, + message = FailureDetails.EnrichmentFailureMessage.InputData( + field = "testField", + value = "testValue".some, + expectation = "testExpectation" + ) + ) + ) + val result = FailureEntity.fromEnrichmentFailure(ef, timestamp, processor) + val expected = FailureEntity.FailureEntityContext( + failureType = "EnrichmentError: enrichmentId", + errors = List( + Json.obj( + "message" := "testField - testExpectation", + "source" := "testField" + ) + ), + schema = schemaKey.some, + data = Json.obj("testField" := "testValue").some, + timestamp = timestamp, + componentName = processor.artifact, + componentVersion = processor.version + ) + result must beEqualTo(expected) + } + + "convert Simple correctly" >> { + val ef = FailureEntity.EnrichmentFailure( + enrichmentFailure = FailureDetails.EnrichmentFailure( + enrichment = FailureDetails + .EnrichmentInformation( + schemaKey = schemaKey, + identifier = "enrichmentId" + ) + .some, + message = FailureDetails.EnrichmentFailureMessage.Simple(error = "testError") + ) + ) + val result = FailureEntity.fromEnrichmentFailure(ef, timestamp, processor) + val expected = FailureEntity.FailureEntityContext( + failureType = "EnrichmentError: enrichmentId", + errors = List(Json.obj("message" := "testError")), + schema = schemaKey.some, + data = None, + timestamp = timestamp, + componentName = processor.artifact, + componentVersion = processor.version + ) + result must beEqualTo(expected) + } + } + + "fromSchemaViolation" should { + "convert NotJson correctly" >> { + val sv = FailureEntity.SchemaViolation( + schemaViolation = FailureDetails.SchemaViolation.NotJson( + field = "testField", + value = "testValue".some, + error = "testError" + ), + source = "testSource", + data = "testData".asJson + ) + val fe = FailureEntity.fromSchemaViolation(sv, timestamp, processor) + val expected = FailureEntity.FailureEntityContext( + failureType = "NotJSON", + errors = List( + Json.obj( + "message" := "testError", + "source" := "testSource" + ) + ), + schema = None, + data = Json.obj("testSource" := "testData").some, + timestamp = timestamp, + componentName = processor.artifact, + componentVersion = processor.version + ) + fe must beEqualTo(expected) + } + + "convert NotIglu correctly" >> { + val sv = FailureEntity.SchemaViolation( + schemaViolation = FailureDetails.SchemaViolation.NotIglu( + json = Json.Null, + error = ParseError.InvalidSchema + ), + source = "testSource", + data = "testData".asJson + ) + val fe = FailureEntity.fromSchemaViolation(sv, timestamp, processor) + val expected = FailureEntity.FailureEntityContext( + failureType = "NotIglu", + errors = List( + Json.obj( + "message" := "Invalid schema", + "source" := "testSource" + ) + ), + schema = None, + data = "testData".asJson.some, + timestamp = timestamp, + componentName = processor.artifact, + componentVersion = processor.version + ) + fe must beEqualTo(expected) + } + + "convert CriterionMismatch correctly" >> { + val sv = FailureEntity.SchemaViolation( + schemaViolation = FailureDetails.SchemaViolation.CriterionMismatch( + schemaKey = schemaKey, + schemaCriterion = schemaCriterion + ), + source = "testSource", + data = "testData".asJson + ) + val fe = FailureEntity.fromSchemaViolation(sv, timestamp, processor) + val expected = FailureEntity.FailureEntityContext( + failureType = "CriterionMismatch", + errors = List( + Json.obj( + "message" := "Unexpected schema: iglu:com.snowplowanalytics/test/jsonschema/1-0-0 does not match the criterion", + "source" := "testSource", + "criterion" := "iglu:com.snowplowanalytics/test/jsonschema/1-*-*" + ) + ), + schema = schemaKey.some, + data = "testData".asJson.some, + timestamp = timestamp, + componentName = processor.artifact, + componentVersion = processor.version + ) + fe must beEqualTo(expected) + } + + "convert ResolutionError correctly" >> { + val sv = FailureEntity.SchemaViolation( + schemaViolation = FailureDetails.SchemaViolation.IgluError( + schemaKey = schemaKey, + error = ClientError.ResolutionError( + value = SortedMap( + "repo1" -> LookupHistory( + errors = Set.empty, + attempts = 1, + lastAttempt = timestamp + ), + "repo2" -> LookupHistory( + errors = Set.empty, + attempts = 2, + lastAttempt = timestamp + ) + ) + ) + ), + source = "testSource", + data = "testData".asJson + ) + val fe = FailureEntity.fromSchemaViolation(sv, timestamp, processor) + val expected = FailureEntity.FailureEntityContext( + failureType = "ResolutionError", + errors = List( + Json.obj( + "message" := "Resolution error: schema iglu:com.snowplowanalytics/test/jsonschema/1-0-0 not found", + "source" := "testSource", + "lookupHistory" := Json.arr( + Json.obj("repository" := "repo1", "errors" := List.empty[String], "attempts" := 1, "lastAttempt" := timestamp), + Json.obj("repository" := "repo2", "errors" := List.empty[String], "attempts" := 2, "lastAttempt" := timestamp) + ) + ) + ), + schema = schemaKey.some, + data = "testData".asJson.some, + timestamp = timestamp, + componentName = processor.artifact, + componentVersion = processor.version + ) + fe must beEqualTo(expected) + } + + "convert InvalidData correctly" >> { + def createSv(schemaKey: SchemaKey) = + FailureEntity.SchemaViolation( + schemaViolation = FailureDetails.SchemaViolation.IgluError( + schemaKey = schemaKey, + error = ClientError.ValidationError( + error = ValidatorError.InvalidData( + messages = NonEmptyList.of( + ValidatorReport(message = "testMessage1", + path = "testPath1".some, + targets = List("testTarget1"), + keyword = "testKeyword1".some + ), + ValidatorReport(message = "testMessage2", + path = "testPath2".some, + targets = List("testTarget2"), + keyword = "testKeyword2".some + ) + ) + ), + supersededBy = None + ) + ), + source = "testSource", + data = "testData".asJson + ) + + val svWithAtomicSchema = createSv(AtomicFields.atomicSchema) + val svWithOrdinarySchema = createSv(schemaKey) + val feWithAtomicSchema = FailureEntity.fromSchemaViolation(svWithAtomicSchema, timestamp, processor) + val feWithOrdinarySchema = FailureEntity.fromSchemaViolation(svWithOrdinarySchema, timestamp, processor) + val expectedWithAtomicSchema = FailureEntity.FailureEntityContext( + failureType = "ValidationError", + errors = List( + Json.obj("message" := "testMessage1", + "source" := "testPath1", + "path" := "testPath1", + "keyword" := "testKeyword1", + "targets" := List("testTarget1") + ), + Json.obj("message" := "testMessage2", + "source" := "testPath2", + "path" := "testPath2", + "keyword" := "testKeyword2", + "targets" := List("testTarget2") + ) + ), + schema = AtomicFields.atomicSchema.some, + data = "testData".asJson.some, + timestamp = timestamp, + componentName = processor.artifact, + componentVersion = processor.version + ) + val expectedWithOrdinarySchema = FailureEntity.FailureEntityContext( + failureType = "ValidationError", + errors = List( + Json.obj("message" := "testMessage1", + "source" := "testSource", + "path" := "testPath1", + "keyword" := "testKeyword1", + "targets" := List("testTarget1") + ), + Json.obj("message" := "testMessage2", + "source" := "testSource", + "path" := "testPath2", + "keyword" := "testKeyword2", + "targets" := List("testTarget2") + ) + ), + schema = schemaKey.some, + data = "testData".asJson.some, + timestamp = timestamp, + componentName = processor.artifact, + componentVersion = processor.version + ) + + feWithAtomicSchema must beEqualTo(expectedWithAtomicSchema) + feWithOrdinarySchema must beEqualTo(expectedWithOrdinarySchema) + } + + "convert InvalidSchema correctly" >> { + val sv = FailureEntity.SchemaViolation( + schemaViolation = FailureDetails.SchemaViolation.IgluError( + schemaKey = schemaKey, + error = ClientError.ValidationError( + error = ValidatorError.InvalidSchema( + issues = NonEmptyList.of( + ValidatorError.SchemaIssue(path = "testPath1", message = "testMessage1"), + ValidatorError.SchemaIssue(path = "testPath2", message = "testMessage2") + ) + ), + supersededBy = None + ) + ), + source = "testSource", + data = "testData".asJson + ) + val fe = FailureEntity.fromSchemaViolation(sv, timestamp, processor) + val expected = FailureEntity.FailureEntityContext( + failureType = "ValidationError", + errors = List( + Json.obj( + "message" := "Invalid schema: iglu:com.snowplowanalytics/test/jsonschema/1-0-0 - testMessage1", + "source" := "testSource", + "path" := "testPath1" + ), + Json.obj( + "message" := "Invalid schema: iglu:com.snowplowanalytics/test/jsonschema/1-0-0 - testMessage2", + "source" := "testSource", + "path" := "testPath2" + ) + ), + schema = schemaKey.some, + data = "testData".asJson.some, + timestamp = timestamp, + componentName = processor.artifact, + componentVersion = processor.version + ) + fe must beEqualTo(expected) + } + } +} 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 44f14f407..ed595cda4 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.Json import io.circe.syntax._ import cats.data.{Ior, NonEmptyList} @@ -45,8 +46,9 @@ 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 uePropertiesFieldName = "ue_properties" + val contextsFieldName = "contexts" + val derivedContextsFieldName = "derived_contexts" val notJson = "foo" val jsonNotJson = notJson.asJson // Just jsonized version of the string @@ -158,7 +160,10 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect .value .map { case Ior.Both( - NonEmptyList(FailureEntity.SchemaViolation(_: FailureDetails.SchemaViolation.NotJson, `ueProperties`, `jsonNotJson`), _), + NonEmptyList( + FailureEntity.SchemaViolation(_: FailureDetails.SchemaViolation.NotJson, `uePropertiesFieldName`, `jsonNotJson`), + _ + ), None ) => ok @@ -167,7 +172,7 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect } "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 json = notIglu.toJson val input = new EnrichedEvent input.setUnstruct_event(notIglu) @@ -175,8 +180,9 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect .extractAndValidateUnstructEvent(input, SpecHelpers.client, SpecHelpers.registryLookup) .value .map { - case Ior.Both(NonEmptyList(FailureEntity.SchemaViolation(_: FailureDetails.SchemaViolation.NotIglu, `ueProperties`, `json`), _), - None + case Ior.Both( + NonEmptyList(FailureEntity.SchemaViolation(_: FailureDetails.SchemaViolation.NotIglu, `uePropertiesFieldName`, `json`), _), + None ) => ok case other => ko(s"[$other] is not an error with NotIglu") @@ -184,7 +190,7 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect } "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 json = noSchema.toJson val input = new EnrichedEvent input.setUnstruct_event(noSchema) @@ -193,7 +199,10 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect .value .map { case Ior.Both( - NonEmptyList(FailureEntity.SchemaViolation(_: FailureDetails.SchemaViolation.CriterionMismatch, `ueProperties`, `json`), _), + NonEmptyList( + FailureEntity.SchemaViolation(_: FailureDetails.SchemaViolation.CriterionMismatch, `uePropertiesFieldName`, `json`), + _ + ), None ) => ok @@ -211,7 +220,10 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect .extractAndValidateUnstructEvent(input, SpecHelpers.client, SpecHelpers.registryLookup) .value .map { - case Ior.Both(NonEmptyList(FailureEntity.SchemaViolation(_: FailureDetails.SchemaViolation.NotJson, `ueProperties`, `ueJson`), _), + case Ior.Both(NonEmptyList( + FailureEntity.SchemaViolation(_: FailureDetails.SchemaViolation.NotJson, `uePropertiesFieldName`, `ueJson`), + _ + ), None ) => ok @@ -222,7 +234,7 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect "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 + val ueJson = ue.toJson input.setUnstruct_event(ue) IgluUtils @@ -230,7 +242,7 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect .value .map { case Ior.Both(NonEmptyList(FailureEntity.SchemaViolation(FailureDetails.SchemaViolation.IgluError(_, _: ValidationError), - `ueProperties`, + `uePropertiesFieldName`, `ueJson` ), _ @@ -244,7 +256,7 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect "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 + val json = invalidEmailSent.toJson input.setUnstruct_event(buildUnstruct(invalidEmailSent)) IgluUtils @@ -252,7 +264,7 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect .value .map { case Ior.Both(NonEmptyList(FailureEntity.SchemaViolation(FailureDetails.SchemaViolation.IgluError(_, _: ValidationError), - `ueProperties`, + `uePropertiesFieldName`, `json` ), _ @@ -266,6 +278,7 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect "return a FailureDetails.SchemaViolation.IgluError containing a ResolutionError if the schema of the SDJ in .data can't be resolved" >> { val input = new EnrichedEvent + val json = noSchema.toJson input.setUnstruct_event(buildUnstruct(noSchema)) IgluUtils @@ -273,13 +286,16 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect .value .map { case Ior.Both( - NonEmptyList(FailureEntity.SchemaViolation(FailureDetails.SchemaViolation.IgluError(_, _: ResolutionError), _, _), _), + NonEmptyList(FailureEntity.SchemaViolation(FailureDetails.SchemaViolation.IgluError(_, _: ResolutionError), + `uePropertiesFieldName`, + `json` + ), + _ + ), None ) => ok - 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") + case other => ko(s"[$other] is not expected one") } } @@ -360,7 +376,9 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect .value .map { case Ior.Both( - NonEmptyList(FailureEntity.SchemaViolation(_: FailureDetails.SchemaViolation.NotJson, `contexts`, `jsonNotJson`), Nil), + NonEmptyList(FailureEntity.SchemaViolation(_: FailureDetails.SchemaViolation.NotJson, `contextsFieldName`, `jsonNotJson`), + Nil + ), Nil ) => ok @@ -370,26 +388,37 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect "return a FailureDetails.SchemaViolation.NotIglu if .contexts contains a properly formatted JSON string that is not self-describing" >> { val input = new EnrichedEvent + val json = notIglu.toJson input.setContexts(notIglu) IgluUtils .extractAndValidateInputContexts(input, SpecHelpers.client, SpecHelpers.registryLookup) .value .map { - case Ior.Both(NonEmptyList(FailureEntity.SchemaViolation(_: FailureDetails.SchemaViolation.NotIglu, _, _), Nil), Nil) => ok + case Ior.Both( + NonEmptyList(FailureEntity.SchemaViolation(_: FailureDetails.SchemaViolation.NotIglu, `contextsFieldName`, `json`), Nil), + Nil + ) => + ok case other => ko(s"[$other] is not an error with NotIglu") } } "return a FailureDetails.SchemaViolation.CriterionMismatch if .contexts contains a self-describing JSON but not with the right schema" >> { val input = new EnrichedEvent + val json = noSchema.toJson input.setContexts(noSchema) IgluUtils .extractAndValidateInputContexts(input, SpecHelpers.client, SpecHelpers.registryLookup) .value .map { - case Ior.Both(NonEmptyList(FailureEntity.SchemaViolation(_: FailureDetails.SchemaViolation.CriterionMismatch, _, _), Nil), Nil) => + case Ior.Both(NonEmptyList( + FailureEntity.SchemaViolation(_: FailureDetails.SchemaViolation.CriterionMismatch, `contextsFieldName`, `json`), + Nil + ), + Nil + ) => ok case other => ko(s"[$other] is not an error with CriterionMismatch") } @@ -399,6 +428,7 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect val input = new EnrichedEvent val notArrayContexts = s"""{"schema": "${inputContextsSchema.toSchemaUri}", "data": ${emailSent1}}""" + val json = notArrayContexts.toJson input.setContexts(notArrayContexts) IgluUtils @@ -406,18 +436,22 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect .value .map { case Ior.Both( - NonEmptyList(FailureEntity.SchemaViolation(FailureDetails.SchemaViolation.IgluError(_, _: ValidationError), _, _), Nil), + NonEmptyList(FailureEntity.SchemaViolation(FailureDetails.SchemaViolation.IgluError(_, _: ValidationError), + `contextsFieldName`, + `json` + ), + Nil + ), Nil ) => ok - 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") + case other => ko(s"[$other] is not expected one") } } "return a FailureDetails.SchemaViolation.IgluError containing a ValidationError if .data contains one invalid context" >> { val input = new EnrichedEvent + val json = invalidEmailSent.toJson input.setContexts(buildInputContexts(List(invalidEmailSent))) IgluUtils @@ -425,18 +459,22 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect .value .map { case Ior.Both( - NonEmptyList(FailureEntity.SchemaViolation(FailureDetails.SchemaViolation.IgluError(_, _: ValidationError), _, _), Nil), + NonEmptyList(FailureEntity.SchemaViolation(FailureDetails.SchemaViolation.IgluError(_, _: ValidationError), + `contextsFieldName`, + `json` + ), + Nil + ), Nil ) => ok - 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") + case other => ko(s"[$other] is not expected one") } } "return a FailureDetails.SchemaViolation.IgluError containing a ResolutionError if .data contains one context whose schema can't be resolved" >> { val input = new EnrichedEvent + val json = noSchema.toJson input.setContexts(buildInputContexts(List(noSchema))) IgluUtils @@ -444,18 +482,23 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect .value .map { case Ior.Both( - NonEmptyList(FailureEntity.SchemaViolation(FailureDetails.SchemaViolation.IgluError(_, _: ResolutionError), _, _), Nil), + NonEmptyList(FailureEntity.SchemaViolation(FailureDetails.SchemaViolation.IgluError(_, _: ResolutionError), + `contextsFieldName`, + `json` + ), + Nil + ), Nil ) => ok - 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") + case other => ko(s"[$other] is not expected one") } } "return 2 expected failures for 2 invalid contexts" >> { val input = new EnrichedEvent + val invalidEmailSentJson = invalidEmailSent.toJson + val noSchemaJson = noSchema.toJson input.setContexts(buildInputContexts(List(invalidEmailSent, noSchema))) IgluUtils @@ -463,8 +506,16 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect .value .map { case Ior.Both(NonEmptyList( - FailureEntity.SchemaViolation(FailureDetails.SchemaViolation.IgluError(_, _: ValidationError), _, _), - List(FailureEntity.SchemaViolation(FailureDetails.SchemaViolation.IgluError(_, _: ResolutionError), _, _)) + FailureEntity.SchemaViolation(FailureDetails.SchemaViolation.IgluError(_, _: ValidationError), + `contextsFieldName`, + `invalidEmailSentJson` + ), + List( + FailureEntity.SchemaViolation(FailureDetails.SchemaViolation.IgluError(_, _: ResolutionError), + `contextsFieldName`, + `noSchemaJson` + ) + ) ), Nil ) => @@ -475,14 +526,19 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect "return an expected failure and an expected SDJ if one context is invalid and one is valid" >> { val input = new EnrichedEvent + val noSchemaJson = noSchema.toJson input.setContexts(buildInputContexts(List(emailSent1, noSchema))) IgluUtils .extractAndValidateInputContexts(input, SpecHelpers.client, SpecHelpers.registryLookup) .value .map { - case Ior.Both(NonEmptyList(FailureEntity.SchemaViolation(_: FailureDetails.SchemaViolation.IgluError, _, _), Nil), List(extract)) - if extract.sdj.schema == emailSentSchema => + case Ior.Both(NonEmptyList( + FailureEntity.SchemaViolation(_: FailureDetails.SchemaViolation.IgluError, `contextsFieldName`, `noSchemaJson`), + Nil + ), + List(extract) + ) if extract.sdj.schema == emailSentSchema => ok case other => ko(s"[$other] is not one IgluError and one SDJ with schema $emailSentSchema") } @@ -536,6 +592,7 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect "validateEnrichmentsContexts" should { "return one expected SchemaViolation for one invalid context" >> { + val json = invalidEmailSent.toJson val contexts = List( SpecHelpers.jsonStringToSDJ(invalidEmailSent).right.get ) @@ -545,7 +602,12 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect .value .map { case Ior.Both( - NonEmptyList(FailureEntity.SchemaViolation(FailureDetails.SchemaViolation.IgluError(_, _: ValidationError), _, _), Nil), + NonEmptyList(FailureEntity.SchemaViolation(FailureDetails.SchemaViolation.IgluError(_, _: ValidationError), + `derivedContextsFieldName`, + `json` + ), + Nil + ), Nil ) => ok @@ -554,6 +616,8 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect } "return two expected SchemaViolation for two invalid contexts" >> { + val invalidEmailSentJson = invalidEmailSent.toJson + val noSchemaJson = noSchema.toJson val contexts = List( SpecHelpers.jsonStringToSDJ(invalidEmailSent).right.get, SpecHelpers.jsonStringToSDJ(noSchema).right.get @@ -564,8 +628,16 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect .value .map { case Ior.Both(NonEmptyList( - FailureEntity.SchemaViolation(FailureDetails.SchemaViolation.IgluError(_, _: ValidationError), _, _), - List(FailureEntity.SchemaViolation(FailureDetails.SchemaViolation.IgluError(_, _: ResolutionError), _, _)) + FailureEntity.SchemaViolation(FailureDetails.SchemaViolation.IgluError(_, _: ValidationError), + `derivedContextsFieldName`, + `invalidEmailSentJson` + ), + List( + FailureEntity.SchemaViolation(FailureDetails.SchemaViolation.IgluError(_, _: ResolutionError), + `derivedContextsFieldName`, + `noSchemaJson` + ) + ) ), Nil ) => @@ -575,6 +647,7 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect } "return one expected SchemaViolation for one invalid context and one valid" >> { + val invalidEmailSentJson = invalidEmailSent.toJson val contexts = List( SpecHelpers.jsonStringToSDJ(invalidEmailSent).right.get, SpecHelpers.jsonStringToSDJ(emailSent1).right.get @@ -585,7 +658,10 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect .value .map { case Ior.Both(NonEmptyList( - FailureEntity.SchemaViolation(FailureDetails.SchemaViolation.IgluError(_, _: ValidationError), _, _), + FailureEntity.SchemaViolation(FailureDetails.SchemaViolation.IgluError(_, _: ValidationError), + `derivedContextsFieldName`, + `invalidEmailSentJson` + ), Nil ), List(sdj) @@ -712,6 +788,7 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect "return the SchemaViolation of the invalid context in the Left and the extracted unstructured event in the Right" >> { val input = new EnrichedEvent + val invalidEmailSentJson = invalidEmailSent.toJson input.setUnstruct_event(buildUnstruct(emailSent1)) input.setContexts(buildInputContexts(List(invalidEmailSent))) @@ -724,7 +801,12 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect .value .map { case Ior.Both( - NonEmptyList(FailureEntity.SchemaViolation(FailureDetails.SchemaViolation.IgluError(_, _: ValidationError), _, _), _), + NonEmptyList(FailureEntity.SchemaViolation(FailureDetails.SchemaViolation.IgluError(_, _: ValidationError), + `contextsFieldName`, + `invalidEmailSentJson` + ), + _ + ), extract ) if extract.contexts.isEmpty && extract.unstructEvent.isDefined && extract.unstructEvent.get.schema == emailSentSchema => ok @@ -737,6 +819,7 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect "return the SchemaViolation of the invalid unstructured event in the Left and the valid context in the Right" >> { val input = new EnrichedEvent + val invalidEmailSentJson = invalidEmailSent.toJson input.setUnstruct_event(buildUnstruct(invalidEmailSent)) input.setContexts(buildInputContexts(List(emailSent1))) @@ -749,7 +832,12 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect .value .map { case Ior.Both( - NonEmptyList(FailureEntity.SchemaViolation(FailureDetails.SchemaViolation.IgluError(_, _: ValidationError), _, _), _), + NonEmptyList(FailureEntity.SchemaViolation(FailureDetails.SchemaViolation.IgluError(_, _: ValidationError), + `uePropertiesFieldName`, + `invalidEmailSentJson` + ), + _ + ), extract ) if extract.contexts.size == 1 && extract.contexts.head.schema == emailSentSchema && extract.unstructEvent.isEmpty => ok @@ -765,12 +853,11 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect input.setUnstruct_event(buildUnstruct(supersedingExample1)) input.setContexts(buildInputContexts(List(supersedingExample1, supersedingExample2))) - val expectedValidationInfoContext = parse( + val expectedValidationInfoContext = """ { | "originalSchema" : "iglu:com.acme/superseding_example/jsonschema/1-0-0", | "validatedWith" : "1-0-1" - |}""".stripMargin - ).toOption.get + |}""".stripMargin.toJson IgluUtils .extractAndValidateInputJsons( @@ -800,4 +887,8 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect def buildInputContexts(sdjs: List[String] = List.empty[String]) = s"""{"schema": "${inputContextsSchema.toSchemaUri}", "data": [${sdjs.mkString(",")}]}""" + + implicit class StringToJson(str: String) { + def toJson: Json = parse(str).toOption.get + } }