From 140ff914056192ea417031f1cd5ef8a7fc91e0e5 Mon Sep 17 00:00:00 2001 From: Benjamin Benoist Date: Wed, 14 Feb 2024 11:08:08 +0000 Subject: [PATCH] Add incomplete events (close #) Before this change, any error in the enriching workflow would short circuit and a bad row would be emitted. After this change, if incomplete events are enabled, the enriching goes to the end with what is possible, accumulating errors as it goes. Errors get attached in derived_contexts. There are now 3 main steps : - Mapping and validating the input. This includes mapping fields of payload_data to the atomic event (e.g. tr_tt to tr_total while converting from string to number) and validating the contexts and unstruct event. Everything that goes wrong gets wrapped up in a SchemaViolations bad row. - Running the enrichments. Everything that goes wrong gets wrapped up in an EnrichmentFailures bad row. - Validating the output. This includes validating the enrichments contexts and the atomic fields lengths. Everything that goes wrong gets wrapped up in an EnrichmentFailures bad row. --- config/config.file.extended.hocon | 8 + config/config.kafka.extended.hocon | 19 + config/config.kinesis.extended.hocon | 49 +- config/config.nsq.extended.hocon | 21 + config/config.pubsub.extended.hocon | 25 + .../snowplow/enrich/common/fs2/Enrich.scala | 45 +- .../enrich/common/fs2/Environment.scala | 5 + .../snowplow/enrich/common/fs2/Run.scala | 4 + .../enrich/common/fs2/config/ConfigFile.scala | 29 +- .../enrich/common/fs2/config/io.scala | 3 +- .../snowplow/enrich/common/fs2/package.scala | 6 +- .../enrich/common/fs2/EnrichSpec.scala | 35 +- .../common/fs2/EventGenEtlPipelineSpec.scala | 15 +- .../common/fs2/blackbox/BlackBoxTesting.scala | 7 +- .../blackbox/adapters/Tp2AdapterSpec.scala | 5 +- .../common/fs2/config/ParsedConfigsSpec.scala | 3 +- .../common/fs2/test/TestEnvironment.scala | 2 + .../common/EtlPipeline.scala | 24 +- .../AtomicFieldsLengthValidator.scala | 53 +- .../enrichments/ClientEnrichments.scala | 10 +- .../enrichments/EnrichmentManager.scala | 250 ++++--- .../common/enrichments/EventEnrichments.scala | 68 +- .../common/enrichments/MiscEnrichments.scala | 15 +- .../common/enrichments/Transform.scala | 8 +- .../registry/CrossNavigationEnrichment.scala | 2 +- .../common/utils/ConversionUtils.scala | 104 ++- .../common/utils/IgluUtils.scala | 162 ++--- .../common/utils/JsonUtils.scala | 34 +- .../common/utils/MapTransformer.scala | 12 +- .../EtlPipelineSpec.scala | 29 +- .../SpecHelpers.scala | 2 + .../enrichments/EnrichmentManagerSpec.scala | 241 ++++--- .../enrichments/clientEnrichmentSpecs.scala | 25 +- .../enrichments/eventEnrichmentSpecs.scala | 45 +- .../enrichments/miscEnrichmentSpecs.scala | 12 +- .../pii/PiiPseudonymizerEnrichmentSpec.scala | 24 +- .../utils/IgluUtilsSpec.scala | 619 ++++++++++-------- .../utils/MapTransformerSpec.scala | 2 +- .../utils/ValidateAndReformatJsonSpec.scala | 51 -- .../utils/conversionUtilsSpecs.scala | 44 +- .../kafka/src/main/resources/application.conf | 15 + .../AzureAuthenticationCallbackHandler.scala | 2 + .../Main.scala | 1 + .../snowplow/enrich/kafka/ConfigSpec.scala | 17 +- .../src/main/resources/application.conf | 17 + .../snowplow/enrich/kinesis/Main.scala | 1 + .../snowplow/enrich/kinesis/ConfigSpec.scala | 15 +- .../snowplow/enrich/nsq/Main.scala | 1 + .../snowplow/enrich/nsq/ConfigSpec.scala | 15 +- .../src/main/resources/application.conf | 12 + .../snowplow/enrich/pubsub/Main.scala | 1 + .../snowplow/enrich/pubsub/ConfigSpec.scala | 13 +- 52 files changed, 1246 insertions(+), 981 deletions(-) delete mode 100644 modules/common/src/test/scala/com.snowplowanalytics.snowplow.enrich.common/utils/ValidateAndReformatJsonSpec.scala diff --git a/config/config.file.extended.hocon b/config/config.file.extended.hocon index b87a9ffd2..171a893fd 100644 --- a/config/config.file.extended.hocon +++ b/config/config.file.extended.hocon @@ -31,6 +31,14 @@ "file": "/var/bad" "maxBytes": 1000000 } + + # Incomplete events output + "incomplete": { + # Local FS supported for testing purposes + "type": "FileSystem" + "file": "/var/incomplete" + "maxBytes": 1000000 + } } # Optional. Concurrency of the app diff --git a/config/config.kafka.extended.hocon b/config/config.kafka.extended.hocon index 78965106f..8beafa1cc 100644 --- a/config/config.kafka.extended.hocon +++ b/config/config.kafka.extended.hocon @@ -89,6 +89,25 @@ "acks": "all" } } + + # Optional. Incomplete events output. + # If set, an incomplete enriched event holding the errors in derived_context will get emitted on top of a bad row + "incomplete": { + "type": "Kafka" + + # Name of the Kafka topic to write to + "topicName": "incomplete" + + # A list of host:port pairs to use for establishing the initial connection to the Kafka cluster + # This list should be in the form host1:port1,host2:port2,... + "bootstrapServers": "localhost:9092" + + # Optional, Kafka producer configuration + # See https://kafka.apache.org/documentation/#producerconfigs for all properties + "producerConf": { + "acks": "all" + } + } } # Optional. Concurrency of the app diff --git a/config/config.kinesis.extended.hocon b/config/config.kinesis.extended.hocon index 46aa49adc..6568096d4 100644 --- a/config/config.kinesis.extended.hocon +++ b/config/config.kinesis.extended.hocon @@ -94,7 +94,7 @@ # Otherwise, the partition key will be a random UUID. # "partitionKey": "user_id" - # Optional. Policy to retry if writing to kinesis fails with unexepected errors + # Optional. Policy to retry if writing to kinesis fails with unexpected errors "backoffPolicy": { "minBackoff": 100 milliseconds "maxBackoff": 10 seconds @@ -144,7 +144,7 @@ # Otherwise, the partition key will be a random UUID. # "partitionKey": "user_id" - # Optional. Policy to retry if writing to kinesis fails with unexepcted errors + # Optional. Policy to retry if writing to kinesis fails with unexpected errors "backoffPolicy": { "minBackoff": 100 milliseconds "maxBackoff": 10 seconds @@ -186,7 +186,50 @@ # https://sdk.amazonaws.com/java/api/latest/software/amazon/awssdk/regions/providers/DefaultAwsRegionProviderChain.html "region": "eu-central-1" - # Optional. Policy to retry if writing to kinesis fails with unexepcted errors + # Optional. Policy to retry if writing to kinesis fails with unexpected errors + "backoffPolicy": { + "minBackoff": 100 milliseconds + "maxBackoff": 10 seconds + "maxRetries": 10 + } + + # Optional. Policy to retry if writing to kinesis exceeds the provisioned throughput. + "throttledBackoffPolicy": { + "minBackoff": 100 milliseconds + "maxBackoff": 1 second + } + + # Optional. Limits the number of events in a single PutRecords request. + # Several requests are made in parallel + # Maximum allowed: 500 + "recordLimit": 500 + + # Optional. Limits the number of bytes in a single PutRecords request, + # including records and partition keys. + # Several requests are made in parallel + # Maximum allowed: 5 MB + "byteLimit": 5242880 + + # Optional. Use a custom Kinesis endpoint. + # Can be used for instance to work locally with localstack + # "customEndpoint": "https://localhost:4566" + } + + # Optional. Incomplete events output. + # If set, an incomplete enriched event holding the errors in derived_context will get emitted on top of a bad row + "incomplete": { + "type": "Kinesis" + + # Name of the Kinesis stream to write to + "streamName": "incomplete" + + # Optional. Region where the Kinesis stream is located + # This field is optional if it can be resolved with AWS region provider chain. + # It checks places like env variables, system properties, AWS profile file. + # https://sdk.amazonaws.com/java/api/latest/software/amazon/awssdk/regions/providers/DefaultAwsRegionProviderChain.html + "region": "eu-central-1" + + # Optional. Policy to retry if writing to kinesis fails with unexpected errors "backoffPolicy": { "minBackoff": 100 milliseconds "maxBackoff": 10 seconds diff --git a/config/config.nsq.extended.hocon b/config/config.nsq.extended.hocon index 07532f0f5..6e5aae5bd 100644 --- a/config/config.nsq.extended.hocon +++ b/config/config.nsq.extended.hocon @@ -95,6 +95,27 @@ "maxRetries": 10 } } + + # Incomplete events output + "incomplete": { + "type": "Nsq" + + # Name of the NSQ topic that will receive the incomplete events + "topic": "incomplete" + + # The host name of nsqd application + "nsqdHost": "127.0.0.1" + + # The port number of nsqd application + "nsqdPort": 4150 + + # Optional. Policy to retry if writing to NSQ fails + "backoffPolicy": { + "minBackoff": 100 milliseconds + "maxBackoff": 10 seconds + "maxRetries": 10 + } + } } # Optional. Concurrency of the app diff --git a/config/config.pubsub.extended.hocon b/config/config.pubsub.extended.hocon index 43388b12b..32425f39d 100644 --- a/config/config.pubsub.extended.hocon +++ b/config/config.pubsub.extended.hocon @@ -114,6 +114,31 @@ # Note the PubSub maximum is 10MB "maxBatchBytes": 8000000 } + + # Optional. Incomplete events output. + # If set, an incomplete enriched event holding the errors in derived_context will get emitted on top of a bad row + "incomplete": { + "type": "PubSub" + + # Name of the PubSub topic that will receive the incomplete events + "topic": "projects/test-project/topics/incomplete" + + # Optional. Delay threshold to use for batching. + # After this amount of time has elapsed, + # before maxBatchSize and maxBatchBytes have been reached, + # messages from the buffer will be sent. + "delayThreshold": 200 milliseconds + + # Optional. Maximum number of messages sent within a batch. + # When the buffer reaches this number of messages they are sent. + # PubSub maximum : 1000 + "maxBatchSize": 1000 + + # Optional. Maximum number of bytes sent within a batch. + # When the buffer reaches this size messages are sent. + # Note the PubSub maximum is 10MB + "maxBatchBytes": 8000000 + } } # Optional. Concurrency of the app diff --git a/modules/common-fs2/src/main/scala/com/snowplowanalytics/snowplow/enrich/common/fs2/Enrich.scala b/modules/common-fs2/src/main/scala/com/snowplowanalytics/snowplow/enrich/common/fs2/Enrich.scala index 89753726b..33ea660ad 100644 --- a/modules/common-fs2/src/main/scala/com/snowplowanalytics/snowplow/enrich/common/fs2/Enrich.scala +++ b/modules/common-fs2/src/main/scala/com/snowplowanalytics/snowplow/enrich/common/fs2/Enrich.scala @@ -16,7 +16,7 @@ import java.util.Base64 import org.joda.time.DateTime -import cats.data.{NonEmptyList, ValidatedNel} +import cats.data.{Ior, NonEmptyList, ValidatedNel} import cats.{Monad, Parallel} import cats.implicits._ @@ -72,7 +72,8 @@ object Enrich { env.featureFlags, env.metrics.invalidCount, env.registryLookup, - env.atomicFields + env.atomicFields, + env.sinkIncomplete.isDefined ) val enriched = @@ -119,7 +120,8 @@ object Enrich { featureFlags: FeatureFlags, invalidCount: F[Unit], registryLookup: RegistryLookup[F], - atomicFields: AtomicFields + atomicFields: AtomicFields, + emitIncomplete: Boolean )( row: Array[Byte] ): F[Result] = { @@ -140,7 +142,8 @@ object Enrich { FeatureFlags.toCommon(featureFlags), invalidCount, registryLookup, - atomicFields + atomicFields, + emitIncomplete ) } yield (enriched, collectorTstamp) @@ -170,7 +173,7 @@ object Enrich { case None => Sync[F].unit } - } yield (List(badRow.invalid), collectorTstamp) + } yield (List(Ior.left(badRow)), collectorTstamp) /** Build a `generic_error` bad row for unhandled runtime errors */ def genericBadRow( @@ -189,17 +192,29 @@ object Enrich { chunk: List[Result], env: Environment[F, A] ): F[Unit] = { - val (bad, enriched) = + val (bad, enriched, incomplete) = chunk .flatMap(_._1) - .map(_.toEither) - .separate + .foldLeft((List.empty[BadRow], List.empty[EnrichedEvent], List.empty[EnrichedEvent])) { + case (previous, item) => + val (bad, enriched, incomplete) = previous + item match { + case Ior.Right(e) => (bad, e :: enriched, incomplete) + case Ior.Left(br) => (br :: bad, enriched, incomplete) + case Ior.Both(br, i) => (br :: bad, enriched, i :: incomplete) + } + } val (moreBad, good) = enriched.map { e => serializeEnriched(e, env.processor, env.streamsSettings.maxRecordSize) .map(bytes => (e, AttributedData(bytes, env.goodPartitionKey(e), env.goodAttributes(e)))) }.separate + val (incompleteTooBig, incompleteBytes) = incomplete.map { e => + serializeEnriched(e, env.processor, env.streamsSettings.maxRecordSize) + .map(bytes => AttributedData(bytes, env.goodPartitionKey(e), env.goodAttributes(e))) + }.separate + val allBad = (bad ++ moreBad).map(badRowResize(env, _)) List( @@ -214,7 +229,10 @@ object Enrich { env.processor, env.streamsSettings.maxRecordSize ) *> env.metrics.enrichLatency(chunk.headOption.flatMap(_._2)), - sinkBad(allBad, env.sinkBad, env.metrics.badCount) + sinkBad(allBad, env.sinkBad, env.metrics.badCount), + if (incompleteTooBig.nonEmpty) Logger[F].warn(s"${incompleteTooBig.size} incomplete events discarded because they are too big") + else Sync[F].unit, + sinkIncomplete(incompleteBytes, env.sinkIncomplete) ).parSequence_ } @@ -272,6 +290,15 @@ object Enrich { Sync[F].unit } + def sinkIncomplete[F[_]: Sync]( + incomplete: List[AttributedData[Array[Byte]]], + maybeSink: Option[AttributedByteSink[F]] + ): F[Unit] = + maybeSink match { + case Some(sink) => sink(incomplete) + case None => Sync[F].unit + } + def serializeEnriched( enriched: EnrichedEvent, processor: Processor, diff --git a/modules/common-fs2/src/main/scala/com/snowplowanalytics/snowplow/enrich/common/fs2/Environment.scala b/modules/common-fs2/src/main/scala/com/snowplowanalytics/snowplow/enrich/common/fs2/Environment.scala index 648ee6c2f..6eaac5d6f 100644 --- a/modules/common-fs2/src/main/scala/com/snowplowanalytics/snowplow/enrich/common/fs2/Environment.scala +++ b/modules/common-fs2/src/main/scala/com/snowplowanalytics/snowplow/enrich/common/fs2/Environment.scala @@ -78,6 +78,7 @@ import com.snowplowanalytics.snowplow.enrich.common.fs2.io.experimental.Metadata * @param sinkGood function that sinks enriched event * @param sinkPii function that sinks pii event * @param sinkBad function that sinks an event that failed validation or enrichment + * @param sinkIncomplete function that sinks incomplete events * @param checkpoint function that checkpoints input stream records * @param getPayload function that extracts the collector payload bytes from a record * @param sentry optional sentry client @@ -111,6 +112,7 @@ final case class Environment[F[_], A]( sinkGood: AttributedByteSink[F], sinkPii: Option[AttributedByteSink[F]], sinkBad: ByteSink[F], + sinkIncomplete: Option[AttributedByteSink[F]], checkpoint: List[A] => F[Unit], getPayload: A => Array[Byte], sentry: Option[SentryClient], @@ -187,6 +189,7 @@ object Environment { sinkGood: Resource[F, AttributedByteSink[F]], sinkPii: Option[Resource[F, AttributedByteSink[F]]], sinkBad: Resource[F, ByteSink[F]], + sinkIncomplete: Option[Resource[F, AttributedByteSink[F]]], clients: Resource[F, List[Client[F]]], checkpoint: List[A] => F[Unit], getPayload: A => Array[Byte], @@ -204,6 +207,7 @@ object Environment { good <- sinkGood bad <- sinkBad pii <- sinkPii.sequence + incomplete <- sinkIncomplete.sequence http4s <- Clients.mkHttp() clts <- clients.map(Clients.init(http4s, _)) igluClient <- IgluCirceClient.parseDefault[F](parsedConfigs.igluJson).resource @@ -231,6 +235,7 @@ object Environment { good, pii, bad, + incomplete, checkpoint, getPayload, sentry, diff --git a/modules/common-fs2/src/main/scala/com/snowplowanalytics/snowplow/enrich/common/fs2/Run.scala b/modules/common-fs2/src/main/scala/com/snowplowanalytics/snowplow/enrich/common/fs2/Run.scala index 746c582af..af246637d 100644 --- a/modules/common-fs2/src/main/scala/com/snowplowanalytics/snowplow/enrich/common/fs2/Run.scala +++ b/modules/common-fs2/src/main/scala/com/snowplowanalytics/snowplow/enrich/common/fs2/Run.scala @@ -57,6 +57,7 @@ object Run { mkSinkGood: Output => Resource[F, AttributedByteSink[F]], mkSinkPii: Output => Resource[F, AttributedByteSink[F]], mkSinkBad: Output => Resource[F, ByteSink[F]], + mkSinkIncomplete: Output => Resource[F, AttributedByteSink[F]], checkpoint: List[A] => F[Unit], mkClients: BlobStorageClients => List[Resource[F, Client[F]]], getPayload: A => Array[Byte], @@ -89,6 +90,7 @@ object Run { case _ => mkSinkBad(file.output.bad) } + sinkIncomplete = file.output.incomplete.map(out => initAttributedSink(out, mkSinkIncomplete)) clients = mkClients(file.blobStorage).sequence exit <- file.input match { case p: Input.FileSystem => @@ -100,6 +102,7 @@ object Run { sinkGood, sinkPii, sinkBad, + sinkIncomplete, clients, _ => Sync[F].unit, identity, @@ -130,6 +133,7 @@ object Run { sinkGood, sinkPii, sinkBad, + sinkIncomplete, clients, checkpointing, getPayload, diff --git a/modules/common-fs2/src/main/scala/com/snowplowanalytics/snowplow/enrich/common/fs2/config/ConfigFile.scala b/modules/common-fs2/src/main/scala/com/snowplowanalytics/snowplow/enrich/common/fs2/config/ConfigFile.scala index c2cd740a9..6ffb12f61 100644 --- a/modules/common-fs2/src/main/scala/com/snowplowanalytics/snowplow/enrich/common/fs2/config/ConfigFile.scala +++ b/modules/common-fs2/src/main/scala/com/snowplowanalytics/snowplow/enrich/common/fs2/config/ConfigFile.scala @@ -59,20 +59,23 @@ object ConfigFile { implicit val configFileDecoder: Decoder[ConfigFile] = deriveConfiguredDecoder[ConfigFile].emap { - case ConfigFile(_, _, _, Some(aup), _, _, _, _, _, _, _, _, _) if aup._1 <= 0L => + case c: ConfigFile if c.assetsUpdatePeriod.exists(_.length <= 0L) => "assetsUpdatePeriod in config file cannot be less than 0".asLeft // TODO: use newtype - // Remove pii output if streamName and region empty - case c @ ConfigFile(_, Outputs(good, Some(output: Output.Kinesis), bad), _, _, _, _, _, _, _, _, _, _, _) - if output.streamName.isEmpty => - c.copy(output = Outputs(good, None, bad)).asRight - // Remove pii output if topic empty - case c @ ConfigFile(_, Outputs(good, Some(Output.PubSub(t, _, _, _, _, _)), bad), _, _, _, _, _, _, _, _, _, _, _) if t.isEmpty => - c.copy(output = Outputs(good, None, bad)).asRight - // Remove pii output if topic empty - case c @ ConfigFile(_, Outputs(good, Some(Output.Kafka(topicName, _, _, _, _)), bad), _, _, _, _, _, _, _, _, _, _, _) - if topicName.isEmpty => - c.copy(output = Outputs(good, None, bad)).asRight - case other => other.asRight + case c: ConfigFile => + val Outputs(good, pii, bad, incomplete) = c.output + val piiCleaned = pii match { + case Some(ki: Output.Kinesis) if ki.streamName.isEmpty => None + case Some(p: Output.PubSub) if p.topic.isEmpty => None + case Some(ka: Output.Kafka) if ka.topicName.isEmpty => None + case _ => pii + } + val incompleteCleaned = incomplete match { + case Some(ki: Output.Kinesis) if ki.streamName.isEmpty => None + case Some(p: Output.PubSub) if p.topic.isEmpty => None + case Some(ka: Output.Kafka) if ka.topicName.isEmpty => None + case _ => incomplete + } + c.copy(output = Outputs(good, piiCleaned, bad, incompleteCleaned)).asRight } /* Defines where to look for default values if they are not in the provided file diff --git a/modules/common-fs2/src/main/scala/com/snowplowanalytics/snowplow/enrich/common/fs2/config/io.scala b/modules/common-fs2/src/main/scala/com/snowplowanalytics/snowplow/enrich/common/fs2/config/io.scala index e43c6dd20..633781548 100644 --- a/modules/common-fs2/src/main/scala/com/snowplowanalytics/snowplow/enrich/common/fs2/config/io.scala +++ b/modules/common-fs2/src/main/scala/com/snowplowanalytics/snowplow/enrich/common/fs2/config/io.scala @@ -209,7 +209,8 @@ object io { case class Outputs( good: Output, pii: Option[Output], - bad: Output + bad: Output, + incomplete: Option[Output] ) object Outputs { implicit val outputsDecoder: Decoder[Outputs] = deriveConfiguredDecoder[Outputs] diff --git a/modules/common-fs2/src/main/scala/com/snowplowanalytics/snowplow/enrich/common/fs2/package.scala b/modules/common-fs2/src/main/scala/com/snowplowanalytics/snowplow/enrich/common/fs2/package.scala index a6a0a2337..40cd78efe 100644 --- a/modules/common-fs2/src/main/scala/com/snowplowanalytics/snowplow/enrich/common/fs2/package.scala +++ b/modules/common-fs2/src/main/scala/com/snowplowanalytics/snowplow/enrich/common/fs2/package.scala @@ -10,7 +10,7 @@ */ package com.snowplowanalytics.snowplow.enrich.common -import cats.data.{EitherT, Validated, ValidatedNel} +import cats.data.{EitherT, Ior, ValidatedNel} import com.snowplowanalytics.snowplow.badrows.BadRow @@ -25,8 +25,8 @@ package object fs2 { type ByteSink[F[_]] = List[Array[Byte]] => F[Unit] type AttributedByteSink[F[_]] = List[AttributedData[Array[Byte]]] => F[Unit] - /** Enrichment result, containing list of (valid and invalid) results as well as the collector timestamp */ - type Result = (List[Validated[BadRow, EnrichedEvent]], Option[Long]) + type Enriched = Ior[BadRow, EnrichedEvent] + type Result = (List[Enriched], Option[Long]) /** Function to transform an origin raw payload into good and/or bad rows */ type Enrich[F[_]] = Array[Byte] => F[Result] diff --git a/modules/common-fs2/src/test/scala/com/snowplowanalytics/snowplow/enrich/common/fs2/EnrichSpec.scala b/modules/common-fs2/src/test/scala/com/snowplowanalytics/snowplow/enrich/common/fs2/EnrichSpec.scala index 85c06a0e9..b285145a2 100644 --- a/modules/common-fs2/src/test/scala/com/snowplowanalytics/snowplow/enrich/common/fs2/EnrichSpec.scala +++ b/modules/common-fs2/src/test/scala/com/snowplowanalytics/snowplow/enrich/common/fs2/EnrichSpec.scala @@ -15,7 +15,7 @@ import java.util.{Base64, UUID} import scala.concurrent.duration._ -import cats.data.{NonEmptyList, Validated} +import cats.data.{Ior, NonEmptyList, Validated} import cats.implicits._ import cats.effect.IO @@ -77,13 +77,14 @@ class EnrichSpec extends Specification with CatsEffect with ScalaCheck { EnrichSpec.featureFlags, IO.unit, SpecHelpers.registryLookup, - AtomicFields.from(valueLimits = Map.empty) + AtomicFields.from(valueLimits = Map.empty), + SpecHelpers.emitIncomplete )( EnrichSpec.payload ) .map(normalizeResult) .map { - case List(Validated.Valid(event)) => event must beEqualTo(expected) + case List(Ior.Right(event)) => event must beEqualTo(expected) case other => ko(s"Expected one valid event, got $other") } } @@ -107,13 +108,14 @@ class EnrichSpec extends Specification with CatsEffect with ScalaCheck { EnrichSpec.featureFlags, IO.unit, SpecHelpers.registryLookup, - AtomicFields.from(valueLimits = Map.empty) + AtomicFields.from(valueLimits = Map.empty), + SpecHelpers.emitIncomplete )( payload ) .map(normalizeResult) .map { - case List(Validated.Valid(e)) => e.event must beSome("page_view") + case List(Ior.Right(e)) => e.event must beSome("page_view") case other => ko(s"Expected one valid event, got $other") } } @@ -145,13 +147,14 @@ class EnrichSpec extends Specification with CatsEffect with ScalaCheck { EnrichSpec.featureFlags.copy(tryBase64Decoding = true), IO.unit, SpecHelpers.registryLookup, - AtomicFields.from(valueLimits = Map.empty) + AtomicFields.from(valueLimits = Map.empty), + SpecHelpers.emitIncomplete )( Base64.getEncoder.encode(EnrichSpec.payload) ) .map(normalizeResult) .map { - case List(Validated.Valid(event)) => event must beEqualTo(expected) + case List(Ior.Right(event)) => event must beEqualTo(expected) case other => ko(s"Expected one valid event, got $other") } } @@ -169,13 +172,14 @@ class EnrichSpec extends Specification with CatsEffect with ScalaCheck { EnrichSpec.featureFlags, IO.unit, SpecHelpers.registryLookup, - AtomicFields.from(valueLimits = Map.empty) + AtomicFields.from(valueLimits = Map.empty), + SpecHelpers.emitIncomplete )( Base64.getEncoder.encode(EnrichSpec.payload) ) .map(normalizeResult) .map { - case List(Validated.Invalid(badRow)) => println(badRow); ok + case List(Ior.Left(_)) => ok case other => ko(s"Expected one bad row, got $other") } } @@ -449,16 +453,16 @@ class EnrichSpec extends Specification with CatsEffect with ScalaCheck { def sinkGood( environment: Environment[IO, Array[Byte]], enriched: EnrichedEvent - ): IO[Unit] = sinkOne(environment, Validated.Valid(enriched)) + ): IO[Unit] = sinkOne(environment, Ior.Right(enriched)) def sinkBad( environment: Environment[IO, Array[Byte]], badRow: BadRow - ): IO[Unit] = sinkOne(environment, Validated.Invalid(badRow)) + ): IO[Unit] = sinkOne(environment, Ior.Left(badRow)) def sinkOne( environment: Environment[IO, Array[Byte]], - event: Validated[BadRow, EnrichedEvent] + event: Ior[BadRow, EnrichedEvent] ): IO[Unit] = Enrich.sinkChunk(List((List(event), None)), environment) } @@ -491,10 +495,11 @@ object EnrichSpec { Validated.Invalid(badRow) } - def normalizeResult(payload: Result): List[Validated[BadRow, Event]] = + def normalizeResult(payload: Result): List[Ior[BadRow, Event]] = payload._1.map { - case Validated.Valid(a) => normalize(ConversionUtils.tabSeparatedEnrichedEvent(a)) - case Validated.Invalid(e) => e.invalid + case Ior.Right(enriched) => normalize(ConversionUtils.tabSeparatedEnrichedEvent(enriched)).toIor + case Ior.Left(err) => Ior.Left(err) + case Ior.Both(_, enriched) => normalize(ConversionUtils.tabSeparatedEnrichedEvent(enriched)).toIor } val minimalEvent = Event diff --git a/modules/common-fs2/src/test/scala/com/snowplowanalytics/snowplow/enrich/common/fs2/EventGenEtlPipelineSpec.scala b/modules/common-fs2/src/test/scala/com/snowplowanalytics/snowplow/enrich/common/fs2/EventGenEtlPipelineSpec.scala index f9632d9d5..55fee9718 100644 --- a/modules/common-fs2/src/test/scala/com/snowplowanalytics/snowplow/enrich/common/fs2/EventGenEtlPipelineSpec.scala +++ b/modules/common-fs2/src/test/scala/com/snowplowanalytics/snowplow/enrich/common/fs2/EventGenEtlPipelineSpec.scala @@ -10,7 +10,7 @@ */ package com.snowplowanalytics.snowplow.enrich.common.fs2 -import cats.data.{Validated, ValidatedNel} +import cats.data.{Ior, ValidatedNel} import cats.effect.testing.specs2.CatsEffect import cats.effect.IO import cats.effect.unsafe.implicits.global @@ -43,10 +43,12 @@ import org.specs2.specification.core.{Fragment, Fragments} import java.time.Instant import scala.util.{Random, Try} + import com.snowplowanalytics.snowplow.enrich.common.enrichments.AtomicFields -class EventGenEtlPipelineSpec extends Specification with CatsEffect { +import com.snowplowanalytics.snowplow.enrich.common.SpecHelpers +class EventGenEtlPipelineSpec extends Specification with CatsEffect { case class ContextMatcher(v: String) implicit val cmDecoder: Decoder[ContextMatcher] = Decoder.decodeString.emapTry { str => @@ -198,7 +200,7 @@ class EventGenEtlPipelineSpec extends Specification with CatsEffect { val dateTime = DateTime.now() val process = Processor("EventGenEtlPipelineSpec", "v1") - def processEvents(e: CollectorPayload): IO[List[Validated[BadRow, EnrichedEvent]]] = + def processEvents(e: CollectorPayload): IO[List[Ior[BadRow, EnrichedEvent]]] = EtlPipeline.processEvents[IO]( adapterRegistry, enrichmentReg, @@ -209,7 +211,8 @@ class EventGenEtlPipelineSpec extends Specification with CatsEffect { EtlPipeline.FeatureFlags(acceptInvalid = false, legacyEnrichmentOrder = false), IO.unit, SpecHelpers.registryLookup, - AtomicFields.from(Map.empty) + AtomicFields.from(Map.empty), + SpecHelpers.emitIncomplete ) def rethrowBadRows[A]: Pipe[IO, ValidatedNel[BadRow, A], A] = @@ -223,8 +226,8 @@ class EventGenEtlPipelineSpec extends Specification with CatsEffect { ).toEither ).rethrow[IO, A] - def rethrowBadRow[A]: Pipe[IO, Validated[BadRow, A], A] = - (in: Stream[IO, Validated[BadRow, A]]) => + def rethrowBadRow[A]: Pipe[IO, Ior[BadRow, A], A] = + (in: Stream[IO, Ior[BadRow, A]]) => in .map(_.leftMap(br => new Exception(br.compact)).toEither) .rethrow[IO, A] diff --git a/modules/common-fs2/src/test/scala/com/snowplowanalytics/snowplow/enrich/common/fs2/blackbox/BlackBoxTesting.scala b/modules/common-fs2/src/test/scala/com/snowplowanalytics/snowplow/enrich/common/fs2/blackbox/BlackBoxTesting.scala index b20a225ca..ee658f36d 100644 --- a/modules/common-fs2/src/test/scala/com/snowplowanalytics/snowplow/enrich/common/fs2/blackbox/BlackBoxTesting.scala +++ b/modules/common-fs2/src/test/scala/com/snowplowanalytics/snowplow/enrich/common/fs2/blackbox/BlackBoxTesting.scala @@ -19,7 +19,7 @@ import cats.effect.kernel.Resource import cats.effect.testing.specs2.CatsEffect -import cats.data.Validated +import cats.data.Ior import cats.data.Validated.{Invalid, Valid} import io.circe.Json @@ -104,12 +104,13 @@ object BlackBoxTesting extends Specification with CatsEffect { featureFlags, IO.unit, SpecHelpers.registryLookup, - AtomicFields.from(valueLimits = Map.empty) + AtomicFields.from(valueLimits = Map.empty), + SpecHelpers.emitIncomplete )( input ) .map { - case (List(Validated.Valid(enriched)), _) => checkEnriched(enriched, expected) + case (List(Ior.Right(enriched)), _) => checkEnriched(enriched, expected) case other => ko(s"there should be one enriched event but got $other") } } diff --git a/modules/common-fs2/src/test/scala/com/snowplowanalytics/snowplow/enrich/common/fs2/blackbox/adapters/Tp2AdapterSpec.scala b/modules/common-fs2/src/test/scala/com/snowplowanalytics/snowplow/enrich/common/fs2/blackbox/adapters/Tp2AdapterSpec.scala index 07698aba5..0aa68adab 100644 --- a/modules/common-fs2/src/test/scala/com/snowplowanalytics/snowplow/enrich/common/fs2/blackbox/adapters/Tp2AdapterSpec.scala +++ b/modules/common-fs2/src/test/scala/com/snowplowanalytics/snowplow/enrich/common/fs2/blackbox/adapters/Tp2AdapterSpec.scala @@ -43,12 +43,13 @@ class Tp2AdapterSpec extends Specification with CatsEffect { EnrichSpec.featureFlags, IO.unit, SpecHelpers.registryLookup, - AtomicFields.from(valueLimits = Map.empty) + AtomicFields.from(valueLimits = Map.empty), + SpecHelpers.emitIncomplete )( input ) .map { - case (l, _) if l.forall(_.isValid) => l must haveSize(10) + case (l, _) if l.forall(_.isRight) => l must haveSize(10) case other => ko(s"there should be 10 enriched events, got $other") } } diff --git a/modules/common-fs2/src/test/scala/com/snowplowanalytics/snowplow/enrich/common/fs2/config/ParsedConfigsSpec.scala b/modules/common-fs2/src/test/scala/com/snowplowanalytics/snowplow/enrich/common/fs2/config/ParsedConfigsSpec.scala index 7e5f770c6..effde6e90 100644 --- a/modules/common-fs2/src/test/scala/com/snowplowanalytics/snowplow/enrich/common/fs2/config/ParsedConfigsSpec.scala +++ b/modules/common-fs2/src/test/scala/com/snowplowanalytics/snowplow/enrich/common/fs2/config/ParsedConfigsSpec.scala @@ -71,7 +71,8 @@ class ParsedConfigsSpec extends Specification with CatsEffect { 1000, 10000000, io.GcpUserAgent("Snowplow OSS") - ) + ), + None ), io.Concurrency(10000, 64), Some(7.days), diff --git a/modules/common-fs2/src/test/scala/com/snowplowanalytics/snowplow/enrich/common/fs2/test/TestEnvironment.scala b/modules/common-fs2/src/test/scala/com/snowplowanalytics/snowplow/enrich/common/fs2/test/TestEnvironment.scala index 53cf12cf9..515aec603 100644 --- a/modules/common-fs2/src/test/scala/com/snowplowanalytics/snowplow/enrich/common/fs2/test/TestEnvironment.scala +++ b/modules/common-fs2/src/test/scala/com/snowplowanalytics/snowplow/enrich/common/fs2/test/TestEnvironment.scala @@ -126,6 +126,7 @@ object TestEnvironment extends CatsEffect { goodRef <- Resource.eval(Ref.of[IO, Vector[AttributedData[Array[Byte]]]](Vector.empty)) piiRef <- Resource.eval(Ref.of[IO, Vector[AttributedData[Array[Byte]]]](Vector.empty)) badRef <- Resource.eval(Ref.of[IO, Vector[Array[Byte]]](Vector.empty)) + incompleteRef <- Resource.eval(Ref.of[IO, Vector[AttributedData[Array[Byte]]]](Vector.empty)) igluClient <- Resource.eval(SpecHelpers.createIgluClient(List(embeddedRegistry))) environment = Environment[IO, Array[Byte]]( igluClient, @@ -141,6 +142,7 @@ object TestEnvironment extends CatsEffect { g => goodRef.update(_ ++ g), Some(p => piiRef.update(_ ++ p)), b => badRef.update(_ ++ b), + Some(i => incompleteRef.update(_ ++ i)), _ => IO.unit, identity, None, diff --git a/modules/common/src/main/scala/com.snowplowanalytics.snowplow.enrich/common/EtlPipeline.scala b/modules/common/src/main/scala/com.snowplowanalytics.snowplow.enrich/common/EtlPipeline.scala index 7bb4b4794..339468f48 100644 --- a/modules/common/src/main/scala/com.snowplowanalytics.snowplow.enrich/common/EtlPipeline.scala +++ b/modules/common/src/main/scala/com.snowplowanalytics.snowplow.enrich/common/EtlPipeline.scala @@ -11,7 +11,7 @@ package com.snowplowanalytics.snowplow.enrich.common import cats.Monad -import cats.data.{Validated, ValidatedNel} +import cats.data.{Ior, Validated, ValidatedNel} import cats.effect.Clock import cats.implicits._ @@ -41,10 +41,6 @@ object EtlPipeline { case class FeatureFlags(acceptInvalid: Boolean, legacyEnrichmentOrder: Boolean) /** - * A helper method to take a ValidatedMaybeCanonicalInput and transform it into a List (possibly - * empty) of ValidatedCanonicalOutputs. - * We have to do some unboxing because enrichEvent expects a raw CanonicalInput as its argument, - * not a MaybeCanonicalInput. * @param adapterRegistry Contains all of the events adapters * @param enrichmentRegistry Contains configuration for all enrichments to apply * @param client Our Iglu client, for schema lookups and validation @@ -53,8 +49,6 @@ object EtlPipeline { * @param input The ValidatedMaybeCanonicalInput * @param featureFlags The feature flags available in the current version of Enrich * @param invalidCount Function to increment the count of invalid events - * @return the ValidatedMaybeCanonicalOutput. Thanks to flatMap, will include any validation - * errors contained within the ValidatedMaybeCanonicalInput */ def processEvents[F[_]: Clock: Monad]( adapterRegistry: AdapterRegistry[F], @@ -66,8 +60,9 @@ object EtlPipeline { featureFlags: FeatureFlags, invalidCount: F[Unit], registryLookup: RegistryLookup[F], - atomicFields: AtomicFields - ): F[List[Validated[BadRow, EnrichedEvent]]] = + atomicFields: AtomicFields, + emitIncomplete: Boolean + ): F[List[Ior[BadRow, EnrichedEvent]]] = input match { case Validated.Valid(Some(payload)) => adapterRegistry @@ -85,16 +80,17 @@ object EtlPipeline { featureFlags, invalidCount, registryLookup, - atomicFields + atomicFields, + emitIncomplete ) - .toValidated + .value } case Validated.Invalid(badRow) => - Monad[F].pure(List(badRow.invalid[EnrichedEvent])) + Monad[F].pure(List(Ior.left(badRow))) } case Validated.Invalid(badRows) => - Monad[F].pure(badRows.map(_.invalid[EnrichedEvent])).map(_.toList) + Monad[F].pure(badRows.toList.map(br => Ior.left(br))) case Validated.Valid(None) => - Monad[F].pure(List.empty[Validated[BadRow, EnrichedEvent]]) + Monad[F].pure(Nil) } } 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 5f2f98582..1ad6e19e6 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 @@ -14,14 +14,12 @@ import org.slf4j.LoggerFactory import cats.Monad import cats.data.Validated.{Invalid, Valid} -import cats.data.{NonEmptyList, ValidatedNel} +import cats.data.{Ior, IorT, NonEmptyList, ValidatedNel} import cats.implicits._ -import com.snowplowanalytics.snowplow.badrows.FailureDetails.EnrichmentFailure -import com.snowplowanalytics.snowplow.badrows.{BadRow, FailureDetails, Processor} +import com.snowplowanalytics.snowplow.badrows.FailureDetails -import com.snowplowanalytics.snowplow.enrich.common.adapters.RawEvent import com.snowplowanalytics.snowplow.enrich.common.enrichments.AtomicFields.LimitedAtomicField import com.snowplowanalytics.snowplow.enrich.common.outputs.EnrichedEvent @@ -35,21 +33,24 @@ object AtomicFieldsLengthValidator { def validate[F[_]: Monad]( event: EnrichedEvent, - rawEvent: RawEvent, - processor: Processor, acceptInvalid: Boolean, invalidCount: F[Unit], atomicFields: AtomicFields - ): F[Either[BadRow, Unit]] = - atomicFields.value - .map(validateField(event)) - .combineAll match { - case Invalid(errors) if acceptInvalid => - handleAcceptableBadRow(invalidCount, event, errors) *> Monad[F].pure(Right(())) - case Invalid(errors) => - Monad[F].pure(buildBadRow(event, rawEvent, processor, errors).asLeft) - case Valid(()) => - Monad[F].pure(Right(())) + ): IorT[F, NonEmptyList[FailureDetails.EnrichmentFailure], Unit] = + IorT { + atomicFields.value + .map(validateField(event)) + .combineAll match { + case Invalid(errors) if acceptInvalid => + handleAcceptableBadRow(invalidCount, event, errors) *> Monad[F].pure(Ior.Right(())) + case Invalid(errors) => + val allErrors = errors + .prepend("Enriched event does not conform to atomic schema field's length restrictions") + .map(asEnrichmentFailure) + Monad[F].pure(Ior.Both(allErrors, ())) + case Valid(()) => + Monad[F].pure(Ior.Right(())) + } } private def validateField( @@ -64,22 +65,6 @@ object AtomicFieldsLengthValidator { Valid(()) } - private def buildBadRow( - event: EnrichedEvent, - rawEvent: RawEvent, - processor: Processor, - errors: NonEmptyList[String] - ): BadRow.EnrichmentFailures = - EnrichmentManager.buildEnrichmentFailuresBadRow( - NonEmptyList( - asEnrichmentFailure("Enriched event does not conform to atomic schema field's length restrictions"), - errors.toList.map(asEnrichmentFailure) - ), - EnrichedEvent.toPartiallyEnrichedEvent(event), - RawEvent.toRawEvent(rawEvent), - processor - ) - private def handleAcceptableBadRow[F[_]: Monad]( invalidCount: F[Unit], event: EnrichedEvent, @@ -92,8 +77,8 @@ object AtomicFieldsLengthValidator { ) ) - private def asEnrichmentFailure(errorMessage: String): EnrichmentFailure = - EnrichmentFailure( + private def asEnrichmentFailure(errorMessage: String): FailureDetails.EnrichmentFailure = + FailureDetails.EnrichmentFailure( enrichment = None, FailureDetails.EnrichmentFailureMessage.Simple(errorMessage) ) diff --git a/modules/common/src/main/scala/com.snowplowanalytics.snowplow.enrich/common/enrichments/ClientEnrichments.scala b/modules/common/src/main/scala/com.snowplowanalytics.snowplow.enrich/common/enrichments/ClientEnrichments.scala index 506b4560a..8b6aded66 100644 --- a/modules/common/src/main/scala/com.snowplowanalytics.snowplow.enrich/common/enrichments/ClientEnrichments.scala +++ b/modules/common/src/main/scala/com.snowplowanalytics.snowplow.enrich/common/enrichments/ClientEnrichments.scala @@ -36,7 +36,7 @@ object ClientEnrichments { * @param res The packed string holding the screen dimensions * @return the ResolutionTuple or an error message, boxed in a Scalaz Validation */ - val extractViewDimensions: (String, String) => Either[FailureDetails.EnrichmentFailure, (JInteger, JInteger)] = + val extractViewDimensions: (String, String) => Either[FailureDetails.SchemaViolation, (JInteger, JInteger)] = (field, res) => (res match { case ResRegex(width, height) => @@ -45,12 +45,8 @@ object ClientEnrichments { .leftMap(_ => "could not be converted to java.lang.Integer s") case _ => s"does not conform to regex ${ResRegex.toString}".asLeft }).leftMap { msg => - val f = FailureDetails.EnrichmentFailureMessage.InputData( - field, - Option(res), - msg - ) - FailureDetails.EnrichmentFailure(None, f) + FailureDetails.SchemaViolation + .NotJson(field, Option(res), msg) } } 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 1d238fdc2..10a82e25c 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 @@ -15,8 +15,8 @@ import java.net.URI import java.time.Instant import org.joda.time.DateTime import io.circe.Json -import cats.{Applicative, Monad} -import cats.data.{EitherT, NonEmptyList, OptionT, StateT} +import cats.{Applicative, Functor, Monad} +import cats.data.{EitherT, Ior, IorT, NonEmptyList, OptionT, StateT} import cats.effect.Clock import cats.implicits._ @@ -54,7 +54,9 @@ object EnrichmentManager { * @param raw Canonical input event to enrich * @param featureFlags The feature flags available in the current version of Enrich * @param invalidCount Function to increment the count of invalid events - * @return Enriched event or bad row if a problem occured + * @return Right(EnrichedEvent) if everything went well. + * Left(BadRow) if something went wrong and incomplete events are not enabled. + * Both(BadRow, EnrichedEvent) if something went wrong but incomplete events are enabled. */ def enrichEvent[F[_]: Monad: Clock]( registry: EnrichmentRegistry[F], @@ -65,15 +67,21 @@ object EnrichmentManager { featureFlags: EtlPipeline.FeatureFlags, invalidCount: F[Unit], registryLookup: RegistryLookup[F], - atomicFields: AtomicFields - ): EitherT[F, BadRow, EnrichedEvent] = - for { - enriched <- EitherT.fromEither[F](setupEnrichedEvent(raw, etlTstamp, processor)) - extractResult <- IgluUtils.extractAndValidateInputJsons(enriched, client, raw, processor, registryLookup) - _ = { - ME.formatUnstructEvent(extractResult.unstructEvent).foreach(e => enriched.unstruct_event = e) - ME.formatContexts(extractResult.contexts).foreach(c => enriched.contexts = c) - } + atomicFields: AtomicFields, + emitIncomplete: Boolean + ): IorT[F, BadRow, EnrichedEvent] = { + val iorT: IorT[F, NonEmptyList[BadRow], EnrichedEvent] = for { + enriched <- IorT.pure[F, NonEmptyList[BadRow]](new EnrichedEvent) + extractResult <- mapAndValidateInput( + raw, + enriched, + etlTstamp, + processor, + client, + registryLookup + ) + .leftMap(NonEmptyList.one) + .possiblyExitingEarly(emitIncomplete) enrichmentsContexts <- runEnrichments( registry, processor, @@ -83,26 +91,55 @@ object EnrichmentManager { extractResult.unstructEvent, featureFlags.legacyEnrichmentOrder ) - _ = ME.formatContexts(enrichmentsContexts ::: extractResult.validationInfoContexts).foreach(c => enriched.derived_contexts = c) - _ <- IgluUtils - .validateEnrichmentsContexts[F](client, enrichmentsContexts, raw, processor, enriched, registryLookup) - _ <- EitherT.rightT[F, BadRow]( - anonIp(enriched, registry.anonIp).foreach(enriched.user_ipaddress = _) + .leftMap(NonEmptyList.one) + .possiblyExitingEarly(emitIncomplete) + _ <- validateEnriched( + enriched, + raw, + enrichmentsContexts, + extractResult.validationInfoContexts, + client, + processor, + registryLookup, + featureFlags.acceptInvalid, + invalidCount, + atomicFields ) - _ <- EitherT.rightT[F, BadRow] { - piiTransform(enriched, registry.piiPseudonymizer).foreach { pii => - enriched.pii = pii.asString - } - } - _ <- validateEnriched(enriched, raw, processor, featureFlags.acceptInvalid, invalidCount, atomicFields) + .leftMap(NonEmptyList.one) + .possiblyExitingEarly(emitIncomplete) } yield enriched + iorT.leftMap(_.last) + } + + def mapAndValidateInput[F[_]: Clock: Monad]( + raw: RawEvent, + enrichedEvent: EnrichedEvent, + etlTstamp: DateTime, + processor: Processor, + client: IgluCirceClient[F], + registryLookup: RegistryLookup[F] + ): IorT[F, BadRow, IgluUtils.EventExtractResult] = { + val iorT = for { + _ <- IorT.fromIor[F](setupEnrichedEvent(raw, enrichedEvent, etlTstamp, processor)) + extract <- IgluUtils.extractAndValidateInputJsons(enrichedEvent, client, registryLookup) + } yield extract + + iorT.leftMap { violations => + buildSchemaViolationsBadRow( + violations, + EnrichedEvent.toPartiallyEnrichedEvent(enrichedEvent), + RawEvent.toRawEvent(raw), + processor + ) + } + } + /** - * Run all the enrichments and aggregate the errors if any + * Run all the enrichments * @param enriched /!\ MUTABLE enriched event, mutated IN-PLACE /!\ - * @return List of contexts to attach to the enriched event if all the enrichments went well - * or [[BadRow.EnrichmentFailures]] if something wrong happened - * with at least one enrichment + * @return All the contexts produced by the enrichments are in the Right. + * All the errors are aggregated in the bad row in the Left. */ private def runEnrichments[F[_]: Monad]( registry: EnrichmentRegistry[F], @@ -112,26 +149,57 @@ object EnrichmentManager { inputContexts: List[SelfDescribingData[Json]], unstructEvent: Option[SelfDescribingData[Json]], legacyOrder: Boolean - ): EitherT[F, BadRow.EnrichmentFailures, List[SelfDescribingData[Json]]] = - EitherT { + ): IorT[F, BadRow, List[SelfDescribingData[Json]]] = + IorT { accState(registry, raw, inputContexts, unstructEvent, legacyOrder) .runS(Accumulation(enriched, Nil, Nil)) .map { case Accumulation(_, failures, contexts) => failures.toNel match { case Some(nel) => - buildEnrichmentFailuresBadRow( - nel, - EnrichedEvent.toPartiallyEnrichedEvent(enriched), - RawEvent.toRawEvent(raw), - processor - ).asLeft + Ior.both( + buildEnrichmentFailuresBadRow( + nel, + EnrichedEvent.toPartiallyEnrichedEvent(enriched), + RawEvent.toRawEvent(raw), + processor + ), + contexts + ) case None => - contexts.asRight + Ior.right(contexts) } } } + private def validateEnriched[F[_]: Clock: Monad]( + 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] = { + 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) + } yield () + + iorT.leftMap { failures => + buildEnrichmentFailuresBadRow( + failures, + EnrichedEvent.toPartiallyEnrichedEvent(enriched), + RawEvent.toRawEvent(raw), + processor + ) + } + } + private[enrichments] case class Accumulation( event: EnrichedEvent, errors: List[FailureDetails.EnrichmentFailure], @@ -217,6 +285,8 @@ object EnrichmentManager { _ <- geoLocation[F](registry.ipLookups) // Execute IP lookup enrichment _ <- sqlContexts // Derive some contexts with custom SQL Query enrichment _ <- apiContexts // Derive some contexts with custom API Request enrichment + _ <- anonIp[F](registry.anonIp) // Anonymize the IP + _ <- piiTransform[F](registry.piiPseudonymizer) // Run PII pseudonymization // format: on } yield () else @@ -243,18 +313,19 @@ object EnrichmentManager { _ <- registry.javascriptScript.traverse(getJsScript[F](_)) // Execute the JavaScript scripting enrichment _ <- sqlContexts // Derive some contexts with custom SQL Query enrichment _ <- apiContexts // Derive some contexts with custom API Request enrichment + _ <- anonIp[F](registry.anonIp) // Anonymize the IP + _ <- piiTransform[F](registry.piiPseudonymizer) // Run PII pseudonymization // format: on } yield () } - /** Create the mutable [[EnrichedEvent]] and initialize it. */ private def setupEnrichedEvent( raw: RawEvent, + e: EnrichedEvent, etlTstamp: DateTime, processor: Processor - ): Either[BadRow.EnrichmentFailures, EnrichedEvent] = { - val e = new EnrichedEvent() + ): Ior[NonEmptyList[FailureDetails.SchemaViolation], EnrichedEvent] = { e.event_id = EE.generateEventId() // May be updated later if we have an `eid` parameter e.v_collector = raw.source.name // May be updated later if we have a `cv` parameter e.v_etl = ME.etlVersion(processor) @@ -271,20 +342,11 @@ object EnrichmentManager { // Map/validate/transform input fields to enriched event fields val transformed = Transform.transform(raw, e) - (collectorTstamp |+| transformed) - .leftMap { enrichmentFailures => - EnrichmentManager.buildEnrichmentFailuresBadRow( - enrichmentFailures, - EnrichedEvent.toPartiallyEnrichedEvent(e), - RawEvent.toRawEvent(raw), - processor - ) - } - .as(e) - .toEither + (collectorTstamp |+| transformed).toIor + .putRight(e) } - def setCollectorTstamp(event: EnrichedEvent, timestamp: Option[DateTime]): Either[FailureDetails.EnrichmentFailure, Unit] = + def setCollectorTstamp(event: EnrichedEvent, timestamp: Option[DateTime]): Either[FailureDetails.SchemaViolation, Unit] = EE.formatCollectorTstamp(timestamp).map { t => event.collector_tstamp = t ().asRight @@ -418,12 +480,21 @@ object EnrichmentManager { result.sequence.bimap(NonEmptyList.one(_), _.toList) } - def anonIp(event: EnrichedEvent, anonIp: Option[AnonIpEnrichment]): Option[String] = - Option(event.user_ipaddress).map { ip => - anonIp match { - case Some(anon) => anon.anonymizeIp(ip) - case None => ip - } + def anonIp[F[_]: Applicative](anonIp: Option[AnonIpEnrichment]): EStateT[F, Unit] = + EStateT.fromEither { + case (event, _) => + anonIp match { + case Some(anon) => + Option(event.user_ipaddress) match { + case Some(ip) => + Option(anon.anonymizeIp(ip)).foreach(event.user_ipaddress = _) + Nil.asRight + case None => + Nil.asRight + } + case None => + Nil.asRight + } } def getUaUtils[F[_]: Applicative](userAgentUtils: Option[UserAgentUtilsEnrichment]): EStateT[F, Unit] = @@ -481,10 +552,11 @@ object EnrichmentManager { event.base_currency = currency.baseCurrency.getCode // Note that jBigDecimalToDouble is applied to either-valid-or-null event POJO // properties, so we don't expect any of these four vals to be a Failure - val trTax = CU.jBigDecimalToDouble("tr_tx", event.tr_tax).toValidatedNel - val tiPrice = CU.jBigDecimalToDouble("ti_pr", event.ti_price).toValidatedNel - val trTotal = CU.jBigDecimalToDouble("tr_tt", event.tr_total).toValidatedNel - val trShipping = CU.jBigDecimalToDouble("tr_sh", event.tr_shipping).toValidatedNel + val enrichmentName = "currency_conversion" + val trTax = CU.jBigDecimalToDouble("tr_tx", event.tr_tax, enrichmentName).toValidatedNel + val tiPrice = CU.jBigDecimalToDouble("ti_pr", event.ti_price, enrichmentName).toValidatedNel + val trTotal = CU.jBigDecimalToDouble("tr_tt", event.tr_total, enrichmentName).toValidatedNel + val trShipping = CU.jBigDecimalToDouble("tr_sh", event.tr_shipping, enrichmentName).toValidatedNel EitherT( (trTotal, trTax, trShipping, tiPrice) .mapN { @@ -745,10 +817,30 @@ object EnrichmentManager { } } - def piiTransform(event: EnrichedEvent, piiPseudonymizer: Option[PiiPseudonymizerEnrichment]): Option[SelfDescribingData[Json]] = - piiPseudonymizer.flatMap(_.transformer(event)) + def piiTransform[F[_]: Applicative](piiPseudonymizer: Option[PiiPseudonymizerEnrichment]): EStateT[F, Unit] = + EStateT.fromEither { + case (event, _) => + piiPseudonymizer match { + case Some(pseudonymizer) => + pseudonymizer.transformer(event).foreach(p => event.pii = p.asString) + Nil.asRight + case None => + Nil.asRight + } + } + + def buildSchemaViolationsBadRow( + vs: NonEmptyList[FailureDetails.SchemaViolation], + pee: Payload.PartiallyEnrichedEvent, + re: Payload.RawEvent, + processor: Processor + ): BadRow.SchemaViolations = + BadRow.SchemaViolations( + processor, + Failure.SchemaViolations(Instant.now(), vs), + Payload.EnrichmentPayload(pee, re) + ) - /** Build `BadRow.EnrichmentFailures` from a list of `FailureDetails.EnrichmentFailure`s */ def buildEnrichmentFailuresBadRow( fs: NonEmptyList[FailureDetails.EnrichmentFailure], pee: Payload.PartiallyEnrichedEvent, @@ -761,21 +853,17 @@ object EnrichmentManager { Payload.EnrichmentPayload(pee, re) ) - /** - * Validates enriched events against atomic schema. - * For now it's possible to accept enriched events that are not valid. - * See https://github.com/snowplow/enrich/issues/517#issuecomment-1033910690 - */ - private def validateEnriched[F[_]: Monad]( - enriched: EnrichedEvent, - raw: RawEvent, - processor: Processor, - acceptInvalid: Boolean, - invalidCount: F[Unit], - atomicFields: AtomicFields - ): EitherT[F, BadRow, Unit] = - EitherT { - //We're using static field's length validation. See more in https://github.com/snowplow/enrich/issues/608 - AtomicFieldsLengthValidator.validate[F](enriched, raw, processor, acceptInvalid, invalidCount, atomicFields) - } + private implicit class IorTOps[F[_], A, B](val iorT: IorT[F, A, B]) extends AnyVal { + + /** If the incomplete events feature is disabled, then convert a Both to a Left, so we don't waste time with next steps */ + def possiblyExitingEarly(emitIncomplete: Boolean)(implicit F: Functor[F]): IorT[F, A, B] = + if (emitIncomplete) iorT + else + IorT { + iorT.value.map { + case Ior.Both(bad, _) => Ior.Left(bad) + case other => other + } + } + } } diff --git a/modules/common/src/main/scala/com.snowplowanalytics.snowplow.enrich/common/enrichments/EventEnrichments.scala b/modules/common/src/main/scala/com.snowplowanalytics.snowplow.enrich/common/enrichments/EventEnrichments.scala index eaf10ae62..6c2408a00 100644 --- a/modules/common/src/main/scala/com.snowplowanalytics.snowplow.enrich/common/enrichments/EventEnrichments.scala +++ b/modules/common/src/main/scala/com.snowplowanalytics.snowplow.enrich/common/enrichments/EventEnrichments.scala @@ -47,22 +47,22 @@ object EventEnrichments { * @param Optional collectorTstamp * @return Validation boxing the result of making the timestamp Redshift-compatible */ - def formatCollectorTstamp(collectorTstamp: Option[DateTime]): Either[FailureDetails.EnrichmentFailure, String] = - (collectorTstamp match { + def formatCollectorTstamp(collectorTstamp: Option[DateTime]): Either[FailureDetails.SchemaViolation, String] = + collectorTstamp match { case None => - FailureDetails.EnrichmentFailureMessage - .InputData("collector_tstamp", None, "should be set") + FailureDetails.SchemaViolation + .NotJson("collector_tstamp", None, "not set") .asLeft case Some(t) => val formattedTimestamp = toTimestamp(t) if (formattedTimestamp.startsWith("-") || t.getYear > 9999 || t.getYear < 0) { val msg = s"formatted as $formattedTimestamp is not Redshift-compatible" - FailureDetails.EnrichmentFailureMessage - .InputData("collector_tstamp", t.toString.some, msg) + FailureDetails.SchemaViolation + .NotJson("collector_tstamp", t.toString.some, msg) .asLeft } else formattedTimestamp.asRight - }).leftMap(FailureDetails.EnrichmentFailure(None, _)) + } /** * Calculate the derived timestamp @@ -110,36 +110,53 @@ object EventEnrichments { } } + case class FormatError( + field: String, + value: Option[String], + msg: String + ) + + val extractTimestamp_ef: (String, String) => Either[FailureDetails.EnrichmentFailure, String] = + (field, tstamp) => + extractTimestamp(field, tstamp).leftMap { error => + val f = FailureDetails.EnrichmentFailureMessage.InputData( + error.field, + error.value, + error.msg + ) + FailureDetails.EnrichmentFailure(None, f) + } + + val extractTimestamp_sv: (String, String) => Either[FailureDetails.SchemaViolation, String] = + (field, tstamp) => + extractTimestamp(field, tstamp).leftMap(error => FailureDetails.SchemaViolation.NotJson(error.field, error.value, error.msg)) + /** * Extracts the timestamp from the format as laid out in the Tracker Protocol: * https://github.com/snowplow/snowplow/wiki/snowplow-tracker-protocol#wiki-common-params * @param tstamp The timestamp as stored in the Tracker Protocol * @return a Tuple of two Strings (date and time), or an error message if the format was invalid */ - val extractTimestamp: (String, String) => Either[FailureDetails.EnrichmentFailure, String] = + val extractTimestamp: (String, String) => Either[FormatError, String] = (field, tstamp) => try { val dt = new DateTime(tstamp.toLong) val timestampString = toTimestamp(dt) - if (timestampString.startsWith("-") || dt.getYear > 9999 || dt.getYear < 0) { - val msg = s"formatting as $timestampString is not Redshift-compatible" - val f = FailureDetails.EnrichmentFailureMessage.InputData( + if (timestampString.startsWith("-") || dt.getYear > 9999 || dt.getYear < 0) + FormatError( field, Option(tstamp), - msg - ) - FailureDetails.EnrichmentFailure(None, f).asLeft - } else + s"formatting as $timestampString is not Redshift-compatible" + ).asLeft + else timestampString.asRight } catch { case _: NumberFormatException => - val msg = "not in the expected format: ms since epoch" - val f = FailureDetails.EnrichmentFailureMessage.InputData( + FormatError( field, Option(tstamp), - msg - ) - FailureDetails.EnrichmentFailure(None, f).asLeft + "not in the expected format: ms since epoch" + ).asLeft } /** @@ -149,7 +166,7 @@ object EventEnrichments { * @param eventCode The event code * @return the event type, or an error message if not recognised, boxed in a Scalaz Validation */ - val extractEventType: (String, String) => Either[FailureDetails.EnrichmentFailure, String] = + val extractEventType: (String, String) => Either[FailureDetails.SchemaViolation, String] = (field, code) => code match { case "se" => "struct".asRight @@ -162,12 +179,9 @@ object EventEnrichments { case "pp" => "page_ping".asRight case _ => val msg = "not recognized as an event type" - val f = FailureDetails.EnrichmentFailureMessage.InputData( - field, - Option(code), - msg - ) - FailureDetails.EnrichmentFailure(None, f).asLeft + FailureDetails.SchemaViolation + .NotJson(field, Option(code), msg) + .asLeft } /** diff --git a/modules/common/src/main/scala/com.snowplowanalytics.snowplow.enrich/common/enrichments/MiscEnrichments.scala b/modules/common/src/main/scala/com.snowplowanalytics.snowplow.enrich/common/enrichments/MiscEnrichments.scala index aaa476eb8..73e71de4d 100644 --- a/modules/common/src/main/scala/com.snowplowanalytics.snowplow.enrich/common/enrichments/MiscEnrichments.scala +++ b/modules/common/src/main/scala/com.snowplowanalytics.snowplow.enrich/common/enrichments/MiscEnrichments.scala @@ -44,7 +44,7 @@ object MiscEnrichments { * @param platform The code for the platform generating this event. * @return a Scalaz ValidatedString. */ - val extractPlatform: (String, String) => Either[FailureDetails.EnrichmentFailure, String] = + val extractPlatform: (String, String) => Either[FailureDetails.SchemaViolation, String] = (field, platform) => platform match { case "web" => "web".asRight // Web, including Mobile Web @@ -58,16 +58,13 @@ object MiscEnrichments { case "headset" => "headset".asRight // AR/VR Headset case _ => val msg = "not recognized as a tracking platform" - val f = FailureDetails.EnrichmentFailureMessage.InputData( - field, - Option(platform), - msg - ) - FailureDetails.EnrichmentFailure(None, f).asLeft + FailureDetails.SchemaViolation + .NotJson(field, Option(platform), msg) + .asLeft } /** Make a String TSV safe */ - val toTsvSafe: (String, String) => Either[FailureDetails.EnrichmentFailure, String] = + val toTsvSafe: (String, String) => Either[FailureDetails.SchemaViolation, String] = (_, value) => CU.makeTsvSafe(value).asRight /** @@ -76,7 +73,7 @@ object MiscEnrichments { * Here we retrieve the first one as it is supposed to be the client one, c.f. * https://en.m.wikipedia.org/wiki/X-Forwarded-For#Format */ - val extractIp: (String, String) => Either[FailureDetails.EnrichmentFailure, String] = + val extractIp: (String, String) => Either[FailureDetails.SchemaViolation, String] = (_, value) => { val lastIp = Option(value).map(_.split("[,|, ]").head).orNull CU.makeTsvSafe(lastIp).asRight diff --git a/modules/common/src/main/scala/com.snowplowanalytics.snowplow.enrich/common/enrichments/Transform.scala b/modules/common/src/main/scala/com.snowplowanalytics.snowplow.enrich/common/enrichments/Transform.scala index 00474b069..7f5196c19 100644 --- a/modules/common/src/main/scala/com.snowplowanalytics.snowplow.enrich/common/enrichments/Transform.scala +++ b/modules/common/src/main/scala/com.snowplowanalytics.snowplow.enrich/common/enrichments/Transform.scala @@ -31,7 +31,7 @@ object Transform { * to "user_ipaddress" in the enriched event * @param enriched /!\ MUTABLE enriched event, mutated IN-PLACE /!\ */ - private[enrichments] def transform(raw: RawEvent, enriched: EnrichedEvent): ValidatedNel[FailureDetails.EnrichmentFailure, Unit] = { + private[enrichments] def transform(raw: RawEvent, enriched: EnrichedEvent): ValidatedNel[FailureDetails.SchemaViolation, Unit] = { val sourceMap: SourceMap = raw.parameters.collect { case (k, Some(v)) => (k, v) } val firstPassTransform = enriched.transform(sourceMap, firstPassTransformMap) val secondPassTransform = enriched.transform(sourceMap, secondPassTransformMap) @@ -56,9 +56,9 @@ object Transform { ("fp", (ME.toTsvSafe, "user_fingerprint")), ("vid", (CU.stringToJInteger2, "domain_sessionidx")), ("sid", (CU.validateUuid, "domain_sessionid")), - ("dtm", (EE.extractTimestamp, "dvce_created_tstamp")), - ("ttm", (EE.extractTimestamp, "true_tstamp")), - ("stm", (EE.extractTimestamp, "dvce_sent_tstamp")), + ("dtm", (EE.extractTimestamp_sv, "dvce_created_tstamp")), + ("ttm", (EE.extractTimestamp_sv, "true_tstamp")), + ("stm", (EE.extractTimestamp_sv, "dvce_sent_tstamp")), ("tna", (ME.toTsvSafe, "name_tracker")), ("tv", (ME.toTsvSafe, "v_tracker")), ("cv", (ME.toTsvSafe, "v_collector")), diff --git a/modules/common/src/main/scala/com.snowplowanalytics.snowplow.enrich/common/enrichments/registry/CrossNavigationEnrichment.scala b/modules/common/src/main/scala/com.snowplowanalytics.snowplow.enrich/common/enrichments/registry/CrossNavigationEnrichment.scala index 443075822..a2f8dd1e9 100644 --- a/modules/common/src/main/scala/com.snowplowanalytics.snowplow.enrich/common/enrichments/registry/CrossNavigationEnrichment.scala +++ b/modules/common/src/main/scala/com.snowplowanalytics.snowplow.enrich/common/enrichments/registry/CrossNavigationEnrichment.scala @@ -192,7 +192,7 @@ object CrossNavigationEnrichment extends ParseableEnrichment { private def extractTstamp(str: String): Either[FailureDetails.EnrichmentFailure, Option[String]] = str match { case "" => None.asRight - case s => EE.extractTimestamp("sp_dtm", s).map(_.some) + case s => EE.extractTimestamp_ef("sp_dtm", s).map(_.some) } /** diff --git a/modules/common/src/main/scala/com.snowplowanalytics.snowplow.enrich/common/utils/ConversionUtils.scala b/modules/common/src/main/scala/com.snowplowanalytics.snowplow.enrich/common/utils/ConversionUtils.scala index 39866edcb..70acf3ca9 100644 --- a/modules/common/src/main/scala/com.snowplowanalytics.snowplow.enrich/common/utils/ConversionUtils.scala +++ b/modules/common/src/main/scala/com.snowplowanalytics.snowplow.enrich/common/utils/ConversionUtils.scala @@ -158,19 +158,16 @@ object ConversionUtils { * @param str The String hopefully containing a UUID * @return either the original String, or an error String */ - val validateUuid: (String, String) => Either[FailureDetails.EnrichmentFailure, String] = + val validateUuid: (String, String) => Either[FailureDetails.SchemaViolation, String] = (field, str) => { def check(s: String)(u: UUID): Boolean = u != null && s.toLowerCase == u.toString val uuid = Try(UUID.fromString(str)).toOption.filter(check(str)) uuid match { case Some(_) => str.toLowerCase.asRight case None => - val f = FailureDetails.EnrichmentFailureMessage.InputData( - field, - Option(str), - "not a valid UUID" - ) - FailureDetails.EnrichmentFailure(None, f).asLeft + FailureDetails.SchemaViolation + .NotJson(field, Option(str), "not a valid UUID") + .asLeft } } @@ -179,17 +176,13 @@ object ConversionUtils { * @param str The String hopefully parseable as an integer * @return either the original String, or an error String */ - val validateInteger: (String, String) => Either[FailureDetails.EnrichmentFailure, String] = + val validateInteger: (String, String) => Either[FailureDetails.SchemaViolation, String] = (field, str) => { Either .catchNonFatal { str.toInt; str } .leftMap { _ => - val f = FailureDetails.EnrichmentFailureMessage.InputData( - field, - Option(str), - "not a valid integer" - ) - FailureDetails.EnrichmentFailure(None, f) + FailureDetails.SchemaViolation + .NotJson(field, Option(str), "not a valid integer") } } @@ -334,15 +327,11 @@ object ConversionUtils { } } - val stringToJInteger2: (String, String) => Either[FailureDetails.EnrichmentFailure, JInteger] = + val stringToJInteger2: (String, String) => Either[FailureDetails.SchemaViolation, JInteger] = (field, str) => stringToJInteger(str).leftMap { e => - val f = FailureDetails.EnrichmentFailureMessage.InputData( - field, - Option(str), - e - ) - FailureDetails.EnrichmentFailure(None, f) + FailureDetails.SchemaViolation + .NotJson(field, Option(str), e) } val stringToJBigDecimal: String => Either[String, JBigDecimal] = str => @@ -364,15 +353,11 @@ object ConversionUtils { .leftMap(e => s"cannot be converted to java.math.BigDecimal. Error : ${e.getMessage}") } - val stringToJBigDecimal2: (String, String) => Either[FailureDetails.EnrichmentFailure, JBigDecimal] = + val stringToJBigDecimal2: (String, String) => Either[FailureDetails.SchemaViolation, JBigDecimal] = (field, str) => stringToJBigDecimal(str).leftMap { e => - val f = FailureDetails.EnrichmentFailureMessage.InputData( - field, - Option(str), - e - ) - FailureDetails.EnrichmentFailure(None, f) + FailureDetails.SchemaViolation + .NotJson(field, Option(str), e) } /** @@ -384,7 +369,7 @@ object ConversionUtils { * @param field The name of the field we are validating. To use in our error message * @return either a failure or a String */ - val stringToDoubleLike: (String, String) => Either[FailureDetails.EnrichmentFailure, String] = + val stringToDoubleLike: (String, String) => Either[FailureDetails.SchemaViolation, String] = (field, str) => Either .catchNonFatal { @@ -398,11 +383,8 @@ object ConversionUtils { } .leftMap { _ => val msg = "cannot be converted to Double-like" - FailureDetails.EnrichmentFailure( - None, - FailureDetails.EnrichmentFailureMessage - .InputData(field, Option(str), msg) - ) + FailureDetails.SchemaViolation + .NotJson(field, Option(str), msg) } /** @@ -411,7 +393,7 @@ object ConversionUtils { * @param field The name of the field we are validating. To use in our error message * @return a Scalaz Validation, being either a Failure String or a Success Double */ - def stringToMaybeDouble(field: String, str: String): Either[FailureDetails.EnrichmentFailure, Option[Double]] = + def stringToMaybeDouble(field: String, str: String): Either[FailureDetails.SchemaViolation, Option[Double]] = Either .catchNonFatal { if (Option(str).isEmpty || str == "null") @@ -423,18 +405,27 @@ object ConversionUtils { } } .leftMap(_ => - FailureDetails.EnrichmentFailure( - None, - FailureDetails.EnrichmentFailureMessage.InputData( - field, - Option(str), - "cannot be converted to Double" - ) - ) + FailureDetails.SchemaViolation + .NotJson(field, Option(str), "cannot be converted to Double") ) /** Convert a java BigDecimal a Double */ - def jBigDecimalToDouble(field: String, f: JBigDecimal): Either[FailureDetails.EnrichmentFailure, Option[Double]] = + def jBigDecimalToDouble(field: String, f: JBigDecimal): Either[FailureDetails.SchemaViolation, Option[Double]] = + Either + .catchNonFatal { + Option(f).map(_.doubleValue) + } + .leftMap(_ => + FailureDetails.SchemaViolation + .NotJson(field, Option(f).map(_.toString), "cannot be converted to Double") + ) + + /** Convert a java BigDecimal a Double */ + def jBigDecimalToDouble( + field: String, + f: JBigDecimal, + enrichmentName: String + ): Either[FailureDetails.EnrichmentFailure, Option[Double]] = Either .catchNonFatal { Option(f).map(_.doubleValue) @@ -445,26 +436,20 @@ object ConversionUtils { FailureDetails.EnrichmentFailureMessage.InputData( field, Option(f).map(_.toString), - "cannot be converted to Double" + s"cannot be converted to Double ($enrichmentName)" ) ) ) /** Convert a Double to a java BigDecimal */ - def doubleToJBigDecimal(field: String, d: Option[Double]): Either[FailureDetails.EnrichmentFailure, Option[JBigDecimal]] = + def doubleToJBigDecimal(field: String, d: Option[Double]): Either[FailureDetails.SchemaViolation, Option[JBigDecimal]] = Either .catchNonFatal { d.map(dd => new JBigDecimal(dd)) } .leftMap(_ => - FailureDetails.EnrichmentFailure( - None, - FailureDetails.EnrichmentFailureMessage.InputData( - field, - d.map(_.toString), - "cannot be converted to java BigDecimal" - ) - ) + FailureDetails.SchemaViolation + .NotJson(field, d.map(_.toString), "cannot be converted to java BigDecimal") ) /** @@ -493,19 +478,16 @@ object ConversionUtils { * @param field The name of the field we are trying to process. To use in our error message * @return either a Failure String or a Success Byte */ - val stringToBooleanLikeJByte: (String, String) => Either[FailureDetails.EnrichmentFailure, JByte] = + val stringToBooleanLikeJByte: (String, String) => Either[FailureDetails.SchemaViolation, JByte] = (field, str) => str match { case "1" => (1.toByte: JByte).asRight case "0" => (0.toByte: JByte).asRight case _ => val msg = "cannot be converted to Boolean-like java.lang.Byte" - val f = FailureDetails.EnrichmentFailureMessage.InputData( - field, - Option(str), - msg - ) - FailureDetails.EnrichmentFailure(None, f).asLeft + FailureDetails.SchemaViolation + .NotJson(field, Option(str), msg) + .asLeft } /** 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 d18936818..98a156125 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 @@ -11,7 +11,7 @@ package com.snowplowanalytics.snowplow.enrich.common.utils import cats.Monad -import cats.data.{EitherT, NonEmptyList, Validated, ValidatedNel} +import cats.data.{EitherT, Ior, IorT, NonEmptyList} import cats.effect.Clock import cats.implicits._ @@ -19,8 +19,6 @@ import io.circe._ import io.circe.syntax._ import io.circe.generic.semiauto._ -import java.time.Instant - import com.snowplowanalytics.iglu.client.{ClientError, IgluCirceClient} import com.snowplowanalytics.iglu.client.resolver.registries.RegistryLookup @@ -29,9 +27,8 @@ import com.snowplowanalytics.iglu.core.circe.implicits._ import com.snowplowanalytics.snowplow.badrows._ +import com.snowplowanalytics.snowplow.enrich.common.enrichments.{MiscEnrichments => ME} import com.snowplowanalytics.snowplow.enrich.common.outputs.EnrichedEvent -import com.snowplowanalytics.snowplow.enrich.common.enrichments.EnrichmentManager -import com.snowplowanalytics.snowplow.enrich.common.adapters.RawEvent /** * Contain the functions to validate: @@ -48,43 +45,26 @@ object IgluUtils { * @param client Iglu client used to validate the SDJs * @param raw Raw input event, used only to put in the bad row in case of problem * @param processor Meta data to put in the bad row - * @return Extracted unstructured event and input contexts if any and if everything valid, - * `BadRow.SchemaViolations` if something went wrong. For instance if the - * unstructured event is invalid and has a context that is invalid, - * the bad row will contain the 2 associated `FailureDetails.SchemaViolation`s + * @return Every SDJ that is invalid is in the Left part of the Ior + * while everything that is valid is in the Right part. */ def extractAndValidateInputJsons[F[_]: Monad: Clock]( enriched: EnrichedEvent, client: IgluCirceClient[F], - raw: RawEvent, - processor: Processor, registryLookup: RegistryLookup[F] - ): EitherT[ - F, - BadRow.SchemaViolations, - EventExtractResult - ] = - EitherT { - for { - contexts <- IgluUtils.extractAndValidateInputContexts(enriched, client, registryLookup) - unstruct <- IgluUtils - .extractAndValidateUnstructEvent(enriched, client, registryLookup) - .map(_.toValidatedNel) - } yield (contexts, unstruct) - .mapN { (c, ue) => - val validationInfoContexts = (c.flatMap(_.validationInfo) ::: ue.flatMap(_.validationInfo).toList).distinct - .map(_.toSdj) - EventExtractResult(contexts = c.map(_.sdj), unstructEvent = ue.map(_.sdj), validationInfoContexts = validationInfoContexts) - } - .leftMap { schemaViolations => - buildSchemaViolationsBadRow( - schemaViolations, - EnrichedEvent.toPartiallyEnrichedEvent(enriched), - RawEvent.toRawEvent(raw), - processor - ) - } - .toEither + ): IorT[F, NonEmptyList[FailureDetails.SchemaViolation], EventExtractResult] = + for { + contexts <- extractAndValidateInputContexts(enriched, client, registryLookup) + unstruct <- extractAndValidateUnstructEvent(enriched, client, registryLookup) + } yield { + ME.formatContexts(contexts.map(_.sdj)).foreach(c => enriched.contexts = c) + ME.formatUnstructEvent(unstruct.map(_.sdj)).foreach(e => enriched.unstruct_event = e) + val validationInfoContexts = (contexts.flatMap(_.validationInfo) ::: unstruct.flatMap(_.validationInfo).toList).distinct + .map(_.toSdj) + EventExtractResult(contexts = contexts.map(_.sdj), + unstructEvent = unstruct.map(_.sdj), + validationInfoContexts = validationInfoContexts + ) } /** @@ -102,18 +82,21 @@ object IgluUtils { registryLookup: RegistryLookup[F], field: String = "ue_properties", criterion: SchemaCriterion = SchemaCriterion("com.snowplowanalytics.snowplow", "unstruct_event", "jsonschema", 1, 0) - ): F[Validated[FailureDetails.SchemaViolation, Option[SdjExtractResult]]] = - (Option(enriched.unstruct_event) match { + ): IorT[F, NonEmptyList[FailureDetails.SchemaViolation], Option[SdjExtractResult]] = + Option(enriched.unstruct_event) match { case Some(rawUnstructEvent) => - for { + val iorT = for { // Validate input Json string and extract unstructured event unstruct <- extractInputData(rawUnstructEvent, field, criterion, client, registryLookup) + .leftMap(NonEmptyList.one) + .toIor // Parse Json unstructured event as SelfDescribingData[Json] - unstructSDJ <- parseAndValidateSDJ_sv(unstruct, client, registryLookup) + unstructSDJ <- parseAndValidateSDJ(unstruct, client, registryLookup) } yield unstructSDJ.some + iorT.recoverWith { case errors => IorT.fromIor[F](Ior.Both(errors, None)) } case None => - EitherT.rightT[F, FailureDetails.SchemaViolation](none[SdjExtractResult]) - }).toValidated + IorT.rightT[F, NonEmptyList[FailureDetails.SchemaViolation]](none[SdjExtractResult]) + } /** * Extract list of custom contexts from event and validate each against its schema @@ -122,7 +105,7 @@ object IgluUtils { * @param field Name of the field containing the contexts, to put in the bad row * in case of failure * @param criterion Expected schema for the JSON containing the contexts - * @return List will all contexts provided that they are all valid + * @return All valid contexts are in the Right while all errors are in the Left */ private[common] def extractAndValidateInputContexts[F[_]: Monad: Clock]( enriched: EnrichedEvent, @@ -130,27 +113,28 @@ object IgluUtils { registryLookup: RegistryLookup[F], field: String = "contexts", criterion: SchemaCriterion = SchemaCriterion("com.snowplowanalytics.snowplow", "contexts", "jsonschema", 1, 0) - ): F[ValidatedNel[FailureDetails.SchemaViolation, List[SdjExtractResult]]] = - (Option(enriched.contexts) match { + ): IorT[F, NonEmptyList[FailureDetails.SchemaViolation], List[SdjExtractResult]] = + Option(enriched.contexts) match { case Some(rawContexts) => - for { + val iorT = for { // Validate input Json string and extract contexts contexts <- extractInputData(rawContexts, field, criterion, client, registryLookup) .map(_.asArray.get.toList) // .get OK because SDJ wrapping the contexts valid .leftMap(NonEmptyList.one) + .toIor // Parse and validate each SDJ and merge the errors - contextsSDJ <- EitherT( - contexts - .map(parseAndValidateSDJ_sv(_, client, registryLookup).toValidatedNel) - .sequence - .map(_.sequence.toEither) - ) - } yield contextsSDJ + contextsSdj <- contexts + .traverse( + parseAndValidateSDJ(_, client, registryLookup) + .map(sdj => List(sdj)) + .recoverWith { case errors => IorT.fromIor[F](Ior.Both(errors, Nil)) } + ) + .map(_.flatten) + } yield contextsSdj + iorT.recoverWith { case errors => IorT.fromIor[F](Ior.Both(errors, Nil)) } case None => - EitherT.rightT[F, NonEmptyList[FailureDetails.SchemaViolation]]( - List.empty[SdjExtractResult] - ) - }).toValidated + IorT.rightT[F, NonEmptyList[FailureDetails.SchemaViolation]](Nil) + } /** * Validate each context added by the enrichments against its schema @@ -159,16 +143,13 @@ object IgluUtils { * @param raw Input event to put in the bad row if at least one context is invalid * @param processor Meta data for the bad row * @param enriched Partially enriched event to put in the bad row - * @return Unit if all the contexts are valid + * @return All valid contexts are in the Right while all errors are in the Left */ private[common] def validateEnrichmentsContexts[F[_]: Monad: Clock]( client: IgluCirceClient[F], sdjs: List[SelfDescribingData[Json]], - raw: RawEvent, - processor: Processor, - enriched: EnrichedEvent, registryLookup: RegistryLookup[F] - ): EitherT[F, BadRow.EnrichmentFailures, Unit] = + ): IorT[F, NonEmptyList[FailureDetails.EnrichmentFailure], List[SelfDescribingData[Json]]] = checkList(client, sdjs, registryLookup) .leftMap( _.map { @@ -181,14 +162,6 @@ object IgluUtils { ) } ) - .leftMap { enrichmentFailures => - EnrichmentManager.buildEnrichmentFailuresBadRow( - enrichmentFailures, - EnrichedEvent.toPartiallyEnrichedEvent(enriched), - RawEvent.toRawEvent(raw), - processor - ) - } /** Used to extract .data for input custom contexts and input unstructured event */ private def extractInputData[F[_]: Monad: Clock]( @@ -243,30 +216,34 @@ object IgluUtils { .leftMap((sdj.schema, _)) } - /** Check a list of SDJs and merge the Iglu errors */ + /** + * Check a list of SDJs. + * @return All valid SDJs are in the Right while all errors are in the Left + */ private def checkList[F[_]: Monad: Clock]( client: IgluCirceClient[F], sdjs: List[SelfDescribingData[Json]], registryLookup: RegistryLookup[F] - ): EitherT[F, NonEmptyList[(SchemaKey, ClientError)], Unit] = - EitherT { - sdjs - .map(check(client, _, registryLookup).toValidatedNel) - .sequence - .map(_.sequence_.toEither) - } + ): IorT[F, NonEmptyList[(SchemaKey, ClientError)], List[SelfDescribingData[Json]]] = + sdjs.map { sdj => + check(client, sdj, registryLookup) + .map(_ => List(sdj)) + .leftMap(NonEmptyList.one) + .toIor + .recoverWith { case errors => IorT.fromIor[F](Ior.Both(errors, Nil)) } + }.foldA /** Parse a Json as a SDJ and check that it's valid */ - private def parseAndValidateSDJ_sv[F[_]: Monad: Clock]( // _sv for SchemaViolation + private def parseAndValidateSDJ[F[_]: Monad: Clock]( json: Json, client: IgluCirceClient[F], registryLookup: RegistryLookup[F] - ): EitherT[F, FailureDetails.SchemaViolation, SdjExtractResult] = + ): IorT[F, NonEmptyList[FailureDetails.SchemaViolation], SdjExtractResult] = for { - sdj <- SelfDescribingData - .parse(json) - .leftMap(FailureDetails.SchemaViolation.NotIglu(json, _)) - .toEitherT[F] + sdj <- IorT + .fromEither[F](SelfDescribingData.parse(json)) + .leftMap[FailureDetails.SchemaViolation](FailureDetails.SchemaViolation.NotIglu(json, _)) + .leftMap(NonEmptyList.one) supersedingSchema <- check(client, sdj, registryLookup) .leftMap { case (schemaKey, clientError) => @@ -274,6 +251,8 @@ object IgluUtils { .IgluError(schemaKey, clientError): FailureDetails.SchemaViolation } + .leftMap(NonEmptyList.one) + .toIor validationInfo = supersedingSchema.map(s => ValidationInfo(sdj.schema, s)) sdjUpdated = replaceSchemaVersion(sdj, validationInfo) } yield SdjExtractResult(sdjUpdated, validationInfo) @@ -309,17 +288,4 @@ object IgluUtils { unstructEvent: Option[SelfDescribingData[Json]], validationInfoContexts: List[SelfDescribingData[Json]] ) - - /** Build `BadRow.SchemaViolations` from a list of `FailureDetails.SchemaViolation`s */ - def buildSchemaViolationsBadRow( - vs: NonEmptyList[FailureDetails.SchemaViolation], - pee: Payload.PartiallyEnrichedEvent, - re: Payload.RawEvent, - processor: Processor - ): BadRow.SchemaViolations = - BadRow.SchemaViolations( - processor, - Failure.SchemaViolations(Instant.now(), vs), - Payload.EnrichmentPayload(pee, re) - ) } diff --git a/modules/common/src/main/scala/com.snowplowanalytics.snowplow.enrich/common/utils/JsonUtils.scala b/modules/common/src/main/scala/com.snowplowanalytics.snowplow.enrich/common/utils/JsonUtils.scala index 680286044..1af0b4a7f 100644 --- a/modules/common/src/main/scala/com.snowplowanalytics.snowplow.enrich/common/utils/JsonUtils.scala +++ b/modules/common/src/main/scala/com.snowplowanalytics.snowplow.enrich/common/utils/JsonUtils.scala @@ -31,30 +31,16 @@ object JsonUtils { private val JsonSchemaDateTimeFormat = DateTimeFormat.forPattern("yyyy-MM-dd'T'HH:mm:ss.SSS'Z'").withZone(DateTimeZone.UTC) - /** Validates a String as correct JSON. */ - val extractUnencJson: (String, String) => Either[FailureDetails.EnrichmentFailure, String] = - (field, str) => - validateAndReformatJson(str) - .leftMap { e => - FailureDetails.EnrichmentFailure( - None, - FailureDetails.EnrichmentFailureMessage - .InputData(field, Option(str), e) - ) - } + val extractUnencJson: (String, String) => Either[FailureDetails.SchemaViolation, String] = + (_, str) => Right(str) - /** Decodes a Base64 (URL safe)-encoded String then validates it as correct JSON. */ - val extractBase64EncJson: (String, String) => Either[FailureDetails.EnrichmentFailure, String] = + val extractBase64EncJson: (String, String) => Either[FailureDetails.SchemaViolation, String] = (field, str) => ConversionUtils .decodeBase64Url(str) - .flatMap(validateAndReformatJson) .leftMap { e => - FailureDetails.EnrichmentFailure( - None, - FailureDetails.EnrichmentFailureMessage - .InputData(field, Option(str), e) - ) + FailureDetails.SchemaViolation + .NotJson(field, Option(str), e) } /** @@ -131,16 +117,6 @@ object JsonUtils { (key, v) } - /** - * Validates and reformats a JSON: - * 1. Checks the JSON is valid - * 2. Reformats, including removing unnecessary whitespace - * @param str the String hopefully containing JSON - * @return either an error String or the reformatted JSON String - */ - private[utils] def validateAndReformatJson(str: String): Either[String, String] = - extractJson(str).map(_.noSpaces) - /** * Converts a JSON string into an EIther[String, Json] * @param instance The JSON string to parse diff --git a/modules/common/src/main/scala/com.snowplowanalytics.snowplow.enrich/common/utils/MapTransformer.scala b/modules/common/src/main/scala/com.snowplowanalytics.snowplow.enrich/common/utils/MapTransformer.scala index 6bc76be1c..6d2d570ca 100644 --- a/modules/common/src/main/scala/com.snowplowanalytics.snowplow.enrich/common/utils/MapTransformer.scala +++ b/modules/common/src/main/scala/com.snowplowanalytics.snowplow.enrich/common/utils/MapTransformer.scala @@ -66,7 +66,7 @@ object MapTransformer { type Field = String // A transformation takes a Key and Value and returns either a failure or anything - type TransformFunc = Function2[Key, Value, Either[FailureDetails.EnrichmentFailure, _]] + type TransformFunc = Function2[Key, Value, Either[FailureDetails.SchemaViolation, _]] // Our source map type SourceMap = Map[Key, Value] @@ -88,7 +88,7 @@ object MapTransformer { transformMap: TransformMap )( implicit m: Manifest[T] - ): ValidatedNel[FailureDetails.EnrichmentFailure, T] = { + ): ValidatedNel[FailureDetails.SchemaViolation, T] = { val newInst = m.runtimeClass.getDeclaredConstructor().newInstance() val result = _transform(newInst, sourceMap, transformMap, getSetters(m.runtimeClass)) // On success, replace the field count with the new instance @@ -116,7 +116,7 @@ object MapTransformer { * @param transformMap Determines how the data should be transformed before storing in the obj * @return a ValidationNel containing a Nel of error Strings, or the count of updated fields */ - def transform(sourceMap: SourceMap, transformMap: TransformMap): ValidatedNel[FailureDetails.EnrichmentFailure, Int] = + def transform(sourceMap: SourceMap, transformMap: TransformMap): ValidatedNel[FailureDetails.SchemaViolation, Int] = _transform[T](obj, sourceMap, transformMap, setters) } @@ -134,8 +134,8 @@ object MapTransformer { sourceMap: SourceMap, transformMap: TransformMap, setters: SettersMap - ): ValidatedNel[FailureDetails.EnrichmentFailure, Int] = { - val results: List[Either[FailureDetails.EnrichmentFailure, Int]] = sourceMap.map { + ): ValidatedNel[FailureDetails.SchemaViolation, Int] = { + val results: List[Either[FailureDetails.SchemaViolation, Int]] = sourceMap.map { case (key, in) => transformMap.get(key) match { case Some((func, field)) => @@ -172,7 +172,7 @@ object MapTransformer { } }.toList - results.foldLeft(0.validNel[FailureDetails.EnrichmentFailure]) { + results.foldLeft(0.validNel[FailureDetails.SchemaViolation]) { case (acc, e) => acc.combine(e.toValidatedNel) } diff --git a/modules/common/src/test/scala/com.snowplowanalytics.snowplow.enrich.common/EtlPipelineSpec.scala b/modules/common/src/test/scala/com.snowplowanalytics.snowplow.enrich.common/EtlPipelineSpec.scala index 612236d3e..b4baad14f 100644 --- a/modules/common/src/test/scala/com.snowplowanalytics.snowplow.enrich.common/EtlPipelineSpec.scala +++ b/modules/common/src/test/scala/com.snowplowanalytics.snowplow.enrich.common/EtlPipelineSpec.scala @@ -10,8 +10,8 @@ */ package com.snowplowanalytics.snowplow.enrich.common -import cats.data.Validated import cats.syntax.validated._ +import cats.data.Ior import cats.effect.IO import cats.effect.testing.specs2.CatsEffect @@ -36,7 +36,6 @@ import com.snowplowanalytics.snowplow.enrich.common.adapters.AdapterRegistry import com.snowplowanalytics.snowplow.enrich.common.adapters.registry.RemoteAdapter import com.snowplowanalytics.snowplow.enrich.common.enrichments.{AtomicFields, EnrichmentRegistry} import com.snowplowanalytics.snowplow.enrich.common.loaders.{CollectorPayload, ThriftLoader} -import com.snowplowanalytics.snowplow.enrich.common.outputs.EnrichedEvent import com.snowplowanalytics.snowplow.enrich.common.SpecHelpers._ @@ -73,11 +72,12 @@ class EtlPipelineSpec extends Specification with ValidatedMatchers with CatsEffe AcceptInvalid.featureFlags, IO.unit, SpecHelpers.registryLookup, - AtomicFields.from(Map.empty) + AtomicFields.from(Map.empty), + emitIncomplete ) } yield output must be like { - case a :: b :: c :: d :: Nil => - (a must beValid).and(b must beInvalid).and(c must beInvalid).and(d must beInvalid) + case Ior.Right(_) :: Ior.Left(_) :: Ior.Left(_) :: Ior.Left(_) :: Nil => ok + case other => ko(s"[$other] is not a list with 1 enriched event and 3 bad rows") } def e2 = @@ -100,11 +100,12 @@ class EtlPipelineSpec extends Specification with ValidatedMatchers with CatsEffe AcceptInvalid.featureFlags, IO.unit, SpecHelpers.registryLookup, - AtomicFields.from(Map.empty) + AtomicFields.from(Map.empty), + emitIncomplete ) - } yield output must beLike { - case Validated.Valid(_: EnrichedEvent) :: Nil => ok - case res => ko(s"[$res] doesn't contain one enriched event") + } yield output must be like { + case Ior.Right(_) :: Nil => ok + case other => ko(s"[$other] is not a list with 1 enriched event") } def e3 = @@ -122,11 +123,12 @@ class EtlPipelineSpec extends Specification with ValidatedMatchers with CatsEffe AcceptInvalid.featureFlags, IO.unit, SpecHelpers.registryLookup, - AtomicFields.from(Map.empty) + AtomicFields.from(Map.empty), + emitIncomplete ) } yield output must be like { - case Validated.Invalid(_: BadRow.CPFormatViolation) :: Nil => ok - case other => ko(s"One invalid CPFormatViolation expected, got ${other}") + case Ior.Left(_: BadRow.CPFormatViolation) :: Nil => ok + case other => ko(s"[$other] is not a CPFormatViolation bad row") } def e4 = @@ -144,7 +146,8 @@ class EtlPipelineSpec extends Specification with ValidatedMatchers with CatsEffe AcceptInvalid.featureFlags, IO.unit, SpecHelpers.registryLookup, - AtomicFields.from(Map.empty) + AtomicFields.from(Map.empty), + emitIncomplete ) } yield output must beEqualTo(Nil) } 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 ae018a13c..f75ff340b 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 @@ -148,6 +148,8 @@ object SpecHelpers extends CatsEffect { def createIgluClient(registries: List[Registry]): IO[IgluCirceClient[IO]] = IgluCirceClient.fromResolver[IO](Resolver(registries, None), cacheSize = 0) + val emitIncomplete = false + val callrailSchemas = CallrailSchemas( call_complete = "iglu:com.callrail/call_complete/jsonschema/1-0-2" ) 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 d598ca932..43b23cc22 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 @@ -19,7 +19,7 @@ import cats.effect.IO import cats.effect.testing.specs2.CatsEffect import cats.implicits._ -import cats.data.NonEmptyList +import cats.data.{Ior, NonEmptyList} import io.circe.Json import io.circe.literal._ @@ -90,13 +90,13 @@ class EnrichmentManagerSpec extends Specification with EitherMatchers with CatsE AcceptInvalid.featureFlags, IO.unit, SpecHelpers.registryLookup, - atomicFieldLimits + atomicFieldLimits, + emitIncomplete ) - enriched.value - .map(_ must beLeft.like { - case _: BadRow.SchemaViolations => ok - case br => ko(s"bad row [$br] is not SchemaViolations") - }) + enriched.value map { + case Ior.Left(_: BadRow.SchemaViolations) => ok + case other => ko(s"[$other] is not a SchemaViolations bad row") + } } "return a SchemaViolations bad row if the input unstructured event is invalid" >> { @@ -128,13 +128,13 @@ class EnrichmentManagerSpec extends Specification with EitherMatchers with CatsE AcceptInvalid.featureFlags, IO.unit, SpecHelpers.registryLookup, - atomicFieldLimits + atomicFieldLimits, + emitIncomplete ) - enriched.value - .map(_ must beLeft.like { - case _: BadRow.SchemaViolations => ok - case br => ko(s"bad row [$br] is not SchemaViolations") - }) + enriched.value map { + case Ior.Left(_: BadRow.SchemaViolations) => ok + case other => ko(s"[$other] is not a SchemaViolations bad row") + } } "return an EnrichmentFailures bad row if one of the enrichment (JS enrichment here) fails" >> { @@ -177,11 +177,12 @@ class EnrichmentManagerSpec extends Specification with EitherMatchers with CatsE AcceptInvalid.featureFlags, IO.unit, SpecHelpers.registryLookup, - atomicFieldLimits + atomicFieldLimits, + emitIncomplete ) - enriched.value - .map(_ must beLeft.like { - case BadRow.EnrichmentFailures( + enriched.value map { + case Ior.Left( + BadRow.EnrichmentFailures( _, Failure.EnrichmentFailures( _, @@ -194,13 +195,11 @@ class EnrichmentManagerSpec extends Specification with EitherMatchers with CatsE ) ), _ - ) => - ok - case br => - ko( - s"bad row [$br] is not an EnrichmentFailures containing one EnrichmentFailureMessage.Simple" - ) - }) + ) + ) => + ok + case other => ko(s"[$other] is not an EnrichmentFailures bad row with one EnrichmentFailureMessage.Simple") + } } "return an EnrichmentFailures bad row containing one IgluError if one of the contexts added by the enrichments is invalid" >> { @@ -247,11 +246,12 @@ class EnrichmentManagerSpec extends Specification with EitherMatchers with CatsE AcceptInvalid.featureFlags, IO.unit, SpecHelpers.registryLookup, - atomicFieldLimits + atomicFieldLimits, + emitIncomplete ) - enriched.value - .map(_ must beLeft.like { - case BadRow.EnrichmentFailures( + enriched.value map { + case Ior.Left( + BadRow.EnrichmentFailures( _, Failure.EnrichmentFailures( _, @@ -263,11 +263,12 @@ class EnrichmentManagerSpec extends Specification with EitherMatchers with CatsE Nil ) ), - payload - ) if payload.enriched.derived_contexts.isDefined => - ok - case br => ko(s"bad row [$br] is not an EnrichmentFailures containing one IgluError and with derived_contexts defined") - }) + _ + ) + ) => + ok + case other => ko(s"[$other] is not an EnrichmentFailures bad row with one IgluError") + } } "emit an EnrichedEvent if everything goes well" >> { @@ -313,9 +314,13 @@ class EnrichmentManagerSpec extends Specification with EitherMatchers with CatsE AcceptInvalid.featureFlags, IO.unit, SpecHelpers.registryLookup, - atomicFieldLimits + atomicFieldLimits, + emitIncomplete ) - enriched.value.map(_ must beRight) + enriched.value.map { + case Ior.Right(_) => ok + case other => ko(s"[$other] is not an enriched event") + } } "emit an EnrichedEvent if a PII value that needs to be hashed is an empty string" >> { @@ -379,9 +384,13 @@ class EnrichmentManagerSpec extends Specification with EitherMatchers with CatsE AcceptInvalid.featureFlags, IO.unit, SpecHelpers.registryLookup, - atomicFieldLimits + atomicFieldLimits, + emitIncomplete ) - enriched.value.map(_ must beRight) + enriched.value.map { + case Ior.Right(_) => ok + case other => ko(s"[$other] is not an enriched event") + } } "emit an EnrichedEvent if a PII value that needs to be hashed is null" >> { @@ -445,9 +454,13 @@ class EnrichmentManagerSpec extends Specification with EitherMatchers with CatsE AcceptInvalid.featureFlags, IO.unit, SpecHelpers.registryLookup, - atomicFieldLimits + atomicFieldLimits, + emitIncomplete ) - enriched.value.map(_ must beRight) + enriched.value.map { + case Ior.Right(_) => ok + case other => ko(s"[$other] is not an enriched event") + } } "fail to emit an EnrichedEvent if a PII value that needs to be hashed is an empty object" >> { @@ -511,9 +524,13 @@ class EnrichmentManagerSpec extends Specification with EitherMatchers with CatsE AcceptInvalid.featureFlags, IO.unit, SpecHelpers.registryLookup, - atomicFieldLimits + atomicFieldLimits, + emitIncomplete ) - enriched.value.map(_ must beLeft) + enriched.value.map { + case Ior.Left(_) => ok + case other => ko(s"[$other] is not a bad row") + } } "fail to emit an EnrichedEvent if a context PII value that needs to be hashed is an empty object" >> { @@ -577,9 +594,13 @@ class EnrichmentManagerSpec extends Specification with EitherMatchers with CatsE AcceptInvalid.featureFlags, IO.unit, SpecHelpers.registryLookup, - atomicFieldLimits + atomicFieldLimits, + emitIncomplete ) - enriched.value.map(_ must beLeft) + enriched.value.map { + case Ior.Left(_) => ok + case other => ko(s"[$other] is not a bad row") + } } "fail to emit an EnrichedEvent if a PII value needs to be hashed in both co and ue and is invalid in one of them" >> { @@ -649,9 +670,13 @@ class EnrichmentManagerSpec extends Specification with EitherMatchers with CatsE AcceptInvalid.featureFlags, IO.unit, SpecHelpers.registryLookup, - atomicFieldLimits + atomicFieldLimits, + emitIncomplete ) - enriched.value.map(_ must beLeft) + enriched.value.map { + case Ior.Left(_) => ok + case other => ko(s"[$other] is not a bad row") + } } "emit an EnrichedEvent for valid integer fields" >> { @@ -677,9 +702,13 @@ class EnrichmentManagerSpec extends Specification with EitherMatchers with CatsE AcceptInvalid.featureFlags, IO.unit, SpecHelpers.registryLookup, - atomicFieldLimits + atomicFieldLimits, + emitIncomplete ) - enriched.value.map(_ must beRight) + enriched.value.map { + case Ior.Right(_) => ok + case other => ko(s"[$other] is not a bad row") + } } } } @@ -707,9 +736,13 @@ class EnrichmentManagerSpec extends Specification with EitherMatchers with CatsE AcceptInvalid.featureFlags, IO.unit, SpecHelpers.registryLookup, - atomicFieldLimits + atomicFieldLimits, + emitIncomplete ) - enriched.value.map(_ must beRight) + enriched.value.map { + case Ior.Right(_) => ok + case other => ko(s"[$other] is not an enriched event") + } } } } @@ -746,11 +779,13 @@ class EnrichmentManagerSpec extends Specification with EitherMatchers with CatsE AcceptInvalid.featureFlags, IO.unit, SpecHelpers.registryLookup, - atomicFieldLimits + atomicFieldLimits, + emitIncomplete ) - enriched.value.map(_ must beRight { ee: EnrichedEvent => - ee.se_value.toString must_== expected - }) + enriched.value.map { + case Ior.Right(enriched) => enriched.se_value.toString must_== expected + case other => ko(s"[$other] is not an enriched event") + } } } @@ -773,12 +808,15 @@ class EnrichmentManagerSpec extends Specification with EitherMatchers with CatsE AcceptInvalid.featureFlags, IO.unit, SpecHelpers.registryLookup, - atomicFieldLimits + atomicFieldLimits, + emitIncomplete ) - enriched.value.map { e => - val res1 = e.map(_.useragent) must beRight(qs_ua) - val res2 = e.map(_.derived_contexts) must beRight((_: String).contains("\"agentName\":\"Firefox\"")) - res1 and res2 + enriched.value.map { + case Ior.Right(enriched) => + val res1 = enriched.useragent must_== qs_ua + val res2 = enriched.derived_contexts must contain("\"agentName\":\"Firefox\"") + res1 and res2 + case other => ko(s"[$other] is not an enriched event") } } @@ -799,10 +837,12 @@ class EnrichmentManagerSpec extends Specification with EitherMatchers with CatsE AcceptInvalid.featureFlags, IO.unit, SpecHelpers.registryLookup, - atomicFieldLimits + atomicFieldLimits, + emitIncomplete ) - enriched.value.map { e => - e.map(_.useragent) must beRight("header-useragent") + enriched.value.map { + case Ior.Right(enriched) => enriched.useragent must_== "header-useragent" + case other => ko(s"[$other] is not an enriched event") } } @@ -824,10 +864,12 @@ class EnrichmentManagerSpec extends Specification with EitherMatchers with CatsE AcceptInvalid.featureFlags, IO.unit, SpecHelpers.registryLookup, - atomicFieldLimits + atomicFieldLimits, + emitIncomplete ) - enriched.value.map { e => - e.map(_.useragent) must beRight(ua) + enriched.value.map { + case Ior.Right(enriched) => enriched.useragent must_== ua + case other => ko(s"[$other] is not an enriched event") } } @@ -850,12 +892,15 @@ class EnrichmentManagerSpec extends Specification with EitherMatchers with CatsE AcceptInvalid.featureFlags, IO.unit, SpecHelpers.registryLookup, - atomicFieldLimits + atomicFieldLimits, + emitIncomplete ) - enriched.value.map { e => - val res1 = e.map(_.useragent) must beRight(qs_ua) - val res2 = e.map(_.derived_contexts) must beRight((_: String).contains("\"agentName\":\"%1$S\"")) - res1 and res2 + enriched.value.map { + case Ior.Right(enriched) => + val res1 = enriched.useragent must_== qs_ua + val res2 = enriched.derived_contexts must contain("\"agentName\":\"%1$S\"") + res1 and res2 + case other => ko(s"[$other] is not an enriched event") } } @@ -896,10 +941,12 @@ class EnrichmentManagerSpec extends Specification with EitherMatchers with CatsE AcceptInvalid.featureFlags, IO.unit, SpecHelpers.registryLookup, - atomicFieldLimits + atomicFieldLimits, + emitIncomplete ) - enriched.value.map { e => - e.map(_.app_id) must beRight("moo") + enriched.value.map { + case Ior.Right(enriched) => enriched.app_id must_== "moo" + case other => ko(s"[$other] is not an enriched event") } } @@ -946,11 +993,15 @@ class EnrichmentManagerSpec extends Specification with EitherMatchers with CatsE AcceptInvalid.featureFlags, IO.unit, SpecHelpers.registryLookup, - atomicFieldLimits + atomicFieldLimits, + emitIncomplete ) - enriched.value.map { e => - (e.map(_.app_id) must beRight("test_app_id")) and - (e.map(_.platform) must beRight("test_platform")) + enriched.value.map { + case Ior.Right(enriched) => + val res1 = enriched.app_id must_== "test_app_id" + val res2 = enriched.platform must_== "test_platform" + res1 and res2 + case other => ko(s"[$other] is not an enriched event") } } @@ -1100,20 +1151,21 @@ class EnrichmentManagerSpec extends Specification with EitherMatchers with CatsE AcceptInvalid.featureFlags, IO.unit, SpecHelpers.registryLookup, - atomicFieldLimits + atomicFieldLimits, + emitIncomplete ) - enriched.value.map(_ must beRight.like { - case e: EnrichedEvent => - val p = EnrichedEvent.toPartiallyEnrichedEvent(e) + enriched.value.map { + case Ior.Right(enriched) => + val p = EnrichedEvent.toPartiallyEnrichedEvent(enriched) val contextsJson = jparse(p.contexts.get).toOption.get val derivedContextsJson = jparse(p.derived_contexts.get).toOption.get val ueJson = jparse(p.unstruct_event.get).toOption.get (contextsJson must beEqualTo(expectedContexts)) and (derivedContextsJson must beEqualTo(expectedDerivedContexts)) and (ueJson must beEqualTo(expectedUnstructEvent)) - case _ => ko - }) + case other => ko(s"[$other] is not an enriched event") + } } } @@ -1588,22 +1640,23 @@ class EnrichmentManagerSpec extends Specification with EitherMatchers with CatsE featureFlags = AcceptInvalid.featureFlags.copy(acceptInvalid = false), IO.unit, SpecHelpers.registryLookup, - atomicFieldLimits + atomicFieldLimits, + emitIncomplete ) .value - result.map(_ must beLeft.like { - case badRow: BadRow.EnrichmentFailures => - val firstError = badRow.failure.messages.head.message - val secondError = badRow.failure.messages.last.message + result.map { + case Ior.Left(ef: BadRow.EnrichmentFailures) => + val firstError = ef.failure.messages.head.message + val secondError = ef.failure.messages.last.message firstError must beEqualTo( EnrichmentFailureMessage.Simple("Enriched event does not conform to atomic schema field's length restrictions") ) secondError must beEqualTo(EnrichmentFailureMessage.Simple("Field v_tracker longer than maximum allowed size 100")) - case br => - ko(s"bad row [$br] is not BadRow.EnrichmentFailures") - }) + case other => + ko(s"[$other] is not an EnrichmentFailures bad row") + } } "not create a bad row if a field is oversized and acceptInvalid is set to true" >> { @@ -1617,11 +1670,15 @@ class EnrichmentManagerSpec extends Specification with EitherMatchers with CatsE featureFlags = AcceptInvalid.featureFlags.copy(acceptInvalid = true), IO.unit, SpecHelpers.registryLookup, - atomicFieldLimits + atomicFieldLimits, + emitIncomplete ) .value - result.map(_ must beRight[EnrichedEvent]) + result.map { + case Ior.Right(_) => ok + case other => ko(s"[$other] is not an enriched event") + } } } } diff --git a/modules/common/src/test/scala/com.snowplowanalytics.snowplow.enrich.common/enrichments/clientEnrichmentSpecs.scala b/modules/common/src/test/scala/com.snowplowanalytics.snowplow.enrich.common/enrichments/clientEnrichmentSpecs.scala index 91c8db02d..74e2cc301 100644 --- a/modules/common/src/test/scala/com.snowplowanalytics.snowplow.enrich.common/enrichments/clientEnrichmentSpecs.scala +++ b/modules/common/src/test/scala/com.snowplowanalytics.snowplow.enrich.common/enrichments/clientEnrichmentSpecs.scala @@ -19,26 +19,15 @@ import com.snowplowanalytics.snowplow.badrows._ class ExtractViewDimensionsSpec extends Specification with DataTables { val FieldName = "res" - def err: String => FailureDetails.EnrichmentFailure = + def err: String => FailureDetails.SchemaViolation = input => - FailureDetails.EnrichmentFailure( - None, - FailureDetails.EnrichmentFailureMessage.InputData( - FieldName, - Option(input), - """does not conform to regex (\d+)x(\d+)""" - ) - ) - def err2: String => FailureDetails.EnrichmentFailure = + FailureDetails.SchemaViolation + .NotJson(FieldName, Option(input), """does not conform to regex (\d+)x(\d+)""") + + def err2: String => FailureDetails.SchemaViolation = input => - FailureDetails.EnrichmentFailure( - None, - FailureDetails.EnrichmentFailureMessage.InputData( - FieldName, - Option(input), - "could not be converted to java.lang.Integer s" - ) - ) + FailureDetails.SchemaViolation + .NotJson(FieldName, Option(input), "could not be converted to java.lang.Integer s") def is = s2""" Extracting screen dimensions (viewports, screen resolution etc) with extractViewDimensions should work $e1""" diff --git a/modules/common/src/test/scala/com.snowplowanalytics.snowplow.enrich.common/enrichments/eventEnrichmentSpecs.scala b/modules/common/src/test/scala/com.snowplowanalytics.snowplow.enrich.common/enrichments/eventEnrichmentSpecs.scala index 92f0448b8..c98445f4e 100644 --- a/modules/common/src/test/scala/com.snowplowanalytics.snowplow.enrich.common/enrichments/eventEnrichmentSpecs.scala +++ b/modules/common/src/test/scala/com.snowplowanalytics.snowplow.enrich.common/enrichments/eventEnrichmentSpecs.scala @@ -18,6 +18,8 @@ import org.specs2.matcher.DataTables import com.snowplowanalytics.snowplow.badrows._ +import com.snowplowanalytics.snowplow.enrich.common.enrichments.EventEnrichments.FormatError + class ExtractEventTypeSpec extends Specification with DataTables { def is = s2""" extractEventType should return the event name for any valid event code $e1 @@ -27,16 +29,10 @@ class ExtractEventTypeSpec extends Specification with DataTables { """ val FieldName = "e" - def err: String => FailureDetails.EnrichmentFailure = + def err: String => FailureDetails.SchemaViolation = input => - FailureDetails.EnrichmentFailure( - None, - FailureDetails.EnrichmentFailureMessage.InputData( - FieldName, - Option(input), - "not recognized as an event type" - ) - ) + FailureDetails.SchemaViolation + .NotJson(FieldName, Option(input), "not recognized as an event type") def e1 = "SPEC NAME" || "INPUT VAL" | "EXPECTED OUTPUT" | @@ -66,9 +62,9 @@ class ExtractEventTypeSpec extends Specification with DataTables { def e3 = // format: off "SPEC NAME" || "INPUT VAL" | "EXPECTED OUTPUT" | - "None" !! None ! FailureDetails.EnrichmentFailure(None, FailureDetails.EnrichmentFailureMessage.InputData("collector_tstamp", None, "should be set")).asLeft | - "Negative timestamp" !! BCTstamp ! FailureDetails.EnrichmentFailure(None, FailureDetails.EnrichmentFailureMessage.InputData("collector_tstamp", Some("-0030-01-01T00:00:00.000Z"),"formatted as -0030-01-01 00:00:00.000 is not Redshift-compatible")).asLeft | - ">10k timestamp" !! FarAwayTstamp ! FailureDetails.EnrichmentFailure(None, FailureDetails.EnrichmentFailureMessage.InputData("collector_tstamp", Some("11970-01-01T00:00:00.000Z"),"formatted as 11970-01-01 00:00:00.000 is not Redshift-compatible")).asLeft | + "None" !! None ! FailureDetails.SchemaViolation.NotJson("collector_tstamp", None, "not set").asLeft | + "Negative timestamp" !! BCTstamp ! FailureDetails.SchemaViolation.NotJson("collector_tstamp", Some("-0030-01-01T00:00:00.000Z"),"formatted as -0030-01-01 00:00:00.000 is not Redshift-compatible").asLeft | + ">10k timestamp" !! FarAwayTstamp ! FailureDetails.SchemaViolation.NotJson("collector_tstamp", Some("11970-01-01T00:00:00.000Z"),"formatted as 11970-01-01 00:00:00.000 is not Redshift-compatible").asLeft | "Valid timestamp" !! SeventiesTstamp ! "1970-01-01 00:00:00.000".asRight |> { // format: on (_, input, expected) => @@ -77,26 +73,11 @@ class ExtractEventTypeSpec extends Specification with DataTables { def e4 = "SPEC NAME" || "INPUT VAL" | "EXPECTED OUTPUT" | - "Not long" !! (("f", "v")) ! FailureDetails - .EnrichmentFailure( - None, - FailureDetails.EnrichmentFailureMessage.InputData( - "f", - Some("v"), - "not in the expected format: ms since epoch" - ) - ) - .asLeft | - "Too long" !! (("f", "1111111111111111")) ! FailureDetails - .EnrichmentFailure( - None, - FailureDetails.EnrichmentFailureMessage.InputData( - "f", - Some("1111111111111111"), - "formatting as 37179-09-17 07:18:31.111 is not Redshift-compatible" - ) - ) - .asLeft | + "Not long" !! (("f", "v")) ! FormatError("f", Some("v"), "not in the expected format: ms since epoch").asLeft | + "Too long" !! (("f", "1111111111111111")) ! FormatError("f", + Some("1111111111111111"), + "formatting as 37179-09-17 07:18:31.111 is not Redshift-compatible" + ).asLeft | "Valid ts" !! (("f", "1")) ! "1970-01-01 00:00:00.001".asRight |> { (_, input, expected) => EventEnrichments.extractTimestamp(input._1, input._2) must_== expected } diff --git a/modules/common/src/test/scala/com.snowplowanalytics.snowplow.enrich.common/enrichments/miscEnrichmentSpecs.scala b/modules/common/src/test/scala/com.snowplowanalytics.snowplow.enrich.common/enrichments/miscEnrichmentSpecs.scala index e13822302..6a83dbdb7 100644 --- a/modules/common/src/test/scala/com.snowplowanalytics.snowplow.enrich.common/enrichments/miscEnrichmentSpecs.scala +++ b/modules/common/src/test/scala/com.snowplowanalytics.snowplow.enrich.common/enrichments/miscEnrichmentSpecs.scala @@ -35,16 +35,10 @@ class EtlVersionSpec extends MutSpecification { /** Tests the extractPlatform function. Uses DataTables. */ class ExtractPlatformSpec extends Specification with DataTables { val FieldName = "p" - def err: String => FailureDetails.EnrichmentFailure = + def err: String => FailureDetails.SchemaViolation = input => - FailureDetails.EnrichmentFailure( - None, - FailureDetails.EnrichmentFailureMessage.InputData( - FieldName, - Option(input), - "not recognized as a tracking platform" - ) - ) + FailureDetails.SchemaViolation + .NotJson(FieldName, Option(input), "not recognized as a tracking platform") def is = s2""" Extracting platforms with extractPlatform should work $e1 diff --git a/modules/common/src/test/scala/com.snowplowanalytics.snowplow.enrich.common/enrichments/registry/pii/PiiPseudonymizerEnrichmentSpec.scala b/modules/common/src/test/scala/com.snowplowanalytics.snowplow.enrich.common/enrichments/registry/pii/PiiPseudonymizerEnrichmentSpec.scala index 53a0894be..25148a1cc 100644 --- a/modules/common/src/test/scala/com.snowplowanalytics.snowplow.enrich.common/enrichments/registry/pii/PiiPseudonymizerEnrichmentSpec.scala +++ b/modules/common/src/test/scala/com.snowplowanalytics.snowplow.enrich.common/enrichments/registry/pii/PiiPseudonymizerEnrichmentSpec.scala @@ -10,7 +10,6 @@ */ package com.snowplowanalytics.snowplow.enrich.common.enrichments.registry.pii -import cats.data.Validated import cats.syntax.option._ import cats.syntax.validated._ @@ -63,7 +62,7 @@ class PiiPseudonymizerEnrichmentSpec extends Specification with ValidatedMatcher removeAddedFields should remove fields added by PII enrichment $e9 """ - def commonSetup(enrichmentReg: EnrichmentRegistry[IO]): IO[List[Validated[BadRow, EnrichedEvent]]] = { + def commonSetup(enrichmentReg: EnrichmentRegistry[IO]): IO[List[Either[BadRow, EnrichedEvent]]] = { val context = CollectorPayload.Context( Some(DateTime.parse("2017-07-14T03:39:39.000+00:00")), @@ -182,9 +181,10 @@ class PiiPseudonymizerEnrichmentSpec extends Specification with ValidatedMatcher AcceptInvalid.featureFlags, IO.unit, SpecHelpers.registryLookup, - AtomicFields.from(Map.empty) + AtomicFields.from(Map.empty), + emitIncomplete ) - } yield result + } yield result.map(_.toEither) } private val ipEnrichment = { @@ -319,7 +319,7 @@ class PiiPseudonymizerEnrichmentSpec extends Specification with ValidatedMatcher actual.map { output => val size = output.size must_== 1 - val validOut = output.head must beValid.like { + val validOut = output.head must beRight.like { case enrichedEvent => (enrichedEvent.app_id must_== expected.app_id) and (enrichedEvent.geo_city must_== expected.geo_city) and @@ -414,7 +414,7 @@ class PiiPseudonymizerEnrichmentSpec extends Specification with ValidatedMatcher actual.map { output => val size = output.size must_== 1 - val validOut = output.head must beValid.like { + val validOut = output.head must beRight.like { case enrichedEvent => val contextJ = parse(enrichedEvent.contexts).toOption.get.hcursor val contextJFirstElement = contextJ.downField("data").downArray @@ -523,7 +523,7 @@ class PiiPseudonymizerEnrichmentSpec extends Specification with ValidatedMatcher actual.map { output => val size = output.size must_== 1 - val validOut = output.head must beValid.like { + val validOut = output.head must beRight.like { case enrichedEvent => val contextJ = parse(enrichedEvent.contexts).toOption.get.hcursor.downField("data") val firstElem = contextJ.downArray.downField("data") @@ -574,7 +574,7 @@ class PiiPseudonymizerEnrichmentSpec extends Specification with ValidatedMatcher actual.map { output => val size = output.size must_== 1 - val validOut = output.head must beValid.like { + val validOut = output.head must beRight.like { case enrichedEvent => val contextJ = parse(enrichedEvent.contexts).toOption.get.hcursor.downField("data") val firstElem = contextJ.downArray.downField("data") @@ -628,7 +628,7 @@ class PiiPseudonymizerEnrichmentSpec extends Specification with ValidatedMatcher actual.map { output => val size = output.size must_== 1 - val validOut = output.head must beValid.like { + val validOut = output.head must beRight.like { case enrichedEvent => val contextJ = parse(enrichedEvent.contexts).toOption.get.hcursor.downField("data") val firstElem = contextJ.downArray.downField("data") @@ -682,7 +682,7 @@ class PiiPseudonymizerEnrichmentSpec extends Specification with ValidatedMatcher actual.map { output => val size = output.size must_== 1 - val validOut = output.head must beValid.like { + val validOut = output.head must beRight.like { case enrichedEvent => val contextJ = parse(enrichedEvent.contexts).toOption.get.hcursor.downField("data") val firstElem = contextJ.downArray.downField("data") @@ -746,7 +746,7 @@ class PiiPseudonymizerEnrichmentSpec extends Specification with ValidatedMatcher actual.map { output => val size = output.size must_== 1 - val validOut = output.head must beValid.like { + val validOut = output.head must beRight.like { case enrichedEvent => val contextJ = parse(enrichedEvent.contexts).toOption.get.hcursor.downField("data") val firstElem = contextJ.downArray.downField("data") @@ -801,7 +801,7 @@ class PiiPseudonymizerEnrichmentSpec extends Specification with ValidatedMatcher actual.map { output => val size = output.size must_== 1 - val validOut = output.head must beValid.like { + val validOut = output.head must beRight.like { case enrichedEvent => val context = parse(enrichedEvent.contexts).toOption.get.hcursor.downField("data").downArray val data = context.downField("data") 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 f60d5778a..3572d8c06 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 @@ -17,7 +17,7 @@ import cats.effect.testing.specs2.CatsEffect import io.circe.parser.parse -import cats.data.NonEmptyList +import cats.data.{Ior, NonEmptyList} import com.snowplowanalytics.iglu.core.{SchemaKey, SchemaVer} @@ -135,7 +135,11 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect "return None if unstruct_event field is empty" >> { IgluUtils .extractAndValidateUnstructEvent(new EnrichedEvent, SpecHelpers.client, SpecHelpers.registryLookup) - .map(_ must beValid(None)) + .value + .map { + case Ior.Right(None) => ok + case other => ko(s"[$other] is not a success with None") + } } "return a SchemaViolation.NotJson if unstruct_event does not contain a properly formatted JSON string" >> { @@ -144,10 +148,11 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect IgluUtils .extractAndValidateUnstructEvent(input, SpecHelpers.client, SpecHelpers.registryLookup) - .map(_ must beInvalid.like { - case _: FailureDetails.SchemaViolation.NotJson => ok - case err => ko(s"[$err] is not NotJson") - }) + .value + .map { + case Ior.Both(NonEmptyList(_: FailureDetails.SchemaViolation.NotJson, _), 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" >> { @@ -156,10 +161,11 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect IgluUtils .extractAndValidateUnstructEvent(input, SpecHelpers.client, SpecHelpers.registryLookup) - .map(_ must beInvalid.like { - case _: FailureDetails.SchemaViolation.NotIglu => ok - case err => ko(s"[$err] is not NotIglu") - }) + .value + .map { + case Ior.Both(NonEmptyList(_: FailureDetails.SchemaViolation.NotIglu, _), 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" >> { @@ -168,10 +174,11 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect IgluUtils .extractAndValidateUnstructEvent(input, SpecHelpers.client, SpecHelpers.registryLookup) - .map(_ must beInvalid.like { - case _: FailureDetails.SchemaViolation.CriterionMismatch => ok - case err => ko(s"[$err] is not CriterionMismatch") - }) + .value + .map { + case Ior.Both(NonEmptyList(_: FailureDetails.SchemaViolation.CriterionMismatch, _), 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" >> { @@ -180,10 +187,11 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect IgluUtils .extractAndValidateUnstructEvent(input, SpecHelpers.client, SpecHelpers.registryLookup) - .map(_ must beInvalid.like { - case _: FailureDetails.SchemaViolation.NotJson => ok - case err => ko(s"[$err] is not NotJson") - }) + .value + .map { + case Ior.Both(NonEmptyList(_: FailureDetails.SchemaViolation.NotJson, _), 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" >> { @@ -192,12 +200,13 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect IgluUtils .extractAndValidateUnstructEvent(input, SpecHelpers.client, SpecHelpers.registryLookup) - .map(_ must beInvalid.like { - case FailureDetails.SchemaViolation.IgluError(_, ValidationError(_, _)) => ok - case ie: FailureDetails.SchemaViolation.IgluError => + .value + .map { + case Ior.Both(NonEmptyList(FailureDetails.SchemaViolation.IgluError(_, ValidationError(_, _)), _), None) => ok + case Ior.Both(NonEmptyList(ie: FailureDetails.SchemaViolation.IgluError, _), None) => ko(s"IgluError [$ie] is not ValidationError") - case err => ko(s"[$err] is not IgluError") - }) + case other => ko(s"[$other] is not an error with IgluError") + } } "return a SchemaViolation.IgluError containing a ValidationError if the JSON in .data is not a valid SDJ" >> { @@ -206,12 +215,13 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect IgluUtils .extractAndValidateUnstructEvent(input, SpecHelpers.client, SpecHelpers.registryLookup) - .map(_ must beInvalid.like { - case FailureDetails.SchemaViolation.IgluError(_, ValidationError(_, _)) => ok - case ie: FailureDetails.SchemaViolation.IgluError => + .value + .map { + case Ior.Both(NonEmptyList(FailureDetails.SchemaViolation.IgluError(_, ValidationError(_, _)), _), None) => ok + case Ior.Both(NonEmptyList(ie: FailureDetails.SchemaViolation.IgluError, _), None) => ko(s"IgluError [$ie] is not ValidationError") - case err => ko(s"[$err] is not IgluError") - }) + case other => ko(s"[$other] is not an error with IgluError") + } } "return a SchemaViolation.IgluError containing a ResolutionError if the schema of the SDJ in .data can't be resolved" >> { @@ -220,12 +230,13 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect IgluUtils .extractAndValidateUnstructEvent(input, SpecHelpers.client, SpecHelpers.registryLookup) - .map(_ must beInvalid.like { - case FailureDetails.SchemaViolation.IgluError(_, ResolutionError(_)) => ok - case ie: FailureDetails.SchemaViolation.IgluError => - ko(s"IgluError [$ie] is not ResolutionError") - case err => ko(s"[$err] is not IgluError") - }) + .value + .map { + case Ior.Both(NonEmptyList(FailureDetails.SchemaViolation.IgluError(_, ResolutionError(_)), _), None) => ok + case Ior.Both(NonEmptyList(ie: FailureDetails.SchemaViolation.IgluError, _), None) => + ko(s"IgluError [$ie] is not a ResolutionError") + case other => ko(s"[$other] is not an error with IgluError") + } } "return the extracted unstructured event if .data is a valid SDJ" >> { @@ -234,14 +245,15 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect IgluUtils .extractAndValidateUnstructEvent(input, SpecHelpers.client, SpecHelpers.registryLookup) - .map(_ must beValid.like { - case Some(IgluUtils.SdjExtractResult(sdj, None)) if sdj.schema == emailSentSchema => ok - case Some(s) => + .value + .map { + case Ior.Right(Some(IgluUtils.SdjExtractResult(sdj, None))) if sdj.schema == emailSentSchema => ok + case Ior.Right(Some(s)) => ko( s"unstructured event's schema [${s.sdj.schema}] does not match expected schema [${emailSentSchema}]" ) - case None => ko("no unstructured event was extracted") - }) + case other => ko(s"no unstructured event was extracted [$other]") + } } "return the extracted unstructured event when schema is superseded by another schema" >> { @@ -255,26 +267,32 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect IgluUtils .extractAndValidateUnstructEvent(input1, SpecHelpers.client, SpecHelpers.registryLookup) - .map(_ must beValid.like { - case Some(IgluUtils.SdjExtractResult(sdj, Some(`expectedValidationInfo`))) if sdj.schema == supersedingExampleSchema101 => ok - case Some(s) => + .value + .map { + case Ior.Right(Some(IgluUtils.SdjExtractResult(sdj, Some(`expectedValidationInfo`)))) + if sdj.schema == supersedingExampleSchema101 => + ok + case Ior.Right(Some(s)) => ko( s"unstructured event's schema [${s.sdj.schema}] does not match expected schema [${supersedingExampleSchema101}]" ) - case None => ko("no unstructured event was extracted") - }) + case other => ko(s"no unstructured event was extracted [$other]") + } // input2 wouldn't be validated with 1-0-0. It would be validated with 1-0-1 only. IgluUtils .extractAndValidateUnstructEvent(input2, SpecHelpers.client, SpecHelpers.registryLookup) - .map(_ must beValid.like { - case Some(IgluUtils.SdjExtractResult(sdj, Some(`expectedValidationInfo`))) if sdj.schema == supersedingExampleSchema101 => ok - case Some(s) => + .value + .map { + case Ior.Right(Some(IgluUtils.SdjExtractResult(sdj, Some(`expectedValidationInfo`)))) + if sdj.schema == supersedingExampleSchema101 => + ok + case Ior.Right(Some(s)) => ko( s"unstructured event's schema [${s.sdj.schema}] does not match expected schema [${supersedingExampleSchema101}]" ) - case None => ko("no unstructured event was extracted") - }) + case other => ko(s"no unstructured event was extracted [$other]") + } } } @@ -282,7 +300,11 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect "return Nil if contexts field is empty" >> { IgluUtils .extractAndValidateInputContexts(new EnrichedEvent, SpecHelpers.client, SpecHelpers.registryLookup) - .map(_ must beValid(Nil)) + .value + .map { + case Ior.Right(Nil) => ok + case other => ko(s"[$other] is not a success with an empty list") + } } "return a SchemaViolation.NotJson if .contexts does not contain a properly formatted JSON string" >> { @@ -291,10 +313,11 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect IgluUtils .extractAndValidateInputContexts(input, SpecHelpers.client, SpecHelpers.registryLookup) - .map(_ must beInvalid.like { - case NonEmptyList(_: FailureDetails.SchemaViolation.NotJson, Nil) => ok - case err => ko(s"[$err] is not one NotJson") - }) + .value + .map { + case Ior.Both(NonEmptyList(_: FailureDetails.SchemaViolation.NotJson, 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" >> { @@ -303,10 +326,11 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect IgluUtils .extractAndValidateInputContexts(input, SpecHelpers.client, SpecHelpers.registryLookup) - .map(_ must beInvalid.like { - case NonEmptyList(_: FailureDetails.SchemaViolation.NotIglu, Nil) => ok - case err => ko(s"[$err] is not one NotIglu") - }) + .value + .map { + case Ior.Both(NonEmptyList(_: 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" >> { @@ -315,10 +339,11 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect IgluUtils .extractAndValidateInputContexts(input, SpecHelpers.client, SpecHelpers.registryLookup) - .map(_ must beInvalid.like { - case NonEmptyList(_: FailureDetails.SchemaViolation.CriterionMismatch, Nil) => ok - case err => ko(s"[$err] is not one CriterionMismatch") - }) + .value + .map { + case Ior.Both(NonEmptyList(_: 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" >> { @@ -329,13 +354,14 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect IgluUtils .extractAndValidateInputContexts(input, SpecHelpers.client, SpecHelpers.registryLookup) - .map(_ must beInvalid.like { - case NonEmptyList(FailureDetails.SchemaViolation.IgluError(_, ValidationError(_, _)), Nil) => + .value + .map { + case Ior.Both(NonEmptyList(FailureDetails.SchemaViolation.IgluError(_, ValidationError(_, _)), Nil), Nil) => ok - case NonEmptyList(ie: FailureDetails.SchemaViolation.IgluError, Nil) => + case Ior.Both(NonEmptyList(ie: FailureDetails.SchemaViolation.IgluError, Nil), Nil) => ko(s"IgluError [$ie] is not ValidationError") - case err => ko(s"[$err] is not one IgluError") - }) + case other => ko(s"[$other] is not an error with IgluError") + } } "return a SchemaViolation.IgluError containing a ValidationError if .data contains one invalid context" >> { @@ -344,13 +370,14 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect IgluUtils .extractAndValidateInputContexts(input, SpecHelpers.client, SpecHelpers.registryLookup) - .map(_ must beInvalid.like { - case NonEmptyList(FailureDetails.SchemaViolation.IgluError(_, ValidationError(_, _)), Nil) => + .value + .map { + case Ior.Both(NonEmptyList(FailureDetails.SchemaViolation.IgluError(_, ValidationError(_, _)), Nil), Nil) => ok - case NonEmptyList(ie: FailureDetails.SchemaViolation.IgluError, Nil) => + case Ior.Both(NonEmptyList(ie: FailureDetails.SchemaViolation.IgluError, Nil), Nil) => ko(s"IgluError [$ie] is not ValidationError") - case err => ko(s"[$err] is not one IgluError") - }) + 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" >> { @@ -359,13 +386,14 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect IgluUtils .extractAndValidateInputContexts(input, SpecHelpers.client, SpecHelpers.registryLookup) - .map(_ must beInvalid.like { - case NonEmptyList(FailureDetails.SchemaViolation.IgluError(_, ResolutionError(_)), Nil) => + .value + .map { + case Ior.Both(NonEmptyList(FailureDetails.SchemaViolation.IgluError(_, ResolutionError(_)), Nil), Nil) => ok - case NonEmptyList(ie: FailureDetails.SchemaViolation.IgluError, Nil) => + case Ior.Both(NonEmptyList(ie: FailureDetails.SchemaViolation.IgluError, Nil), Nil) => ko(s"IgluError [$ie] is not ResolutionError") - case err => ko(s"[$err] is not one IgluError") - }) + case other => ko(s"[$other] is not an error with IgluError") + } } "return 2 expected failures for 2 invalid contexts" >> { @@ -374,26 +402,31 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect IgluUtils .extractAndValidateInputContexts(input, SpecHelpers.client, SpecHelpers.registryLookup) - .map(_ must beInvalid.like { - case NonEmptyList( - FailureDetails.SchemaViolation.IgluError(_, ValidationError(_, _)), - List(FailureDetails.SchemaViolation.IgluError(_, ResolutionError(_))) + .value + .map { + case Ior.Both(NonEmptyList( + FailureDetails.SchemaViolation.IgluError(_, ValidationError(_, _)), + List(FailureDetails.SchemaViolation.IgluError(_, ResolutionError(_))) + ), + Nil ) => ok - case errs => ko(s"[$errs] is not one ValidationError and one ResolutionError") - }) + case other => ko(s"[$other] is not one ValidationError and one ResolutionError") + } } + // TODO: check good schema "return an expected failure if one context is valid and the other invalid" >> { val input = new EnrichedEvent input.setContexts(buildInputContexts(List(emailSent1, noSchema))) IgluUtils .extractAndValidateInputContexts(input, SpecHelpers.client, SpecHelpers.registryLookup) - .map(_ must beInvalid.like { - case NonEmptyList(_: FailureDetails.SchemaViolation.IgluError, Nil) => ok - case err => ko(s"[$err] is not one IgluError") - }) + .value + .map { + case Ior.Both(NonEmptyList(_: FailureDetails.SchemaViolation.IgluError, Nil), List(_)) => ok + case other => ko(s"[$other] is not one IgluError and one valid SDJ") + } } "return the extracted SDJs for 2 valid input contexts" >> { @@ -402,12 +435,13 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect IgluUtils .extractAndValidateInputContexts(input, SpecHelpers.client, SpecHelpers.registryLookup) - .map(_ must beValid.like { - case sdjs if sdjs.size == 2 && sdjs.forall(i => i.sdj.schema == emailSentSchema && i.validationInfo.isEmpty) => + .value + .map { + case Ior.Right(sdjs) if sdjs.size == 2 && sdjs.forall(i => i.sdj.schema == emailSentSchema && i.validationInfo.isEmpty) => ok - case res => - ko(s"[$res] are not 2 SDJs with expected schema [${emailSentSchema.toSchemaUri}]") - }) + case other => + ko(s"[$other] is not 2 SDJs with expected schema [${emailSentSchema.toSchemaUri}]") + } } "return the extracted SDJ for an input that has a required property set to null if the schema explicitly allows it" >> { @@ -416,12 +450,13 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect IgluUtils .extractAndValidateInputContexts(input, SpecHelpers.client, SpecHelpers.registryLookup) - .map(_ must beValid.like { - case sdj if sdj.size == 1 && sdj.forall(_.sdj.schema == clientSessionSchema) => + .value + .map { + case Ior.Right(sdjs) if sdjs.size == 1 && sdjs.forall(_.sdj.schema == clientSessionSchema) => ok - case _ => - ko("$.previousSessionId: is missing but it is required") - }) + case other => + ko(s"[$other] is not 1 SDJ with expected schema [${clientSessionSchema.toSchemaUri}]") + } } "return the extracted context when schema is superseded by another schema" >> { @@ -430,234 +465,242 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect IgluUtils .extractAndValidateInputContexts(input, SpecHelpers.client, SpecHelpers.registryLookup) - .map(_ must beValid.like { - case sdj if sdj.size == 2 && sdj.forall(_.sdj.schema == supersedingExampleSchema101) => + .value + .map { + case Ior.Right(sdjs) if sdjs.size == 2 && sdjs.forall(_.sdj.schema == supersedingExampleSchema101) => ok - case _ => - ko("Failed to extract context when schema is superseded by another schema") - }) + case other => + ko(s"[$other] is not 2 SDJs with expected schema [${supersedingExampleSchema101.toSchemaUri}]") + } } } "validateEnrichmentsContexts" should { - "return a BadRow.EnrichmentFailures with one expected failure for one invalid context" >> { + "return one expected failure for one invalid context" >> { val contexts = List( SpecHelpers.jsonStringToSDJ(invalidEmailSent).right.get ) IgluUtils - .validateEnrichmentsContexts(SpecHelpers.client, contexts, raw, processor, enriched, SpecHelpers.registryLookup) + .validateEnrichmentsContexts(SpecHelpers.client, contexts, SpecHelpers.registryLookup) .value - .map(_ must beLeft.like { - case BadRow.EnrichmentFailures(_, failures, _) => - failures.messages match { - case NonEmptyList( - FailureDetails.EnrichmentFailure( - _, - FailureDetails.EnrichmentFailureMessage.IgluError(_, ValidationError(_, _)) - ), - _ - ) => - ok - case err => ko(s"bad row is EnrichmentFailures but [$err] is not one ValidationError") - } - case br => ko(s"bad row [$br] is not EnrichmentFailures") - }) - } - - "return a BadRow.EnrichmentFailures 2 expected failures for 2 invalid contexts" >> { + .map { + case Ior.Both(NonEmptyList( + FailureDetails.EnrichmentFailure( + _, + FailureDetails.EnrichmentFailureMessage.IgluError(_, ValidationError(_, _)) + ), + _ + ), + Nil + ) => + ok + case other => ko(s"[$other] is not one EnrichmentFailure with ValidationError") + } + } + + "return 2 expected failures for 2 invalid contexts" >> { val contexts = List( SpecHelpers.jsonStringToSDJ(invalidEmailSent).right.get, SpecHelpers.jsonStringToSDJ(noSchema).right.get ) IgluUtils - .validateEnrichmentsContexts(SpecHelpers.client, contexts, raw, processor, enriched, SpecHelpers.registryLookup) + .validateEnrichmentsContexts(SpecHelpers.client, contexts, SpecHelpers.registryLookup) .value - .map(_ must beLeft.like { - case BadRow.EnrichmentFailures(_, failures, _) => - failures.messages match { - case NonEmptyList( - FailureDetails.EnrichmentFailure( - _, - FailureDetails.EnrichmentFailureMessage.IgluError(_, ValidationError(_, _)) - ), - List( - FailureDetails.EnrichmentFailure( - _, - FailureDetails.EnrichmentFailureMessage.IgluError(_, ResolutionError(_)) - ) - ) - ) => - ok - case errs => - ko( - s"bad row is EnrichmentFailures but [$errs] is not one ValidationError and one ResolutionError" - ) - } - case br => ko(s"bad row [$br] is not EnrichmentFailures") - }) - } - - "return a BadRow.EnrichmentFailures with an expected failure for 1 valid context and one invalid" >> { + .map { + case Ior.Both(NonEmptyList( + FailureDetails.EnrichmentFailure( + _, + FailureDetails.EnrichmentFailureMessage.IgluError(_, ValidationError(_, _)) + ), + List( + FailureDetails.EnrichmentFailure( + _, + FailureDetails.EnrichmentFailureMessage.IgluError(_, ResolutionError(_)) + ) + ) + ), + Nil + ) => + ok + case other => ko(s"[$other] is not one is not one ValidationError and one ResolutionError") + } + } + + // TODO: check schema in the Right + "return a failure and a SDJ for one valid context and one invalid" >> { val contexts = List( SpecHelpers.jsonStringToSDJ(invalidEmailSent).right.get, SpecHelpers.jsonStringToSDJ(emailSent1).right.get ) IgluUtils - .validateEnrichmentsContexts(SpecHelpers.client, contexts, raw, processor, enriched, SpecHelpers.registryLookup) + .validateEnrichmentsContexts(SpecHelpers.client, contexts, SpecHelpers.registryLookup) .value - .map(_ must beLeft.like { - case BadRow.EnrichmentFailures(_, failures, _) => - failures.messages match { - case NonEmptyList( - FailureDetails.EnrichmentFailure( - _, - FailureDetails.EnrichmentFailureMessage.IgluError(_, ValidationError(_, _)) - ), - Nil - ) => - ok - case err => ko(s"bad row is EnrichmentFailures but [$err] is not one ValidationError") - } - case br => ko(s"bad row [$br] is not EnrichmentFailures") - }) + .map { + case Ior.Both(NonEmptyList( + FailureDetails.EnrichmentFailure( + _, + FailureDetails.EnrichmentFailureMessage.IgluError(_, ValidationError(_, _)) + ), + Nil + ), + List(_) + ) => + ok + case other => ko(s"[$other] is not one error with a ValidationError and one valid SDJ") + } } - "not return any error for 2 valid contexts" >> { + // TODO: check the schemas + "return 2 valid contexts" >> { val contexts = List( SpecHelpers.jsonStringToSDJ(emailSent1).right.get, SpecHelpers.jsonStringToSDJ(emailSent2).right.get ) IgluUtils - .validateEnrichmentsContexts(SpecHelpers.client, contexts, raw, processor, enriched, SpecHelpers.registryLookup) - .value - .map(_ must beRight) - } - } - - "extractAndValidateInputJsons" should { - "return a SchemaViolations containing 1 error if the input event contains an invalid unstructured event" >> { - val input = new EnrichedEvent - input.setUnstruct_event(buildUnstruct(invalidEmailSent)) - - IgluUtils - .extractAndValidateInputJsons( - input, - SpecHelpers.client, - raw, - processor, - SpecHelpers.registryLookup - ) - .value - .map(_ must beLeft.like { - case BadRow.SchemaViolations(_, failure, _) if failure.messages.size == 1 => ok - case br => ko(s"bad row [$br] is not a SchemaViolations containing 1 error") - }) - } - - "return a SchemaViolations containing 1 error if the input event contains 1 invalid context" >> { - val input = new EnrichedEvent - input.setContexts(buildInputContexts(List(invalidEmailSent))) - - IgluUtils - .extractAndValidateInputJsons( - input, - SpecHelpers.client, - raw, - processor, - SpecHelpers.registryLookup - ) - .value - .map(_ must beLeft.like { - case BadRow.SchemaViolations(_, failure, _) if failure.messages.size == 1 => ok - case br => ko(s"bad row [$br] is not a SchemaViolations containing 1 error") - }) - } - - "return a SchemaViolations containing 2 errors if the input event contains an invalid unstructured event and 1 invalid context" >> { - val input = new EnrichedEvent - input.setUnstruct_event(invalidEmailSent) - input.setContexts(buildInputContexts(List(invalidEmailSent))) - - IgluUtils - .extractAndValidateInputJsons( - input, - SpecHelpers.client, - raw, - processor, - SpecHelpers.registryLookup - ) - .value - .map(_ must beLeft.like { - case BadRow.SchemaViolations(_, failure, _) if failure.messages.size == 2 => ok - case br => ko(s"bad row [$br] is not a SchemaViolations containing 2 errors") - }) - } - - "return the extracted unstructured event and the extracted input contexts if they are all valid" >> { - val input = new EnrichedEvent - input.setUnstruct_event(buildUnstruct(emailSent1)) - input.setContexts(buildInputContexts(List(emailSent1, emailSent2))) - - IgluUtils - .extractAndValidateInputJsons( - input, - SpecHelpers.client, - raw, - processor, - SpecHelpers.registryLookup - ) + .validateEnrichmentsContexts(SpecHelpers.client, contexts, SpecHelpers.registryLookup) .value - .map(_ must beRight.like { - case IgluUtils.EventExtractResult(contexts, Some(unstructEvent), validationInfos) - if contexts.size == 2 - && validationInfos.isEmpty - && (unstructEvent :: contexts).forall(_.schema == emailSentSchema) => - ok - case res => - ko( - s"[$res] is not a list with 2 extracted contexts and an option with the extracted unstructured event" - ) - }) - } - - "return the extracted unstructured event and the extracted input contexts when schema is superseded by another schema" >> { - val input = new EnrichedEvent - input.setUnstruct_event(buildUnstruct(supersedingExample1)) - input.setContexts(buildInputContexts(List(supersedingExample1, supersedingExample2))) - - val expectedValidationInfoContext = parse( - """ { + .map { + case Ior.Right(List(_, _)) => ok + case other => ko(s"[$other] doesn't contain the 2 valid contexts") + } + } + + "extractAndValidateInputJsons" should { + "return one SchemaViolation if the input event contains an invalid unstructured event" >> { + val input = new EnrichedEvent + input.setUnstruct_event(buildUnstruct(invalidEmailSent)) + + IgluUtils + .extractAndValidateInputJsons( + input, + SpecHelpers.client, + SpecHelpers.registryLookup + ) + .value + .map { + case Ior.Both( + NonEmptyList( + _: FailureDetails.SchemaViolation, + Nil + ), + IgluUtils.EventExtractResult(Nil, None, Nil) + ) => + ok + case other => ko(s"[$other] isn't an error with SchemaViolation") + } + } + + "return one SchemaViolation if the input event contains an invalid context" >> { + val input = new EnrichedEvent + input.setContexts(buildInputContexts(List(invalidEmailSent))) + + IgluUtils + .extractAndValidateInputJsons( + input, + SpecHelpers.client, + SpecHelpers.registryLookup + ) + .value + .map { + case Ior.Both( + NonEmptyList( + _: FailureDetails.SchemaViolation, + Nil + ), + IgluUtils.EventExtractResult(Nil, None, Nil) + ) => + ok + case other => ko(s"[$other] isn't an error with SchemaViolation") + } + } + + "return 2 SchemaViolation if the input event contains an invalid unstructured event and 1 invalid context" >> { + val input = new EnrichedEvent + input.setUnstruct_event(invalidEmailSent) + input.setContexts(buildInputContexts(List(invalidEmailSent))) + + IgluUtils + .extractAndValidateInputJsons( + input, + SpecHelpers.client, + SpecHelpers.registryLookup + ) + .value + .map { + case Ior.Both( + NonEmptyList( + _: FailureDetails.SchemaViolation, + List(_: FailureDetails.SchemaViolation) + ), + IgluUtils.EventExtractResult(Nil, None, Nil) + ) => + ok + case other => ko(s"[$other] isn't 2 errors with SchemaViolation") + } + } + + "return the extracted unstructured event and the extracted input contexts if they are all valid" >> { + val input = new EnrichedEvent + input.setUnstruct_event(buildUnstruct(emailSent1)) + input.setContexts(buildInputContexts(List(emailSent1, emailSent2))) + + IgluUtils + .extractAndValidateInputJsons( + input, + SpecHelpers.client, + SpecHelpers.registryLookup + ) + .value + .map { + case Ior.Right(IgluUtils.EventExtractResult(contexts, Some(unstructEvent), validationInfos)) + if contexts.size == 2 + && validationInfos.isEmpty + && (unstructEvent :: contexts).forall(_.schema == emailSentSchema) => + ok + case other => + ko( + s"[$other] doesn't contain the 2 contexts and the unstructured event" + ) + } + } + + "return the extracted unstructured event and the extracted input contexts when schema is superseded by another schema" >> { + val input = new EnrichedEvent + input.setUnstruct_event(buildUnstruct(supersedingExample1)) + input.setContexts(buildInputContexts(List(supersedingExample1, supersedingExample2))) + + val expectedValidationInfoContext = parse( + """ { | "originalSchema" : "iglu:com.acme/superseding_example/jsonschema/1-0-0", | "validatedWith" : "1-0-1" |}""".stripMargin - ).toOption.get - - IgluUtils - .extractAndValidateInputJsons( - input, - SpecHelpers.client, - raw, - processor, - SpecHelpers.registryLookup - ) - .value - .map(_ must beRight.like { - case IgluUtils.EventExtractResult(contexts, Some(unstructEvent), List(validationInfo)) - if contexts.size == 2 - && unstructEvent.schema == supersedingExampleSchema101 - && contexts.count(_.schema == supersedingExampleSchema101) == 2 - && validationInfo.schema == IgluUtils.ValidationInfo.schemaKey - && validationInfo.data == expectedValidationInfoContext => - ok - case res => - ko( - s"[$res] is not a list with 2 extracted contexts and an option with the extracted unstructured event" - ) - }) + ).toOption.get + + IgluUtils + .extractAndValidateInputJsons( + input, + SpecHelpers.client, + SpecHelpers.registryLookup + ) + .value + .map { + case Ior.Right(IgluUtils.EventExtractResult(contexts, Some(unstructEvent), List(validationInfo))) + if contexts.size == 2 + && unstructEvent.schema == supersedingExampleSchema101 + && contexts.count(_.schema == supersedingExampleSchema101) == 2 + && validationInfo.schema == IgluUtils.ValidationInfo.schemaKey + && validationInfo.data == expectedValidationInfoContext => + ok + case other => + ko( + s"[$other] doesn't contain the 2 contexts and the unstructured event with the superseded schema" + ) + } + } } } diff --git a/modules/common/src/test/scala/com.snowplowanalytics.snowplow.enrich.common/utils/MapTransformerSpec.scala b/modules/common/src/test/scala/com.snowplowanalytics.snowplow.enrich.common/utils/MapTransformerSpec.scala index 5a2f5fcbf..6703b88fe 100644 --- a/modules/common/src/test/scala/com.snowplowanalytics.snowplow.enrich.common/utils/MapTransformerSpec.scala +++ b/modules/common/src/test/scala/com.snowplowanalytics.snowplow.enrich.common/utils/MapTransformerSpec.scala @@ -52,7 +52,7 @@ final class TargetBean { class MapTransformerSpec extends Specification with ValidatedMatchers { - val identity: (String, String) => Either[FailureDetails.EnrichmentFailure, String] = + val identity: (String, String) => Either[FailureDetails.SchemaViolation, String] = (_, value) => value.asRight val sourceMap = Map( diff --git a/modules/common/src/test/scala/com.snowplowanalytics.snowplow.enrich.common/utils/ValidateAndReformatJsonSpec.scala b/modules/common/src/test/scala/com.snowplowanalytics.snowplow.enrich.common/utils/ValidateAndReformatJsonSpec.scala deleted file mode 100644 index f5cd921ec..000000000 --- a/modules/common/src/test/scala/com.snowplowanalytics.snowplow.enrich.common/utils/ValidateAndReformatJsonSpec.scala +++ /dev/null @@ -1,51 +0,0 @@ -/* - * Copyright (c) 2012-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.utils - -import org.specs2.Specification -import org.specs2.matcher.DataTables - -class ValidateAndReformatJsonSpec extends Specification with DataTables { - def is = s2""" - extracting and reformatting (where necessary) valid JSONs with work $e1 - extracting invalid JSONs should fail $e2 - """ - - def e1 = - "SPEC NAME" || "INPUT STR" | "EXPECTED" | - "Empty JSON" !! "{}" ! "{}" | - "Simple JSON #1" !! """{"key":"value"}""" ! """{"key":"value"}""" | - "Simple JSON #2" !! """[1,2,3]""" ! """[1,2,3]""" | - "Reformatted JSON #1" !! """{ "key" : 23 }""" ! """{"key":23}""" | - "Reformatted JSON #2" !! """[1.00, 2.00, 3.00, 4.00]""" ! """[1.00,2.00,3.00,4.00]""" | - "Reformatted JSON #3" !! """ - { - "a": 23 - }""" ! """{"a":23}""" |> { (_, str, expected) => - JsonUtils.validateAndReformatJson(str) must beRight(expected) - } - - def err1 = s"invalid json: exhausted input" - def err2: (String, Int, Int) => String = - (got, line, col) => s"invalid json: expected json value got '$got' (line $line, column $col)" - def err3: (String, Int, Int) => String = - (got, line, col) => s"""invalid json: expected " got '$got' (line $line, column $col)""" - - def e2 = - "SPEC NAME" || "INPUT STR" | "EXPECTED" | - "Empty string" !! "" ! err1 | - "Double colons" !! """{"a"::2}""" ! err2(":2}", 1, 6) | - "Random noise" !! "^45fj_" ! err2("^45fj_", 1, 1) | - "Bad key" !! """{9:"a"}""" ! err3("""9:"a"}""", 1, 2) |> { (_, str, expected) => - JsonUtils.validateAndReformatJson(str) must beLeft(expected) - } - -} diff --git a/modules/common/src/test/scala/com.snowplowanalytics.snowplow.enrich.common/utils/conversionUtilsSpecs.scala b/modules/common/src/test/scala/com.snowplowanalytics.snowplow.enrich.common/utils/conversionUtilsSpecs.scala index a6dcd1c39..8caf3e6ab 100644 --- a/modules/common/src/test/scala/com.snowplowanalytics.snowplow.enrich.common/utils/conversionUtilsSpecs.scala +++ b/modules/common/src/test/scala/com.snowplowanalytics.snowplow.enrich.common/utils/conversionUtilsSpecs.scala @@ -263,14 +263,8 @@ class ValidateUuidSpec extends Specification with DataTables with ScalaCheck { def e2 = prop { (str: String) => ConversionUtils.validateUuid(FieldName, str) must beLeft( - FailureDetails.EnrichmentFailure( - None, - FailureDetails.EnrichmentFailureMessage.InputData( - FieldName, - Option(str), - "not a valid UUID" - ) - ) + FailureDetails.SchemaViolation + .NotJson(FieldName, Option(str), "not a valid UUID") ) } } @@ -288,14 +282,8 @@ class ValidateIntegerSpec extends Specification { def e2 = { val str = "abc" ConversionUtils.validateInteger(FieldName, str) must beLeft( - FailureDetails.EnrichmentFailure( - None, - FailureDetails.EnrichmentFailureMessage.InputData( - FieldName, - Some(str), - "not a valid integer" - ) - ) + FailureDetails.SchemaViolation + .NotJson(FieldName, Some(str), "not a valid integer") ) } } @@ -326,16 +314,10 @@ class StringToDoubleLikeSpec extends Specification with DataTables { """ val FieldName = "val" - def err: String => FailureDetails.EnrichmentFailure = + def err: String => FailureDetails.SchemaViolation = input => - FailureDetails.EnrichmentFailure( - None, - FailureDetails.EnrichmentFailureMessage.InputData( - FieldName, - Option(input), - "cannot be converted to Double-like" - ) - ) + FailureDetails.SchemaViolation + .NotJson(FieldName, Option(input), "cannot be converted to Double-like") def e1 = "SPEC NAME" || "INPUT STR" | "EXPECTED" | @@ -410,16 +392,10 @@ class StringToBooleanLikeJByteSpec extends Specification with DataTables { """ val FieldName = "val" - def err: String => FailureDetails.EnrichmentFailure = + def err: String => FailureDetails.SchemaViolation = input => - FailureDetails.EnrichmentFailure( - None, - FailureDetails.EnrichmentFailureMessage.InputData( - FieldName, - Option(input), - "cannot be converted to Boolean-like java.lang.Byte" - ) - ) + FailureDetails.SchemaViolation + .NotJson(FieldName, Option(input), "cannot be converted to Boolean-like java.lang.Byte") def e1 = "SPEC NAME" || "INPUT STR" | "EXPECTED" | diff --git a/modules/kafka/src/main/resources/application.conf b/modules/kafka/src/main/resources/application.conf index 5c5897eda..639e1b15f 100644 --- a/modules/kafka/src/main/resources/application.conf +++ b/modules/kafka/src/main/resources/application.conf @@ -49,6 +49,21 @@ "partitionKey": "" "headers": [] } + + "incomplete": { + "type": "Kafka" + # we need all the fields to exist to have defaults + "topicName": "" + "bootstrapServers": "" + "producerConf": { + "acks": "all" + "security.protocol": "SASL_SSL" + "sasl.mechanism": "OAUTHBEARER" + "sasl.jaas.config": "org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginModule required;" + } + "partitionKey": "" + "headers": [] + } } "concurrency" : { diff --git a/modules/kafka/src/main/scala/com.snowplowanalytics.snowplow.enrich.kafka/AzureAuthenticationCallbackHandler.scala b/modules/kafka/src/main/scala/com.snowplowanalytics.snowplow.enrich.kafka/AzureAuthenticationCallbackHandler.scala index 34bc3eb8c..c9783f6b7 100644 --- a/modules/kafka/src/main/scala/com.snowplowanalytics.snowplow.enrich.kafka/AzureAuthenticationCallbackHandler.scala +++ b/modules/kafka/src/main/scala/com.snowplowanalytics.snowplow.enrich.kafka/AzureAuthenticationCallbackHandler.scala @@ -44,6 +44,8 @@ class BadSinkAuthHandler extends AzureAuthenticationCallbackHandler class PiiSinkAuthHandler extends AzureAuthenticationCallbackHandler +class IncompleteSinkAuthHandler extends AzureAuthenticationCallbackHandler + class AzureAuthenticationCallbackHandler extends AuthenticateCallbackHandler { val credentials = new DefaultAzureCredentialBuilder().build() diff --git a/modules/kafka/src/main/scala/com.snowplowanalytics.snowplow.enrich.kafka/Main.scala b/modules/kafka/src/main/scala/com.snowplowanalytics.snowplow.enrich.kafka/Main.scala index 0a9ef88e0..9ae70af42 100644 --- a/modules/kafka/src/main/scala/com.snowplowanalytics.snowplow.enrich.kafka/Main.scala +++ b/modules/kafka/src/main/scala/com.snowplowanalytics.snowplow.enrich.kafka/Main.scala @@ -61,6 +61,7 @@ object Main extends IOApp { out => Sink.initAttributed(out, classOf[GoodSinkAuthHandler].getName), out => Sink.initAttributed(out, classOf[PiiSinkAuthHandler].getName), out => Sink.init(out, classOf[BadSinkAuthHandler].getName), + out => Sink.initAttributed(out, classOf[IncompleteSinkAuthHandler].getName), checkpoint, createBlobStorageClient, _.record.value, diff --git a/modules/kafka/src/test/scala/com/snowplowanalytics/snowplow/enrich/kafka/ConfigSpec.scala b/modules/kafka/src/test/scala/com/snowplowanalytics/snowplow/enrich/kafka/ConfigSpec.scala index 054c5916c..85b5e2cf7 100644 --- a/modules/kafka/src/test/scala/com/snowplowanalytics/snowplow/enrich/kafka/ConfigSpec.scala +++ b/modules/kafka/src/test/scala/com/snowplowanalytics/snowplow/enrich/kafka/ConfigSpec.scala @@ -90,6 +90,20 @@ class ConfigSpec extends Specification with CatsEffect { "sasl.mechanism" -> "OAUTHBEARER", "sasl.jaas.config" -> "org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginModule required;" ) + ), + Some( + io.Output.Kafka( + "incomplete", + "localhost:9092", + "", + Set(), + Map( + "acks" -> "all", + "security.protocol" -> "SASL_SSL", + "sasl.mechanism" -> "OAUTHBEARER", + "sasl.jaas.config" -> "org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginModule required;" + ) + ) ) ), io.Concurrency(256, 1), @@ -200,7 +214,8 @@ class ConfigSpec extends Specification with CatsEffect { "sasl.mechanism" -> "OAUTHBEARER", "sasl.jaas.config" -> "org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginModule required;" ) - ) + ), + None ), io.Concurrency(256, 1), None, diff --git a/modules/kinesis/src/main/resources/application.conf b/modules/kinesis/src/main/resources/application.conf index a75006bde..ed99b2dd0 100644 --- a/modules/kinesis/src/main/resources/application.conf +++ b/modules/kinesis/src/main/resources/application.conf @@ -64,6 +64,23 @@ "recordLimit": 500 "byteLimit": 5242880 } + + "incomplete": { + "type": "Kinesis" + # we need all the fields to exist to have defaults + "streamName": "" + "backoffPolicy": { + "minBackoff": 100 milliseconds + "maxBackoff": 10 seconds + "maxRetries": 10 + } + "throttledBackoffPolicy": { + "minBackoff": 100 milliseconds + "maxBackoff": 1 second + } + "recordLimit": 500 + "byteLimit": 5242880 + } } "concurrency" : { diff --git a/modules/kinesis/src/main/scala/com/snowplowanalytics/snowplow/enrich/kinesis/Main.scala b/modules/kinesis/src/main/scala/com/snowplowanalytics/snowplow/enrich/kinesis/Main.scala index 16a16ec19..159dbb9ca 100644 --- a/modules/kinesis/src/main/scala/com/snowplowanalytics/snowplow/enrich/kinesis/Main.scala +++ b/modules/kinesis/src/main/scala/com/snowplowanalytics/snowplow/enrich/kinesis/Main.scala @@ -59,6 +59,7 @@ object Main extends IOApp { out => Sink.initAttributed(out), out => Sink.initAttributed(out), out => Sink.init(out), + out => Sink.initAttributed(out), checkpoint[IO], _ => List(S3Client.mk[IO]), getPayload, diff --git a/modules/kinesis/src/test/scala/com/snowplowanalytics/snowplow/enrich/kinesis/ConfigSpec.scala b/modules/kinesis/src/test/scala/com/snowplowanalytics/snowplow/enrich/kinesis/ConfigSpec.scala index c7ad3136b..77af46d98 100644 --- a/modules/kinesis/src/test/scala/com/snowplowanalytics/snowplow/enrich/kinesis/ConfigSpec.scala +++ b/modules/kinesis/src/test/scala/com/snowplowanalytics/snowplow/enrich/kinesis/ConfigSpec.scala @@ -79,6 +79,18 @@ class ConfigSpec extends Specification with CatsEffect { 500, 5242880, None + ), + Some( + io.Output.Kinesis( + "incomplete", + Some("eu-central-1"), + None, + io.BackoffPolicy(100.millis, 10.seconds, Some(10)), + io.BackoffPolicy(100.millis, 1.second, None), + 500, + 5242880, + None + ) ) ), io.Concurrency(256, 1), @@ -173,7 +185,8 @@ class ConfigSpec extends Specification with CatsEffect { 500, 5242880, None - ) + ), + None ), io.Concurrency(256, 1), None, diff --git a/modules/nsq/src/main/scala/com/snowplowanalytics/snowplow/enrich/nsq/Main.scala b/modules/nsq/src/main/scala/com/snowplowanalytics/snowplow/enrich/nsq/Main.scala index 93fd912f9..a479c458e 100644 --- a/modules/nsq/src/main/scala/com/snowplowanalytics/snowplow/enrich/nsq/Main.scala +++ b/modules/nsq/src/main/scala/com/snowplowanalytics/snowplow/enrich/nsq/Main.scala @@ -58,6 +58,7 @@ object Main extends IOApp { out => Sink.initAttributed(out), out => Sink.initAttributed(out), out => Sink.init(out), + out => Sink.initAttributed(out), checkpoint, createBlobStorageClient, _.data, diff --git a/modules/nsq/src/test/scala/com/snowplowanalytics/snowplow/enrich/nsq/ConfigSpec.scala b/modules/nsq/src/test/scala/com/snowplowanalytics/snowplow/enrich/nsq/ConfigSpec.scala index f0ef5d1ba..442e15205 100644 --- a/modules/nsq/src/test/scala/com/snowplowanalytics/snowplow/enrich/nsq/ConfigSpec.scala +++ b/modules/nsq/src/test/scala/com/snowplowanalytics/snowplow/enrich/nsq/ConfigSpec.scala @@ -81,6 +81,18 @@ class ConfigSpec extends Specification with CatsEffect { maxBackoff = 10.seconds, maxRetries = Some(10) ) + ), + Some( + io.Output.Nsq( + "incomplete", + "127.0.0.1", + 4150, + BackoffPolicy( + minBackoff = 100.milliseconds, + maxBackoff = 10.seconds, + maxRetries = Some(10) + ) + ) ) ), io.Concurrency(256, 3), @@ -187,7 +199,8 @@ class ConfigSpec extends Specification with CatsEffect { maxBackoff = 10.seconds, maxRetries = Some(10) ) - ) + ), + None ), io.Concurrency(256, 3), None, diff --git a/modules/pubsub/src/main/resources/application.conf b/modules/pubsub/src/main/resources/application.conf index bdb750caf..45dd83119 100644 --- a/modules/pubsub/src/main/resources/application.conf +++ b/modules/pubsub/src/main/resources/application.conf @@ -42,6 +42,18 @@ "productName": "Snowplow OSS" } } + + "incomplete": { + "type": "PubSub" + # we need all the fields to exist to have defaults + "topic": "" + "delayThreshold": 200 milliseconds + "maxBatchSize": 1000 + "maxBatchBytes": 8000000, + "gcpUserAgent": { + "productName": "Snowplow OSS" + } + } } "concurrency" : { diff --git a/modules/pubsub/src/main/scala/com/snowplowanalytics/snowplow/enrich/pubsub/Main.scala b/modules/pubsub/src/main/scala/com/snowplowanalytics/snowplow/enrich/pubsub/Main.scala index daee27114..cb0bf67fb 100644 --- a/modules/pubsub/src/main/scala/com/snowplowanalytics/snowplow/enrich/pubsub/Main.scala +++ b/modules/pubsub/src/main/scala/com/snowplowanalytics/snowplow/enrich/pubsub/Main.scala @@ -60,6 +60,7 @@ object Main extends IOApp { out => Sink.initAttributed(out), out => Sink.initAttributed(out), out => Sink.init(out), + out => Sink.initAttributed(out), checkpoint, _ => List(Resource.eval(GcsClient.mk[IO])), _.value, diff --git a/modules/pubsub/src/test/scala/com/snowplowanalytics/snowplow/enrich/pubsub/ConfigSpec.scala b/modules/pubsub/src/test/scala/com/snowplowanalytics/snowplow/enrich/pubsub/ConfigSpec.scala index 804348310..8765d91bc 100644 --- a/modules/pubsub/src/test/scala/com/snowplowanalytics/snowplow/enrich/pubsub/ConfigSpec.scala +++ b/modules/pubsub/src/test/scala/com/snowplowanalytics/snowplow/enrich/pubsub/ConfigSpec.scala @@ -69,6 +69,16 @@ class ConfigSpec extends Specification with CatsEffect { 1000, 8000000, io.GcpUserAgent("Snowplow OSS") + ), + Some( + io.Output.PubSub( + "projects/test-project/topics/incomplete", + None, + 200.milliseconds, + 1000, + 8000000, + io.GcpUserAgent("Snowplow OSS") + ) ) ), io.Concurrency(256, 3), @@ -155,7 +165,8 @@ class ConfigSpec extends Specification with CatsEffect { 1000, 8000000, io.GcpUserAgent("Snowplow OSS") - ) + ), + None ), io.Concurrency(256, 3), None,