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,