From f8476dd3131a6d014b57a002d14b9661ec6ecbf5 Mon Sep 17 00:00:00 2001 From: Benjamin Benoist Date: Wed, 14 Feb 2024 11:08:08 +0000 Subject: [PATCH] Add possibilty to emit failed events in TSV format into a third stream (#872) Before this change, any error in the enriching workflow would short circuit and a failed event would get emitted as JSON. After this change, if incomplete events are enabled, the enriching goes to the end with what is possible, accumulating errors as it goes. Errors get attached in derived_contexts and a failed event gets emitted to a third stream with TSV format (same as enriched event). --- .github/workflows/test.yml | 2 +- build.sbt | 3 +- config/config.file.extended.hocon | 8 + config/config.kafka.extended.hocon | 19 + config/config.kinesis.extended.hocon | 49 +- config/config.nsq.extended.hocon | 21 + config/config.pubsub.extended.hocon | 25 + .../snowplow/enrich/common/fs2/Enrich.scala | 46 +- .../enrich/common/fs2/Environment.scala | 7 +- .../snowplow/enrich/common/fs2/Run.scala | 4 + .../enrich/common/fs2/config/ConfigFile.scala | 29 +- .../enrich/common/fs2/config/io.scala | 3 +- .../enrich/common/fs2/io/Metrics.scala | 47 +- .../enrich/common/fs2/io/StatsDReporter.scala | 1 + .../snowplow/enrich/common/fs2/package.scala | 6 +- .../enrich/common/fs2/EnrichSpec.scala | 86 +- .../common/fs2/EventGenEtlPipelineSpec.scala | 15 +- .../common/fs2/blackbox/BlackBoxTesting.scala | 7 +- .../blackbox/adapters/Tp2AdapterSpec.scala | 5 +- .../common/fs2/config/ParsedConfigsSpec.scala | 3 +- .../ApiRequestEnrichmentSpec.scala | 6 +- .../fs2/enrichments/IabEnrichmentSpec.scala | 6 +- .../fs2/enrichments/YauaaEnrichmentSpec.scala | 3 +- .../common/fs2/io/StatsDReporterSpec.scala | 8 +- .../enrich/common/fs2/test/Counter.scala | 6 +- .../common/fs2/test/TestEnvironment.scala | 13 +- .../common/EtlPipeline.scala | 24 +- .../common/enrichments/AtomicFields.scala | 188 +-- .../AtomicFieldsLengthValidator.scala | 41 +- .../enrichments/ClientEnrichments.scala | 2 +- .../enrichments/EnrichmentManager.scala | 311 ++--- .../common/enrichments/EventEnrichments.scala | 12 +- .../common/enrichments/Failure.scala | 191 +++ .../common/enrichments/MiscEnrichments.scala | 2 +- .../common/utils/AtomicError.scala | 20 +- .../common/utils/ConversionUtils.scala | 18 +- .../common/utils/IgluUtils.scala | 178 +-- .../common/utils/JsonUtils.scala | 19 +- .../EtlPipelineSpec.scala | 29 +- .../SpecHelpers.scala | 25 +- .../enrichments/AtomicFieldsSpec.scala | 62 + .../enrichments/EnrichmentManagerSpec.scala | 1051 ++++++++++++++--- .../enrichments/FailureSpec.scala | 395 +++++++ .../enrichments/clientEnrichmentSpecs.scala | 22 +- .../enrichments/eventEnrichmentSpecs.scala | 23 +- .../enrichments/miscEnrichmentSpecs.scala | 10 +- .../pii/PiiPseudonymizerEnrichmentSpec.scala | 24 +- .../utils/IgluUtilsSpec.scala | 604 +++++++--- .../utils/ValidateAndReformatJsonSpec.scala | 51 - .../utils/conversionUtilsSpecs.scala | 42 +- .../kafka/src/main/resources/application.conf | 15 + .../AzureAuthenticationCallbackHandler.scala | 2 + .../Main.scala | 1 + .../snowplow/enrich/kafka/ConfigSpec.scala | 17 +- .../enrich/enrich-localstack-statsd.hocon | 60 + .../resources/enrich/enrich-localstack.hocon | 7 + .../snowplow/enrich/kinesis/Containers.scala | 159 +-- .../snowplow/enrich/kinesis/DockerPull.scala | 35 + .../enrich/kinesis/EnrichKinesisSpec.scala | 105 +- .../enrich/kinesis/KinesisConfig.scala | 7 +- .../snowplow/enrich/kinesis/utils.scala | 47 +- .../src/main/resources/application.conf | 17 + .../snowplow/enrich/kinesis/Main.scala | 1 + .../snowplow/enrich/kinesis/ConfigSpec.scala | 15 +- .../snowplow/enrich/nsq/Main.scala | 1 + .../snowplow/enrich/nsq/ConfigSpec.scala | 15 +- .../src/main/resources/application.conf | 12 + .../snowplow/enrich/pubsub/Main.scala | 1 + .../snowplow/enrich/pubsub/ConfigSpec.scala | 13 +- project/BuildSettings.scala | 5 +- project/Dependencies.scala | 2 + 71 files changed, 3326 insertions(+), 983 deletions(-) create mode 100644 modules/common/src/main/scala/com.snowplowanalytics.snowplow.enrich/common/enrichments/Failure.scala create mode 100644 modules/common/src/test/scala/com.snowplowanalytics.snowplow.enrich.common/enrichments/AtomicFieldsSpec.scala create mode 100644 modules/common/src/test/scala/com.snowplowanalytics.snowplow.enrich.common/enrichments/FailureSpec.scala delete mode 100644 modules/common/src/test/scala/com.snowplowanalytics.snowplow.enrich.common/utils/ValidateAndReformatJsonSpec.scala create mode 100644 modules/kinesis/src/it/resources/enrich/enrich-localstack-statsd.hocon create mode 100644 modules/kinesis/src/it/scala/com/snowplowanalytics/snowplow/enrich/kinesis/DockerPull.scala diff --git a/.github/workflows/test.yml b/.github/workflows/test.yml index 43e503e63..42fd5ed22 100644 --- a/.github/workflows/test.yml +++ b/.github/workflows/test.yml @@ -43,7 +43,7 @@ jobs: env: PGPASSWORD: supersecret1 - name: Run tests - run: SBT_OPTS="-Xms1G -Xmx8G -Xss4M -XX:MaxMetaspaceSize=1024M" sbt coverage +test + run: SBT_OPTS="-Xms1G -Xmx8G -Xss4M -XX:MaxMetaspaceSize=1024M" TESTCONTAINERS_RYUK_DISABLED=true sbt coverage +test env: OER_KEY: ${{ secrets.OER_KEY }} - name: Check Scala formatting diff --git a/build.sbt b/build.sbt index ac5004a2a..324fc296c 100644 --- a/build.sbt +++ b/build.sbt @@ -100,7 +100,8 @@ lazy val kinesisDistroless = project .settings(libraryDependencies ++= kinesisDependencies ++ Seq( // integration tests dependencies specs2CEIt, - testContainersIt + testContainersIt, + dockerJavaIt )) .settings(excludeDependencies ++= exclusions) .settings(addCompilerPlugin(betterMonadicFor)) 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 e22126f54..6929018f9 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 8980c023f..c7f336a96 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..9984e5b3d 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, env.metrics.incompleteCount) ).parSequence_ } @@ -272,6 +290,16 @@ object Enrich { Sync[F].unit } + def sinkIncomplete[F[_]: Sync]( + incomplete: List[AttributedData[Array[Byte]]], + maybeSink: Option[AttributedByteSink[F]], + incMetrics: Int => F[Unit] + ): F[Unit] = + maybeSink match { + case Some(sink) => sink(incomplete) *> incMetrics(incomplete.size) + 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..0858b1f64 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,11 +207,12 @@ 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 remoteAdaptersEnabled = file.remoteAdapters.configs.nonEmpty - metrics <- Resource.eval(Metrics.build[F](file.monitoring.metrics, remoteAdaptersEnabled)) + metrics <- Resource.eval(Metrics.build[F](file.monitoring.metrics, remoteAdaptersEnabled, incomplete.isDefined)) metadata <- Resource.eval(metadataReporter[F](file, processor.artifact, http4s)) assets = parsedConfigs.enrichmentConfigs.flatMap(_.filesToCache) remoteAdapters <- prepareRemoteAdapters[F](file.remoteAdapters, metrics) @@ -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/io/Metrics.scala b/modules/common-fs2/src/main/scala/com/snowplowanalytics/snowplow/enrich/common/fs2/io/Metrics.scala index 8b008766a..7bfe9c212 100644 --- a/modules/common-fs2/src/main/scala/com/snowplowanalytics/snowplow/enrich/common/fs2/io/Metrics.scala +++ b/modules/common-fs2/src/main/scala/com/snowplowanalytics/snowplow/enrich/common/fs2/io/Metrics.scala @@ -43,6 +43,9 @@ trait Metrics[F[_]] { /** Increment bad events */ def badCount(nb: Int): F[Unit] + /** Increment incomplete events */ + def incompleteCount(nb: Int): F[Unit] + /** Increment invalid enriched events count */ def invalidCount: F[Unit] @@ -63,6 +66,7 @@ object Metrics { val RawCounterName = "raw" val GoodCounterName = "good" val BadCounterName = "bad" + val IncompleteCounterName = "incomplete" val InvalidCounterName = "invalid_enriched" val RemoteAdaptersSuccessCounterName = "remote_adapters_success" val RemoteAdaptersFailureCounterName = "remote_adapters_failure" @@ -73,6 +77,7 @@ object Metrics { rawCount: Int, goodCount: Int, badCount: Int, + incompleteCount: Option[Int], invalidCount: Int, remoteAdaptersSuccessCount: Option[Int], remoteAdaptersFailureCount: Option[Int], @@ -85,33 +90,35 @@ object Metrics { def build[F[_]: Async]( config: MetricsReporters, - remoteAdaptersEnabled: Boolean + remoteAdaptersEnabled: Boolean, + incompleteEventsEnabled: Boolean ): F[Metrics[F]] = config match { case MetricsReporters(None, None, _) => noop[F].pure[F] - case MetricsReporters(statsd, stdout, _) => impl[F](statsd, stdout, remoteAdaptersEnabled) + case MetricsReporters(statsd, stdout, _) => impl[F](statsd, stdout, remoteAdaptersEnabled, incompleteEventsEnabled) } private def impl[F[_]: Async]( statsd: Option[MetricsReporters.StatsD], stdout: Option[MetricsReporters.Stdout], - remoteAdaptersEnabled: Boolean + remoteAdaptersEnabled: Boolean, + incompleteEventsEnabled: Boolean ): F[Metrics[F]] = for { - refsStatsd <- MetricRefs.init[F](remoteAdaptersEnabled) - refsStdout <- MetricRefs.init[F](remoteAdaptersEnabled) + refsStatsd <- MetricRefs.init[F](remoteAdaptersEnabled, incompleteEventsEnabled) + refsStdout <- MetricRefs.init[F](remoteAdaptersEnabled, incompleteEventsEnabled) } yield new Metrics[F] { def report: Stream[F, Unit] = { val rep1 = statsd .map { config => - reporterStream(StatsDReporter.make[F](config), refsStatsd, config.period, remoteAdaptersEnabled) + reporterStream(StatsDReporter.make[F](config), refsStatsd, config.period, remoteAdaptersEnabled, incompleteEventsEnabled) } .getOrElse(Stream.never[F]) val rep2 = stdout .map { config => - reporterStream(Resource.eval(stdoutReporter(config)), refsStdout, config.period, remoteAdaptersEnabled) + reporterStream(Resource.eval(stdoutReporter(config)), refsStdout, config.period, remoteAdaptersEnabled, incompleteEventsEnabled) } .getOrElse(Stream.never[F]) @@ -142,6 +149,10 @@ object Metrics { refsStatsd.badCount.update(_ + nb) *> refsStdout.badCount.update(_ + nb) + def incompleteCount(nb: Int): F[Unit] = + refsStatsd.incompleteCount.update(_.map(_ + nb)) *> + refsStdout.incompleteCount.update(_.map(_ + nb)) + def invalidCount: F[Unit] = refsStatsd.invalidCount.update(_ + 1) *> refsStdout.invalidCount.update(_ + 1) @@ -161,6 +172,7 @@ object Metrics { rawCount: Ref[F, Int], goodCount: Ref[F, Int], badCount: Ref[F, Int], + incompleteCount: Ref[F, Option[Int]], invalidCount: Ref[F, Int], remoteAdaptersSuccessCount: Ref[F, Option[Int]], remoteAdaptersFailureCount: Ref[F, Option[Int]], @@ -168,12 +180,13 @@ object Metrics { ) private object MetricRefs { - def init[F[_]: Sync](remoteAdaptersEnabled: Boolean): F[MetricRefs[F]] = + def init[F[_]: Sync](remoteAdaptersEnabled: Boolean, incompleteEventsEnabled: Boolean): F[MetricRefs[F]] = for { latency <- Ref.of[F, Option[Long]](None) rawCounter <- Ref.of[F, Int](0) goodCounter <- Ref.of[F, Int](0) badCounter <- Ref.of[F, Int](0) + incompleteCounter <- Ref.of[F, Option[Int]](if (incompleteEventsEnabled) Some(0) else None) invalidCounter <- Ref.of[F, Int](0) remoteAdaptersSuccessCounter <- Ref.of[F, Option[Int]](if (remoteAdaptersEnabled) Some(0) else None) remoteAdaptersFailureCounter <- Ref.of[F, Option[Int]](if (remoteAdaptersEnabled) Some(0) else None) @@ -183,18 +196,24 @@ object Metrics { rawCounter, goodCounter, badCounter, + incompleteCounter, invalidCounter, remoteAdaptersSuccessCounter, remoteAdaptersFailureCounter, remoteAdaptersTimeoutCounter ) - def snapshot[F[_]: Monad](refs: MetricRefs[F], remoteAdaptersEnabled: Boolean): F[MetricSnapshot] = + def snapshot[F[_]: Monad]( + refs: MetricRefs[F], + remoteAdaptersEnabled: Boolean, + incompleteEventsEnabled: Boolean + ): F[MetricSnapshot] = for { latency <- refs.latency.getAndSet(None) rawCount <- refs.rawCount.getAndSet(0) goodCount <- refs.goodCount.getAndSet(0) badCount <- refs.badCount.getAndSet(0) + incompleteCount <- refs.incompleteCount.getAndSet(if (incompleteEventsEnabled) Some(0) else None) invalidCount <- refs.invalidCount.getAndSet(0) remoteAdaptersSuccessCount <- refs.remoteAdaptersSuccessCount.getAndSet(if (remoteAdaptersEnabled) Some(0) else None) remoteAdaptersFailureCount <- refs.remoteAdaptersFailureCount.getAndSet(if (remoteAdaptersEnabled) Some(0) else None) @@ -203,6 +222,7 @@ object Metrics { rawCount, goodCount, badCount, + incompleteCount, invalidCount, remoteAdaptersSuccessCount, remoteAdaptersFailureCount, @@ -214,12 +234,13 @@ object Metrics { reporter: Resource[F, Reporter[F]], metrics: MetricRefs[F], period: FiniteDuration, - remoteAdaptersEnabled: Boolean + remoteAdaptersEnabled: Boolean, + incompleteEventsEnabled: Boolean ): Stream[F, Unit] = for { rep <- Stream.resource(reporter) _ <- Stream.fixedDelay[F](period) - snapshot <- Stream.eval(MetricRefs.snapshot(metrics, remoteAdaptersEnabled)) + snapshot <- Stream.eval(MetricRefs.snapshot(metrics, remoteAdaptersEnabled, incompleteEventsEnabled)) _ <- Stream.eval(rep.report(snapshot)) } yield () @@ -234,6 +255,9 @@ object Metrics { _ <- logger.info(s"${MetricsReporters.normalizeMetric(config.prefix, RawCounterName)} = ${snapshot.rawCount}") _ <- logger.info(s"${MetricsReporters.normalizeMetric(config.prefix, GoodCounterName)} = ${snapshot.goodCount}") _ <- logger.info(s"${MetricsReporters.normalizeMetric(config.prefix, BadCounterName)} = ${snapshot.badCount}") + _ <- snapshot.incompleteCount + .map(cnt => logger.info(s"${MetricsReporters.normalizeMetric(config.prefix, IncompleteCounterName)} = $cnt")) + .getOrElse(Applicative[F].unit) _ <- logger.info(s"${MetricsReporters.normalizeMetric(config.prefix, InvalidCounterName)} = ${snapshot.invalidCount}") _ <- snapshot.enrichLatency .map(latency => logger.info(s"${MetricsReporters.normalizeMetric(config.prefix, LatencyGaugeName)} = $latency")) @@ -257,6 +281,7 @@ object Metrics { def rawCount(nb: Int): F[Unit] = Applicative[F].unit def goodCount(nb: Int): F[Unit] = Applicative[F].unit def badCount(nb: Int): F[Unit] = Applicative[F].unit + def incompleteCount(nb: Int): F[Unit] = Applicative[F].unit def invalidCount: F[Unit] = Applicative[F].unit def remoteAdaptersSuccessCount: F[Unit] = Applicative[F].unit def remoteAdaptersFailureCount: F[Unit] = Applicative[F].unit diff --git a/modules/common-fs2/src/main/scala/com/snowplowanalytics/snowplow/enrich/common/fs2/io/StatsDReporter.scala b/modules/common-fs2/src/main/scala/com/snowplowanalytics/snowplow/enrich/common/fs2/io/StatsDReporter.scala index 152e2a7c8..9a876d531 100644 --- a/modules/common-fs2/src/main/scala/com/snowplowanalytics/snowplow/enrich/common/fs2/io/StatsDReporter.scala +++ b/modules/common-fs2/src/main/scala/com/snowplowanalytics/snowplow/enrich/common/fs2/io/StatsDReporter.scala @@ -75,6 +75,7 @@ object StatsDReporter { Metrics.BadCounterName -> snapshot.badCount.toString, Metrics.InvalidCounterName -> snapshot.invalidCount.toString ) ++ snapshot.enrichLatency.map(l => Metrics.LatencyGaugeName -> l.toString) ++ + snapshot.incompleteCount.map(cnt => Metrics.IncompleteCounterName -> cnt.toString) ++ snapshot.remoteAdaptersSuccessCount.map(cnt => Metrics.RemoteAdaptersSuccessCounterName -> cnt.toString) ++ snapshot.remoteAdaptersFailureCount.map(cnt => Metrics.RemoteAdaptersFailureCounterName -> cnt.toString) ++ snapshot.remoteAdaptersTimeoutCount.map(cnt => Metrics.RemoteAdaptersTimeoutCounterName -> cnt.toString) 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..31c206697 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 @@ -38,7 +38,7 @@ import com.snowplowanalytics.iglu.core.{SchemaKey, SchemaVer} import com.snowplowanalytics.snowplow.analytics.scalasdk.Event import com.snowplowanalytics.snowplow.badrows.{BadRow, Failure, FailureDetails, Processor, Payload => BadRowPayload} -import com.snowplowanalytics.snowplow.enrich.common.enrichments.registry.IpLookupsEnrichment +import com.snowplowanalytics.snowplow.enrich.common.enrichments.registry.{IpLookupsEnrichment, JavascriptScriptEnrichment} import com.snowplowanalytics.snowplow.enrich.common.enrichments.{AtomicFields, MiscEnrichments} import com.snowplowanalytics.snowplow.enrich.common.loaders.CollectorPayload import com.snowplowanalytics.snowplow.enrich.common.outputs.EnrichedEvent @@ -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") } } @@ -183,21 +187,55 @@ class EnrichSpec extends Specification with CatsEffect with ScalaCheck { } "enrich" should { - "update metrics with raw, good and bad counters" in { - val input = Stream.emits(List(Array.empty[Byte], EnrichSpec.payload)) - TestEnvironment.make(input).use { test => + "update metrics with raw, good, bad and incomplete counters" in { + val script = """ + function process(event, params) { + if(event.getUser_ipaddress() == "foo") { + throw "BOOM"; + } + return [ ]; + }""" + val config = json"""{ + "parameters": { + "script": ${ConversionUtils.encodeBase64Url(script)} + } + }""" + val schemaKey = SchemaKey( + "com.snowplowanalytics.snowplow", + "javascript_script_config", + "jsonschema", + SchemaVer.Full(1, 0, 0) + ) + val jsEnrichConf = + JavascriptScriptEnrichment.parse(config, schemaKey).toOption.get + + val context = EnrichSpec.context.copy(ipAddress = Some("foo")) + val payload = EnrichSpec.collectorPayload.copy(context = context) + + val input = Stream.emits( + List( + Array.empty[Byte], + EnrichSpec.payload, + payload.toRaw + ) + ) + + TestEnvironment.make(input, List(jsEnrichConf)).use { test => val enrichStream = Enrich.run[IO, Array[Byte]](test.env) for { _ <- enrichStream.compile.drain bad <- test.bad good <- test.good + incomplete <- test.incomplete counter <- test.counter.get } yield { - (counter.raw must_== 2L) + (counter.raw must_== 3L) (counter.good must_== 1L) - (counter.bad must_== 1L) - (bad.size must_== 1) + (counter.bad must_== 2L) + (counter.incomplete must_== 1L) + (bad.size must_== 2) (good.size must_== 1) + (incomplete.size must_== 1) } } } @@ -245,9 +283,10 @@ class EnrichSpec extends Specification with CatsEffect with ScalaCheck { test .run(_.copy(assetsUpdatePeriod = Some(1800.millis))) .map { - case (bad, pii, good) => + case (bad, pii, good, incomplete) => (bad must be empty) (pii must be empty) + (incomplete must be empty) (good must contain(exactly(one, two))) } } @@ -449,16 +488,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 +530,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/enrichments/ApiRequestEnrichmentSpec.scala b/modules/common-fs2/src/test/scala/com/snowplowanalytics/snowplow/enrich/common/fs2/enrichments/ApiRequestEnrichmentSpec.scala index 5e6e09b47..fc7f003ad 100644 --- a/modules/common-fs2/src/test/scala/com/snowplowanalytics/snowplow/enrich/common/fs2/enrichments/ApiRequestEnrichmentSpec.scala +++ b/modules/common-fs2/src/test/scala/com/snowplowanalytics/snowplow/enrich/common/fs2/enrichments/ApiRequestEnrichmentSpec.scala @@ -82,9 +82,10 @@ class ApiRequestEnrichmentSpec extends Specification with CatsEffect { val testWithHttp = HttpServer.resource *> TestEnvironment.make(input, List(enrichment)) testWithHttp.use { test => test.run().map { - case (bad, pii, good) => + case (bad, pii, good, incomplete) => bad must beEmpty pii must beEmpty + incomplete must beEmpty good.map(_.derived_contexts) must contain(exactly(expected)) } } @@ -117,9 +118,10 @@ class ApiRequestEnrichmentSpec extends Specification with CatsEffect { TestEnvironment.make(input, List(enrichment)).use { test => test.run().map { - case (bad, pii, good) => + case (bad, pii, good, incomplete) => good must beEmpty pii must beEmpty + incomplete must haveSize(nbEvents) bad.collect { case ef: BadRow.EnrichmentFailures => ef } must haveSize(nbEvents) } } diff --git a/modules/common-fs2/src/test/scala/com/snowplowanalytics/snowplow/enrich/common/fs2/enrichments/IabEnrichmentSpec.scala b/modules/common-fs2/src/test/scala/com/snowplowanalytics/snowplow/enrich/common/fs2/enrichments/IabEnrichmentSpec.scala index ac75076bc..504db244c 100644 --- a/modules/common-fs2/src/test/scala/com/snowplowanalytics/snowplow/enrich/common/fs2/enrichments/IabEnrichmentSpec.scala +++ b/modules/common-fs2/src/test/scala/com/snowplowanalytics/snowplow/enrich/common/fs2/enrichments/IabEnrichmentSpec.scala @@ -59,9 +59,10 @@ class IabEnrichmentSpec extends Specification with CatsEffect { val testWithHttp = HttpServer.resource *> TestEnvironment.make(input, List(IabEnrichmentSpec.enrichmentConf)) testWithHttp.use { test => test.run().map { - case (bad, pii, good) => + case (bad, pii, good, incomplete) => (bad must be empty) (pii must be empty) + (incomplete must be empty) good.map(_.derived_contexts) must contain(exactly(expected)) } } @@ -95,9 +96,10 @@ class IabEnrichmentSpec extends Specification with CatsEffect { val testWithHttp = HttpServer.resource *> TestEnvironment.make(input, List(IabEnrichmentSpec.enrichmentConf)) testWithHttp.use { test => test.run(_.copy(assetsUpdatePeriod = Some(1800.millis))).map { - case (bad, pii, good) => + case (bad, pii, good, incomplete) => (bad must be empty) (pii must be empty) + (incomplete must be empty) good.map(_.derived_contexts) must contain(exactly(expectedOne, expectedTwo)) } } diff --git a/modules/common-fs2/src/test/scala/com/snowplowanalytics/snowplow/enrich/common/fs2/enrichments/YauaaEnrichmentSpec.scala b/modules/common-fs2/src/test/scala/com/snowplowanalytics/snowplow/enrich/common/fs2/enrichments/YauaaEnrichmentSpec.scala index 3e6951da3..254bc6201 100644 --- a/modules/common-fs2/src/test/scala/com/snowplowanalytics/snowplow/enrich/common/fs2/enrichments/YauaaEnrichmentSpec.scala +++ b/modules/common-fs2/src/test/scala/com/snowplowanalytics/snowplow/enrich/common/fs2/enrichments/YauaaEnrichmentSpec.scala @@ -90,9 +90,10 @@ class YauaaEnrichmentSpec extends Specification with CatsEffect { TestEnvironment.make(input, List(enrichment)).use { test => test.run().map { - case (bad, pii, good) => + case (bad, pii, good, incomplete) => (bad must be empty) (pii must be empty) + (incomplete must be empty) good.map(_.derived_contexts) must contain(exactly(expected)) } } diff --git a/modules/common-fs2/src/test/scala/com/snowplowanalytics/snowplow/enrich/common/fs2/io/StatsDReporterSpec.scala b/modules/common-fs2/src/test/scala/com/snowplowanalytics/snowplow/enrich/common/fs2/io/StatsDReporterSpec.scala index 1907512e3..bd275f2e8 100644 --- a/modules/common-fs2/src/test/scala/com/snowplowanalytics/snowplow/enrich/common/fs2/io/StatsDReporterSpec.scala +++ b/modules/common-fs2/src/test/scala/com/snowplowanalytics/snowplow/enrich/common/fs2/io/StatsDReporterSpec.scala @@ -21,7 +21,7 @@ class StatsDReporterSpec extends Specification { "StatsDeporter" should { "serialize metrics" in { - val snapshot = Metrics.MetricSnapshot(Some(10000L), 10, 20, 30, 0, Some(40), Some(0), Some(0)) + val snapshot = Metrics.MetricSnapshot(Some(10000L), 10, 20, 30, Some(25), 0, Some(40), Some(0), Some(0)) val result = StatsDReporter.serializedMetrics(snapshot, TestConfig) @@ -30,6 +30,7 @@ class StatsDReporterSpec extends Specification { "snowplow.test.raw:10|c|#tag1:abc", "snowplow.test.good:20|c|#tag1:abc", "snowplow.test.bad:30|c|#tag1:abc", + "snowplow.test.incomplete:25|c|#tag1:abc", "snowplow.test.latency:10000|g|#tag1:abc", "snowplow.test.invalid_enriched:0|c|#tag1:abc", "snowplow.test.remote_adapters_success:40|c|#tag1:abc", @@ -40,7 +41,7 @@ class StatsDReporterSpec extends Specification { } "serialize metrics when latency is empty" in { - val snapshot = Metrics.MetricSnapshot(None, 10, 20, 30, 40, Some(40), Some(0), Some(0)) + val snapshot = Metrics.MetricSnapshot(None, 10, 20, 30, None, 40, Some(40), Some(0), Some(0)) val result = StatsDReporter.serializedMetrics(snapshot, TestConfig) @@ -58,7 +59,7 @@ class StatsDReporterSpec extends Specification { } "serialize metrics when remote adapter metrics are empty" in { - val snapshot = Metrics.MetricSnapshot(Some(10000L), 10, 20, 30, 40, None, None, None) + val snapshot = Metrics.MetricSnapshot(Some(10000L), 10, 20, 30, Some(25), 40, None, None, None) val result = StatsDReporter.serializedMetrics(snapshot, TestConfig) @@ -67,6 +68,7 @@ class StatsDReporterSpec extends Specification { "snowplow.test.raw:10|c|#tag1:abc", "snowplow.test.good:20|c|#tag1:abc", "snowplow.test.bad:30|c|#tag1:abc", + "snowplow.test.incomplete:25|c|#tag1:abc", "snowplow.test.latency:10000|g|#tag1:abc", "snowplow.test.invalid_enriched:40|c|#tag1:abc" ) diff --git a/modules/common-fs2/src/test/scala/com/snowplowanalytics/snowplow/enrich/common/fs2/test/Counter.scala b/modules/common-fs2/src/test/scala/com/snowplowanalytics/snowplow/enrich/common/fs2/test/Counter.scala index e48ac5476..1d8bf01fb 100644 --- a/modules/common-fs2/src/test/scala/com/snowplowanalytics/snowplow/enrich/common/fs2/test/Counter.scala +++ b/modules/common-fs2/src/test/scala/com/snowplowanalytics/snowplow/enrich/common/fs2/test/Counter.scala @@ -24,6 +24,7 @@ case class Counter( raw: Long, good: Long, bad: Long, + incomplete: Long, latency: Option[Long], invalid: Long, remoteAdaptersSuccessCount: Option[Long], @@ -32,7 +33,7 @@ case class Counter( ) object Counter { - val empty: Counter = Counter(0L, 0L, 0L, None, 0L, None, None, None) + val empty: Counter = Counter(0L, 0L, 0L, 0L, None, 0L, None, None, None) def make[F[_]: Sync]: F[Ref[F, Counter]] = Ref.of[F, Counter](empty) @@ -56,6 +57,9 @@ object Counter { def badCount(nb: Int): F[Unit] = ref.update(cnt => cnt.copy(bad = cnt.bad + nb)) + def incompleteCount(nb: Int): F[Unit] = + ref.update(cnt => cnt.copy(incomplete = cnt.incomplete + nb)) + def invalidCount: F[Unit] = ref.update(cnt => cnt.copy(invalid = cnt.invalid + 1)) 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..cf089ce93 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 @@ -56,7 +56,8 @@ case class TestEnvironment[A]( counter: Ref[IO, Counter], good: IO[Vector[AttributedData[Array[Byte]]]], pii: IO[Vector[AttributedData[Array[Byte]]]], - bad: IO[Vector[Array[Byte]]] + bad: IO[Vector[Array[Byte]]], + incomplete: IO[Vector[AttributedData[Array[Byte]]]] ) { /** @@ -70,7 +71,7 @@ case class TestEnvironment[A]( */ def run( updateEnv: Environment[IO, A] => Environment[IO, A] = identity - ): IO[(Vector[BadRow], Vector[Event], Vector[Event])] = { + ): IO[(Vector[BadRow], Vector[Event], Vector[Event], Vector[Event])] = { val updatedEnv = updateEnv(env) val stream = Enrich .run[IO, A](updatedEnv) @@ -88,9 +89,11 @@ case class TestEnvironment[A]( goodVec <- good piiVec <- pii badVec <- bad + incompleteVec <- incomplete } yield (badVec.map(TestEnvironment.parseBad(_)), piiVec.flatMap(p => EnrichSpec.normalize(new String(p.data, UTF_8)).toOption), - goodVec.flatMap(g => EnrichSpec.normalize(new String(g.data, UTF_8)).toOption) + goodVec.flatMap(g => EnrichSpec.normalize(new String(g.data, UTF_8)).toOption), + incompleteVec.flatMap(i => EnrichSpec.normalize(new String(i.data, UTF_8)).toOption) ) } @@ -126,6 +129,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 +145,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, @@ -160,7 +165,7 @@ object TestEnvironment extends CatsEffect { AtomicFields.from(valueLimits = Map.empty) ) _ <- Resource.eval(logger.info("TestEnvironment initialized")) - } yield TestEnvironment(environment, counter, goodRef.get, piiRef.get, badRef.get) + } yield TestEnvironment(environment, counter, goodRef.get, piiRef.get, badRef.get, incompleteRef.get) def parseBad(bytes: Array[Byte]): BadRow = { val badRowStr = new String(bytes, UTF_8) 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 736cd661e..6ac39c633 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 @@ -10,7 +10,7 @@ */ package com.snowplowanalytics.snowplow.enrich.common -import cats.data.{Validated, ValidatedNel} +import cats.data.{Ior, Validated, ValidatedNel} import cats.effect.kernel.Sync import cats.implicits._ @@ -40,10 +40,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 @@ -52,8 +48,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[_]: Sync]( adapterRegistry: AdapterRegistry[F], @@ -65,8 +59,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 @@ -84,16 +79,17 @@ object EtlPipeline { featureFlags, invalidCount, registryLookup, - atomicFields + atomicFields, + emitIncomplete ) - .toValidated + .value } case Validated.Invalid(badRow) => - Sync[F].pure(List(badRow.invalid[EnrichedEvent])) + Sync[F].pure(List(Ior.left(badRow))) } case Validated.Invalid(badRows) => - Sync[F].pure(badRows.map(_.invalid[EnrichedEvent])).map(_.toList) + Sync[F].pure(badRows.toList.map(br => Ior.left(br))) case Validated.Valid(None) => - Sync[F].pure(List.empty[Validated[BadRow, EnrichedEvent]]) + Sync[F].pure(Nil) } } diff --git a/modules/common/src/main/scala/com.snowplowanalytics.snowplow.enrich/common/enrichments/AtomicFields.scala b/modules/common/src/main/scala/com.snowplowanalytics.snowplow.enrich/common/enrichments/AtomicFields.scala index a3f7150a6..a34b028bb 100644 --- a/modules/common/src/main/scala/com.snowplowanalytics.snowplow.enrich/common/enrichments/AtomicFields.scala +++ b/modules/common/src/main/scala/com.snowplowanalytics.snowplow.enrich/common/enrichments/AtomicFields.scala @@ -12,6 +12,8 @@ package com.snowplowanalytics.snowplow.enrich.common.enrichments import cats.data.NonEmptyList +import io.circe.syntax._ + import com.snowplowanalytics.snowplow.badrows.FailureDetails import com.snowplowanalytics.iglu.client.ClientError.ValidationError @@ -31,97 +33,98 @@ object AtomicFields { final case class AtomicField( name: String, - enrichedValueExtractor: EnrichedEvent => String + enrichedValueExtractor: EnrichedEvent => String, + nullify: EnrichedEvent => Unit ) final case class LimitedAtomicField(value: AtomicField, limit: Int) // format: off val supportedFields: List[AtomicField] = List( - AtomicField(name = "app_id", _.app_id ), - AtomicField(name = "platform", _.platform ), - AtomicField(name = "event", _.event ), - AtomicField(name = "event_id", _.event ), - AtomicField(name = "name_tracker", _.name_tracker ), - AtomicField(name = "v_tracker", _.v_tracker ), - AtomicField(name = "v_collector", _.v_collector ), - AtomicField(name = "v_etl", _.v_etl ), - AtomicField(name = "user_id", _.user_id ), - AtomicField(name = "user_ipaddress", _.user_ipaddress ), - AtomicField(name = "user_fingerprint", _.user_fingerprint ), - AtomicField(name = "domain_userid", _.domain_userid ), - AtomicField(name = "network_userid", _.network_userid ), - AtomicField(name = "geo_country", _.geo_country ), - AtomicField(name = "geo_region", _.geo_region ), - AtomicField(name = "geo_city", _.geo_city ), - AtomicField(name = "geo_zipcode", _.geo_zipcode ), - AtomicField(name = "geo_region_name", _.geo_region_name ), - AtomicField(name = "ip_isp", _.ip_isp ), - AtomicField(name = "ip_organization", _.ip_organization ), - AtomicField(name = "ip_domain", _.ip_domain ), - AtomicField(name = "ip_netspeed", _.ip_netspeed ), - AtomicField(name = "page_url", _.page_url ), - AtomicField(name = "page_title", _.page_title ), - AtomicField(name = "page_referrer", _.page_referrer ), - AtomicField(name = "page_urlscheme", _.page_urlscheme ), - AtomicField(name = "page_urlhost", _.page_urlhost ), - AtomicField(name = "page_urlpath", _.page_urlpath ), - AtomicField(name = "page_urlquery", _.page_urlquery ), - AtomicField(name = "page_urlfragment", _.page_urlfragment ), - AtomicField(name = "refr_urlscheme", _.refr_urlscheme ), - AtomicField(name = "refr_urlhost", _.refr_urlhost ), - AtomicField(name = "refr_urlpath", _.refr_urlpath ), - AtomicField(name = "refr_urlquery", _.refr_urlquery ), - AtomicField(name = "refr_urlfragment", _.refr_urlfragment ), - AtomicField(name = "refr_medium", _.refr_medium ), - AtomicField(name = "refr_source", _.refr_source ), - AtomicField(name = "refr_term", _.refr_term ), - AtomicField(name = "mkt_medium", _.mkt_medium ), - AtomicField(name = "mkt_source", _.mkt_source ), - AtomicField(name = "mkt_term", _.mkt_term ), - AtomicField(name = "mkt_content", _.mkt_content ), - AtomicField(name = "mkt_campaign", _.mkt_campaign ), - AtomicField(name = "se_category", _.se_category ), - AtomicField(name = "se_action", _.se_action ), - AtomicField(name = "se_label", _.se_label ), - AtomicField(name = "se_property", _.se_property ), - AtomicField(name = "tr_orderid", _.tr_orderid ), - AtomicField(name = "tr_affiliation", _.tr_affiliation ), - AtomicField(name = "tr_city", _.tr_city ), - AtomicField(name = "tr_state", _.tr_state ), - AtomicField(name = "tr_country", _.tr_country ), - AtomicField(name = "ti_orderid", _.ti_orderid ), - AtomicField(name = "ti_sku", _.ti_sku ), - AtomicField(name = "ti_name", _.ti_name ), - AtomicField(name = "ti_category", _.ti_category ), - AtomicField(name = "useragent", _.useragent ), - AtomicField(name = "br_name", _.br_name ), - AtomicField(name = "br_family", _.br_family ), - AtomicField(name = "br_version", _.br_version ), - AtomicField(name = "br_type", _.br_type ), - AtomicField(name = "br_renderengine", _.br_renderengine ), - AtomicField(name = "br_lang", _.br_lang ), - AtomicField(name = "br_colordepth", _.br_colordepth ), - AtomicField(name = "os_name", _.os_name ), - AtomicField(name = "os_family", _.os_family ), - AtomicField(name = "os_manufacturer", _.os_manufacturer ), - AtomicField(name = "os_timezone", _.os_timezone ), - AtomicField(name = "dvce_type", _.dvce_type ), - AtomicField(name = "doc_charset", _.doc_charset ), - AtomicField(name = "tr_currency", _.tr_currency ), - AtomicField(name = "ti_currency", _.ti_currency ), - AtomicField(name = "base_currency", _.base_currency ), - AtomicField(name = "geo_timezone", _.geo_timezone ), - AtomicField(name = "mkt_clickid", _.mkt_clickid ), - AtomicField(name = "mkt_network", _.mkt_network ), - AtomicField(name = "etl_tags", _.etl_tags ), - AtomicField(name = "refr_domain_userid", _.refr_domain_userid ), - AtomicField(name = "domain_sessionid", _.domain_sessionid ), - AtomicField(name = "event_vendor", _.event_vendor ), - AtomicField(name = "event_name", _.event_name ), - AtomicField(name = "event_format", _.event_format ), - AtomicField(name = "event_version", _.event_version ), - AtomicField(name = "event_fingerprint", _.event_fingerprint ), + AtomicField(name = "app_id", _.app_id, _.app_id = null), + AtomicField(name = "platform", _.platform, _.platform = null), + AtomicField(name = "event", _.event, _.event = null), + AtomicField(name = "event_id", _.event_id, _ => ()), // required in loading + AtomicField(name = "name_tracker", _.name_tracker, _.name_tracker = null), + AtomicField(name = "v_tracker", _.v_tracker, _.v_tracker = null), + AtomicField(name = "v_collector", _.v_collector, _ => ()), // required in loading + AtomicField(name = "v_etl", _.v_etl, _ => ()), // required in loading + AtomicField(name = "user_id", _.user_id, _.user_id = null), + AtomicField(name = "user_ipaddress", _.user_ipaddress, _.user_ipaddress = null), + AtomicField(name = "user_fingerprint", _.user_fingerprint, _.user_fingerprint = null), + AtomicField(name = "domain_userid", _.domain_userid, _.domain_userid = null), + AtomicField(name = "network_userid", _.network_userid, _.network_userid = null), + AtomicField(name = "geo_country", _.geo_country, _.geo_country = null), + AtomicField(name = "geo_region", _.geo_region, _.geo_region = null), + AtomicField(name = "geo_city", _.geo_city, _.geo_city = null), + AtomicField(name = "geo_zipcode", _.geo_zipcode, _.geo_zipcode = null), + AtomicField(name = "geo_region_name", _.geo_region_name, _.geo_region_name = null), + AtomicField(name = "ip_isp", _.ip_isp, _.ip_isp = null), + AtomicField(name = "ip_organization", _.ip_organization, _.ip_organization = null), + AtomicField(name = "ip_domain", _.ip_domain, _.ip_domain = null), + AtomicField(name = "ip_netspeed", _.ip_netspeed, _.ip_netspeed = null), + AtomicField(name = "page_url", _.page_url, _.page_url = null), + AtomicField(name = "page_title", _.page_title, _.page_title = null), + AtomicField(name = "page_referrer", _.page_referrer, _.page_referrer = null), + AtomicField(name = "page_urlscheme", _.page_urlscheme, _.page_urlscheme = null), + AtomicField(name = "page_urlhost", _.page_urlhost, _.page_urlhost = null), + AtomicField(name = "page_urlpath", _.page_urlpath, _.page_urlpath = null), + AtomicField(name = "page_urlquery", _.page_urlquery, _.page_urlquery = null), + AtomicField(name = "page_urlfragment", _.page_urlfragment, _.page_urlfragment = null), + AtomicField(name = "refr_urlscheme", _.refr_urlscheme, _.refr_urlscheme = null), + AtomicField(name = "refr_urlhost", _.refr_urlhost, _.refr_urlhost = null), + AtomicField(name = "refr_urlpath", _.refr_urlpath, _.refr_urlpath = null), + AtomicField(name = "refr_urlquery", _.refr_urlquery, _.refr_urlquery = null), + AtomicField(name = "refr_urlfragment", _.refr_urlfragment, _.refr_urlfragment = null), + AtomicField(name = "refr_medium", _.refr_medium, _.refr_medium = null), + AtomicField(name = "refr_source", _.refr_source, _.refr_source = null), + AtomicField(name = "refr_term", _.refr_term, _.refr_term = null), + AtomicField(name = "mkt_medium", _.mkt_medium, _.mkt_medium = null), + AtomicField(name = "mkt_source", _.mkt_source, _.mkt_source = null), + AtomicField(name = "mkt_term", _.mkt_term, _.mkt_term = null), + AtomicField(name = "mkt_content", _.mkt_content, _.mkt_content = null), + AtomicField(name = "mkt_campaign", _.mkt_campaign, _.mkt_campaign = null), + AtomicField(name = "se_category", _.se_category, _.se_category = null), + AtomicField(name = "se_action", _.se_action, _.se_action = null), + AtomicField(name = "se_label", _.se_label, _.se_label = null), + AtomicField(name = "se_property", _.se_property, _.se_property = null), + AtomicField(name = "tr_orderid", _.tr_orderid, _.tr_orderid = null), + AtomicField(name = "tr_affiliation", _.tr_affiliation, _.tr_affiliation = null), + AtomicField(name = "tr_city", _.tr_city, _.tr_city = null), + AtomicField(name = "tr_state", _.tr_state, _.tr_state = null), + AtomicField(name = "tr_country", _.tr_country, _.tr_country = null), + AtomicField(name = "ti_orderid", _.ti_orderid, _.ti_orderid = null), + AtomicField(name = "ti_sku", _.ti_sku, _.ti_sku = null), + AtomicField(name = "ti_name", _.ti_name, _.ti_name = null), + AtomicField(name = "ti_category", _.ti_category, _.ti_category = null), + AtomicField(name = "useragent", _.useragent, _.useragent = null), + AtomicField(name = "br_name", _.br_name, _.br_name = null), + AtomicField(name = "br_family", _.br_family, _.br_family = null), + AtomicField(name = "br_version", _.br_version, _.br_version = null), + AtomicField(name = "br_type", _.br_type, _.br_type = null), + AtomicField(name = "br_renderengine", _.br_renderengine, _.br_renderengine = null), + AtomicField(name = "br_lang", _.br_lang, _.br_lang = null), + AtomicField(name = "br_colordepth", _.br_colordepth, _.br_colordepth = null), + AtomicField(name = "os_name", _.os_name, _.os_name = null), + AtomicField(name = "os_family", _.os_family, _.os_family = null), + AtomicField(name = "os_manufacturer", _.os_manufacturer, _.os_manufacturer = null), + AtomicField(name = "os_timezone", _.os_timezone, _.os_timezone = null), + AtomicField(name = "dvce_type", _.dvce_type, _.dvce_type = null), + AtomicField(name = "doc_charset", _.doc_charset, _.doc_charset = null), + AtomicField(name = "tr_currency", _.tr_currency, _.tr_currency = null), + AtomicField(name = "ti_currency", _.ti_currency, _.ti_currency = null), + AtomicField(name = "base_currency", _.base_currency, _.base_currency = null), + AtomicField(name = "geo_timezone", _.geo_timezone, _.geo_timezone = null), + AtomicField(name = "mkt_clickid", _.mkt_clickid, _.mkt_clickid = null), + AtomicField(name = "mkt_network", _.mkt_network, _.mkt_network = null), + AtomicField(name = "etl_tags", _.etl_tags, _.etl_tags = null), + AtomicField(name = "refr_domain_userid", _.refr_domain_userid, _.refr_domain_userid = null), + AtomicField(name = "domain_sessionid", _.domain_sessionid, _.domain_sessionid = null), + AtomicField(name = "event_vendor", _.event_vendor, _.event_vendor = null), + AtomicField(name = "event_name", _.event_name, _.event_name = null), + AtomicField(name = "event_format", _.event_format, _.event_format = null), + AtomicField(name = "event_version", _.event_version, _.event_version = null), + AtomicField(name = "event_fingerprint", _.event_fingerprint, _.event_fingerprint = null), ) // format: on @@ -134,12 +137,19 @@ object AtomicFields { AtomicFields(withLimits) } - def errorsToSchemaViolation(errors: NonEmptyList[AtomicError]): FailureDetails.SchemaViolation = { + def errorsToSchemaViolation(errors: NonEmptyList[AtomicError]): Failure.SchemaViolation = { val clientError = ValidationError(ValidatorError.InvalidData(errors.map(_.toValidatorReport)), None) - FailureDetails.SchemaViolation.IgluError( - AtomicFields.atomicSchema, - clientError + val failureData = errors.toList.map(e => e.field := e.value).toMap.asJson + + Failure.SchemaViolation( + schemaViolation = FailureDetails.SchemaViolation.IgluError( + AtomicFields.atomicSchema, + clientError + ), + // Source atomic field and actual value of the field should be already on the ValidatorReport list + source = AtomicError.source, + data = failureData ) } } diff --git a/modules/common/src/main/scala/com.snowplowanalytics.snowplow.enrich/common/enrichments/AtomicFieldsLengthValidator.scala b/modules/common/src/main/scala/com.snowplowanalytics.snowplow.enrich/common/enrichments/AtomicFieldsLengthValidator.scala index 62024ba2c..b995e94b6 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,12 +14,9 @@ import org.slf4j.LoggerFactory import cats.Monad import cats.data.Validated.{Invalid, Valid} -import cats.data.NonEmptyList - +import cats.data.{Ior, IorT, NonEmptyList} import cats.implicits._ -import com.snowplowanalytics.snowplow.badrows.FailureDetails - import com.snowplowanalytics.snowplow.enrich.common.enrichments.AtomicFields.LimitedAtomicField import com.snowplowanalytics.snowplow.enrich.common.outputs.EnrichedEvent import com.snowplowanalytics.snowplow.enrich.common.utils.AtomicError @@ -36,32 +33,38 @@ object AtomicFieldsLengthValidator { event: EnrichedEvent, acceptInvalid: Boolean, invalidCount: F[Unit], - atomicFields: AtomicFields - ): F[Either[FailureDetails.SchemaViolation, Unit]] = - atomicFields.value - .map(field => validateField(event, field).toValidatedNel) - .combineAll match { - case Invalid(errors) if acceptInvalid => - handleAcceptableErrors(invalidCount, event, errors) *> Monad[F].pure(Right(())) - case Invalid(errors) => - Monad[F].pure(AtomicFields.errorsToSchemaViolation(errors).asLeft) - case Valid(()) => - Monad[F].pure(Right(())) + atomicFields: AtomicFields, + emitIncomplete: Boolean + ): IorT[F, Failure.SchemaViolation, Unit] = + IorT { + atomicFields.value + .map(validateField(event, _, emitIncomplete).toValidatedNel) + .combineAll match { + case Invalid(errors) if acceptInvalid => + handleAcceptableErrors(invalidCount, event, errors) *> Monad[F].pure(Ior.Right(())) + case Invalid(errors) => + Monad[F].pure(Ior.Both(AtomicFields.errorsToSchemaViolation(errors), ())) + case Valid(()) => + Monad[F].pure(Ior.Right(())) + } } private def validateField( event: EnrichedEvent, - atomicField: LimitedAtomicField + atomicField: LimitedAtomicField, + emitIncomplete: Boolean ): Either[AtomicError.FieldLengthError, Unit] = { val actualValue = atomicField.value.enrichedValueExtractor(event) - if (actualValue != null && actualValue.length > atomicField.limit) + if (actualValue != null && actualValue.length > atomicField.limit) { + if (emitIncomplete) atomicField.value.nullify(event) AtomicError .FieldLengthError( s"Field is longer than maximum allowed size ${atomicField.limit}", - atomicField.value.name + atomicField.value.name, + Option(actualValue) ) .asLeft - else + } else Right(()) } 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 16853d603..b0a541370 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 @@ -45,7 +45,7 @@ object ClientEnrichments { .leftMap(_ => "Could not be converted to java.lang.Integer s") case _ => s"Does not conform to regex ${ResRegex.toString}".asLeft }).leftMap { msg => - AtomicError.ParseError(msg, field) + AtomicError.ParseError(msg, field, Option(res)) } } 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 1487c9740..b1c006d7a 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,10 +15,10 @@ 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.effect.kernel.{Clock, Sync} +import cats.{Applicative, Functor, Monad} +import cats.data.{EitherT, Ior, IorT, NonEmptyList, OptionT, StateT} import cats.implicits._ +import cats.effect.kernel.{Clock, Sync} import com.snowplowanalytics.refererparser._ @@ -26,11 +26,11 @@ import com.snowplowanalytics.iglu.client.IgluCirceClient import com.snowplowanalytics.iglu.client.resolver.registries.RegistryLookup import com.snowplowanalytics.snowplow.enrich.common.utils.AtomicError -import com.snowplowanalytics.iglu.core.SelfDescribingData +import com.snowplowanalytics.iglu.core.{SchemaKey, SchemaVer, SelfDescribingData} import com.snowplowanalytics.iglu.core.circe.implicits._ import com.snowplowanalytics.snowplow.badrows._ -import com.snowplowanalytics.snowplow.badrows.{FailureDetails, Payload, Processor} +import com.snowplowanalytics.snowplow.badrows.{Failure => BadRowFailure} import com.snowplowanalytics.snowplow.enrich.common.{EtlPipeline, QueryStringParameters, RawEventParameters} import com.snowplowanalytics.snowplow.enrich.common.adapters.RawEvent @@ -43,8 +43,6 @@ import com.snowplowanalytics.snowplow.enrich.common.enrichments.registry.sqlquer import com.snowplowanalytics.snowplow.enrich.common.enrichments.web.{PageEnrichments => WPE} import com.snowplowanalytics.snowplow.enrich.common.outputs.EnrichedEvent import com.snowplowanalytics.snowplow.enrich.common.utils.{IgluUtils, ConversionUtils => CU} -import _root_.com.snowplowanalytics.iglu.core.SchemaKey -import com.snowplowanalytics.iglu.core.SchemaVer object EnrichmentManager { @@ -57,7 +55,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[_]: Sync]( registry: EnrichmentRegistry[F], @@ -68,44 +68,112 @@ object EnrichmentManager { featureFlags: EtlPipeline.FeatureFlags, invalidCount: F[Unit], registryLookup: RegistryLookup[F], - atomicFields: AtomicFields - ): EitherT[F, BadRow, EnrichedEvent] = - for { - enriched <- EitherT.rightT[F, BadRow](new EnrichedEvent) - extractResult <- mapAndValidateInput( - raw, - enriched, - etlTstamp, - processor, - client, - registryLookup - ) - _ = { - ME.formatUnstructEvent(extractResult.unstructEvent).foreach(e => enriched.unstruct_event = e) - ME.formatContexts(extractResult.contexts).foreach(c => enriched.contexts = c) - } - enrichmentsContexts <- runEnrichments( - registry, - processor, - raw, - enriched, - extractResult.contexts, - extractResult.unstructEvent, - featureFlags.legacyEnrichmentOrder - ) - _ = ME.formatContexts(enrichmentsContexts ::: extractResult.validationInfoContexts).foreach(c => enriched.derived_contexts = c) - _ <- validateEnriched( - enriched, - raw, - enrichmentsContexts, - client, - processor, - registryLookup, - featureFlags.acceptInvalid, - invalidCount, - atomicFields - ) - } yield enriched + atomicFields: AtomicFields, + emitIncomplete: Boolean + ): IorT[F, BadRow, EnrichedEvent] = { + def enrich(enriched: EnrichedEvent): IorT[F, NonEmptyList[NonEmptyList[Failure]], List[SelfDescribingData[Json]]] = + for { + extractResult <- mapAndValidateInput( + raw, + enriched, + etlTstamp, + processor, + client, + registryLookup + ) + .leftMap(NonEmptyList.one) + .possiblyExitingEarly(emitIncomplete) + _ = { + enriched.contexts = ME.formatContexts(extractResult.contexts).orNull + enriched.unstruct_event = ME.formatUnstructEvent(extractResult.unstructEvent).orNull + } + enrichmentsContexts <- runEnrichments( + registry, + raw, + enriched, + extractResult.contexts, + extractResult.unstructEvent, + featureFlags.legacyEnrichmentOrder + ) + .leftMap(NonEmptyList.one) + .possiblyExitingEarly(emitIncomplete) + validContexts <- validateEnriched( + enriched, + enrichmentsContexts, + client, + registryLookup, + featureFlags.acceptInvalid, + invalidCount, + atomicFields, + emitIncomplete + ) + .leftMap(NonEmptyList.one) + .possiblyExitingEarly(emitIncomplete) + derivedContexts = validContexts ::: extractResult.validationInfoContexts + } yield derivedContexts + + // derived contexts are set lastly because we want to include failure entities + // to derived contexts as well and we can get failure entities only in the end + // of the enrichment process + IorT( + for { + enrichedEvent <- Sync[F].delay(new EnrichedEvent) + enrichmentResult <- enrich(enrichedEvent).value + now = Instant.now() + _ = setDerivedContexts(enrichedEvent, enrichmentResult, now, processor) + result = enrichmentResult + .leftMap { fe => + createBadRow( + fe, + EnrichedEvent.toPartiallyEnrichedEvent(enrichedEvent), + RawEvent.toRawEvent(raw), + now, + processor + ) + } + .map(_ => enrichedEvent) + } yield result + ) + } + + private def createBadRow( + fe: NonEmptyList[NonEmptyList[Failure]], + pe: Payload.PartiallyEnrichedEvent, + re: Payload.RawEvent, + timestamp: Instant, + processor: Processor + ): BadRow = { + val firstList = fe.head + firstList.head match { + case h: Failure.SchemaViolation => + val sv = firstList.tail.collect { case f: Failure.SchemaViolation => f } + BadRow.SchemaViolations( + processor, + BadRowFailure.SchemaViolations(timestamp, NonEmptyList(h, sv).map(_.schemaViolation)), + Payload.EnrichmentPayload(pe, re) + ) + case h: Failure.EnrichmentFailure => + val ef = firstList.tail.collect { case f: Failure.EnrichmentFailure => f } + BadRow.EnrichmentFailures( + processor, + BadRowFailure.EnrichmentFailures(timestamp, NonEmptyList(h, ef).map(_.enrichmentFailure)), + Payload.EnrichmentPayload(pe, re) + ) + } + } + + def setDerivedContexts( + enriched: EnrichedEvent, + enrichmentResult: Ior[NonEmptyList[NonEmptyList[Failure]], List[SelfDescribingData[Json]]], + timestamp: Instant, + processor: Processor + ): Unit = { + val derivedContexts = enrichmentResult.leftMap { ll => + ll.flatten.toList + .map(_.toSDJ(timestamp, processor)) + }.merge + ME.formatContexts(derivedContexts).foreach(c => enriched.derived_contexts = c) + } private def mapAndValidateInput[F[_]: Sync]( raw: RawEvent, @@ -114,80 +182,62 @@ object EnrichmentManager { processor: Processor, client: IgluCirceClient[F], registryLookup: RegistryLookup[F] - ): EitherT[F, BadRow, IgluUtils.EventExtractResult] = - EitherT { - for { - setup <- setupEnrichedEvent[F](raw, enrichedEvent, etlTstamp, processor).map(_.toValidatedNel) - iglu <- IgluUtils.extractAndValidateInputJsons(enrichedEvent, client, registryLookup) - } yield (iglu <* setup).leftMap { violations => - buildSchemaViolationsBadRow( - violations, - EnrichedEvent.toPartiallyEnrichedEvent(enrichedEvent), - RawEvent.toRawEvent(raw), - processor - ) - }.toEither - } + ): IorT[F, NonEmptyList[Failure], IgluUtils.EventExtractResult] = + for { + _ <- setupEnrichedEvent[F](raw, enrichedEvent, etlTstamp, processor) + .leftMap(NonEmptyList.one) + extract <- IgluUtils + .extractAndValidateInputJsons(enrichedEvent, client, registryLookup) + .leftMap { l: NonEmptyList[Failure] => l } + } yield extract /** - * 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], - processor: Processor, raw: RawEvent, enriched: EnrichedEvent, inputContexts: List[SelfDescribingData[Json]], unstructEvent: Option[SelfDescribingData[Json]], legacyOrder: Boolean - ): EitherT[F, BadRow, List[SelfDescribingData[Json]]] = - EitherT { + ): IorT[F, NonEmptyList[Failure], 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( + nel.map(Failure.EnrichmentFailure), + contexts + ) case None => - contexts.asRight + Ior.right(contexts) } } } private def validateEnriched[F[_]: Clock: Monad]( enriched: EnrichedEvent, - raw: RawEvent, enrichmentsContexts: List[SelfDescribingData[Json]], client: IgluCirceClient[F], - processor: Processor, registryLookup: RegistryLookup[F], acceptInvalid: Boolean, invalidCount: F[Unit], - atomicFields: AtomicFields - ): EitherT[F, BadRow, Unit] = - EitherT { - for { - atomic <- AtomicFieldsLengthValidator.validate[F](enriched, acceptInvalid, invalidCount, atomicFields).map(_.toValidatedNel) - contexts <- IgluUtils.validateEnrichmentsContexts[F](client, enrichmentsContexts, registryLookup) - } yield (atomic |+| contexts).void.leftMap { violations => - buildSchemaViolationsBadRow( - violations, - EnrichedEvent.toPartiallyEnrichedEvent(enriched), - RawEvent.toRawEvent(raw), - processor - ) - }.toEither - } + atomicFields: AtomicFields, + emitIncomplete: Boolean + ): IorT[F, NonEmptyList[Failure], List[SelfDescribingData[Json]]] = + for { + validContexts <- IgluUtils.validateEnrichmentsContexts[F](client, enrichmentsContexts, registryLookup) + _ <- AtomicFieldsLengthValidator + .validate[F](enriched, acceptInvalid, invalidCount, atomicFields, emitIncomplete) + .leftMap { v: Failure => NonEmptyList.one(v) } + } yield validContexts private[enrichments] case class Accumulation( event: EnrichedEvent, @@ -315,27 +365,29 @@ object EnrichmentManager { e: EnrichedEvent, etlTstamp: DateTime, processor: Processor - ): F[Either[FailureDetails.SchemaViolation, Unit]] = - Sync[F].delay { - 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) - e.etl_tstamp = EE.toTimestamp(etlTstamp) - e.network_userid = raw.context.userId.map(_.toString).orNull // May be updated later by 'nuid' - e.user_ipaddress = ME - .extractIp("user_ipaddress", raw.context.ipAddress.orNull) - .toOption - .orNull // May be updated later by 'ip' - // May be updated later if we have a `ua` parameter - setUseragent(e, raw.context.useragent) - // Validate that the collectorTstamp exists and is Redshift-compatible - val collectorTstamp = setCollectorTstamp(e, raw.context.timestamp).toValidatedNel - // Map/validate/transform input fields to enriched event fields - val transformed = Transform.transform(raw, e) - - (collectorTstamp |+| transformed) - .leftMap(AtomicFields.errorsToSchemaViolation) - .toEither + ): IorT[F, Failure.SchemaViolation, Unit] = + IorT { + Sync[F].delay { + 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) + e.etl_tstamp = EE.toTimestamp(etlTstamp) + e.network_userid = raw.context.userId.map(_.toString).orNull // May be updated later by 'nuid' + e.user_ipaddress = ME + .extractIp("user_ipaddress", raw.context.ipAddress.orNull) + .toOption + .orNull // May be updated later by 'ip' + // May be updated later if we have a `ua` parameter + setUseragent(e, raw.context.useragent) + // Validate that the collectorTstamp exists and is Redshift-compatible + val collectorTstamp = setCollectorTstamp(e, raw.context.timestamp).toValidatedNel + // Map/validate/transform input fields to enriched event fields + val transformed = Transform.transform(raw, e) + + (collectorTstamp |+| transformed).void.toIor + .leftMap(AtomicFields.errorsToSchemaViolation) + .putRight(()) + } } def setCollectorTstamp(event: EnrichedEvent, timestamp: Option[DateTime]): Either[AtomicError.ParseError, Unit] = @@ -824,28 +876,17 @@ object EnrichmentManager { } } - private 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) - ) - - private def buildEnrichmentFailuresBadRow( - fs: NonEmptyList[FailureDetails.EnrichmentFailure], - pee: Payload.PartiallyEnrichedEvent, - re: Payload.RawEvent, - processor: Processor - ) = - BadRow.EnrichmentFailures( - processor, - Failure.EnrichmentFailures(Instant.now(), fs), - Payload.EnrichmentPayload(pee, re) - ) + 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 99d854796..deb5f812a 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 @@ -51,12 +51,12 @@ object EventEnrichments { */ def formatCollectorTstamp(collectorTstamp: Option[DateTime]): Either[AtomicError.ParseError, String] = collectorTstamp match { - case None => AtomicError.ParseError("Field not set", "collector_tstamp").asLeft + case None => AtomicError.ParseError("Field not set", "collector_tstamp", None).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" - AtomicError.ParseError(msg, "collector_tstamp").asLeft + AtomicError.ParseError(msg, "collector_tstamp", Option(t).map(_.toString)).asLeft } else formattedTimestamp.asRight } @@ -122,7 +122,8 @@ object EventEnrichments { AtomicError .ParseError( s"Formatting as $timestampString is not Redshift-compatible", - field + field, + Option(tstamp) ) .asLeft else @@ -132,7 +133,8 @@ object EventEnrichments { AtomicError .ParseError( "Not in the expected format: ms since epoch", - field + field, + Option(tstamp) ) .asLeft } @@ -157,7 +159,7 @@ object EventEnrichments { case "pp" => "page_ping".asRight case _ => val msg = "Not a valid event type" - AtomicError.ParseError(msg, field).asLeft + AtomicError.ParseError(msg, field, Option(code)).asLeft } /** diff --git a/modules/common/src/main/scala/com.snowplowanalytics.snowplow.enrich/common/enrichments/Failure.scala b/modules/common/src/main/scala/com.snowplowanalytics.snowplow.enrich/common/enrichments/Failure.scala new file mode 100644 index 000000000..3991528cb --- /dev/null +++ b/modules/common/src/main/scala/com.snowplowanalytics.snowplow.enrich/common/enrichments/Failure.scala @@ -0,0 +1,191 @@ +/* + * Copyright (c) 2024-present Snowplow Analytics Ltd. + * All rights reserved. + * + * This software is made available by Snowplow Analytics, Ltd., + * under the terms of the Snowplow Limited Use License Agreement, Version 1.0 + * located at https://docs.snowplow.io/limited-use-license-1.0 + * BY INSTALLING, DOWNLOADING, ACCESSING, USING OR DISTRIBUTING ANY PORTION + * OF THE SOFTWARE, YOU AGREE TO THE TERMS OF SUCH LICENSE AGREEMENT. + */ +package com.snowplowanalytics.snowplow.enrich.common.enrichments + +import java.time.Instant + +import cats.syntax.option._ + +import io.circe.{Encoder, Json} +import io.circe.generic.semiauto._ +import io.circe.syntax._ + +import com.snowplowanalytics.snowplow.badrows._ + +import com.snowplowanalytics.iglu.client.ClientError +import com.snowplowanalytics.iglu.client.validator.ValidatorError + +import com.snowplowanalytics.iglu.core.{SchemaKey, SchemaVer, SelfDescribingData} +import com.snowplowanalytics.iglu.core.circe.implicits.schemaKeyCirceJsonEncoder + +/** + * Represents a failure encountered during enrichment of the event. + * Failure entities will be attached to incomplete events as derived contexts. + */ +sealed trait Failure { + def toSDJ(timestamp: Instant, processor: Processor): SelfDescribingData[Json] +} + +object Failure { + + val failureSchemaKey = SchemaKey("com.snowplowanalytics.snowplow", "failure", "jsonschema", SchemaVer.Full(1, 0, 0)) + + case class SchemaViolation( + schemaViolation: FailureDetails.SchemaViolation, + source: String, + data: Json + ) extends Failure { + def toSDJ(timestamp: Instant, processor: Processor): SelfDescribingData[Json] = { + val feJson = fromSchemaViolation(this, timestamp, processor) + SelfDescribingData(failureSchemaKey, feJson.asJson) + } + + } + + case class EnrichmentFailure( + enrichmentFailure: FailureDetails.EnrichmentFailure + ) extends Failure { + def toSDJ(timestamp: Instant, processor: Processor): SelfDescribingData[Json] = { + val feJson = fromEnrichmentFailure(this, timestamp, processor) + SelfDescribingData(failureSchemaKey, feJson.asJson) + } + } + + case class FailureContext( + failureType: String, + errors: List[Json], + schema: Option[SchemaKey], + data: Option[Json], + timestamp: Instant, + componentName: String, + componentVersion: String + ) + + object FailureContext { + implicit val failureContextEncoder: Encoder[FailureContext] = deriveEncoder[FailureContext] + } + + def fromEnrichmentFailure( + ef: EnrichmentFailure, + timestamp: Instant, + processor: Processor + ): FailureContext = { + val failureType = s"EnrichmentError: ${ef.enrichmentFailure.enrichment.map(_.identifier).getOrElse("")}" + val schemaKey = ef.enrichmentFailure.enrichment.map(_.schemaKey) + val (errors, data) = ef.enrichmentFailure.message match { + case FailureDetails.EnrichmentFailureMessage.InputData(field, value, expectation) => + ( + List( + Json.obj( + "message" := s"$field - $expectation", + "source" := field + ) + ), + Json.obj(field := value).some + ) + case FailureDetails.EnrichmentFailureMessage.Simple(error) => + ( + List( + Json.obj( + "message" := error + ) + ), + None + ) + case FailureDetails.EnrichmentFailureMessage.IgluError(_, error) => + // EnrichmentFailureMessage.IgluError isn't used anywhere in the project. + // We are return this value for completeness. + ( + List( + Json.obj( + "message" := error + ) + ), + None + ) + } + FailureContext( + failureType = failureType, + errors = errors, + schema = schemaKey, + data = data, + timestamp = timestamp, + componentName = processor.artifact, + componentVersion = processor.version + ) + } + + def fromSchemaViolation( + v: SchemaViolation, + timestamp: Instant, + processor: Processor + ): FailureContext = { + val (failureType, errors, schema, data) = v.schemaViolation match { + case FailureDetails.SchemaViolation.NotJson(_, _, err) => + val error = Json.obj("message" := err, "source" := v.source) + ("NotJSON", List(error), None, Json.obj(v.source := v.data).some) + case FailureDetails.SchemaViolation.NotIglu(_, err) => + val message = err.message("").split(":").headOption + val error = Json.obj("message" := message, "source" := v.source) + ("NotIglu", List(error), None, v.data.some) + case FailureDetails.SchemaViolation.CriterionMismatch(schemaKey, schemaCriterion) => + val message = s"Unexpected schema: ${schemaKey.toSchemaUri} does not match the criterion" + val error = Json.obj( + "message" := message, + "source" := v.source, + "criterion" := schemaCriterion.asString + ) + ("CriterionMismatch", List(error), schemaKey.some, v.data.some) + case FailureDetails.SchemaViolation.IgluError(schemaKey, ClientError.ResolutionError(lh)) => + val message = s"Resolution error: schema ${schemaKey.toSchemaUri} not found" + val lookupHistory = lh.toList + .map { + case (repo, lookups) => + lookups.asJson.deepMerge(Json.obj("repository" := repo.asJson)) + } + val error = Json.obj( + "message" := message, + "source" := v.source, + "lookupHistory" := lookupHistory + ) + ("ResolutionError", List(error), schemaKey.some, v.data.some) + case FailureDetails.SchemaViolation.IgluError(schemaKey, ClientError.ValidationError(ValidatorError.InvalidData(e), _)) => + val errors = e.toList.map { r => + Json.obj( + "message" := r.message, + "source" := v.source, + "path" := r.path, + "keyword" := r.keyword, + "targets" := r.targets + ) + } + ("ValidationError", errors, schemaKey.some, v.data.some) + case FailureDetails.SchemaViolation.IgluError(schemaKey, ClientError.ValidationError(ValidatorError.InvalidSchema(e), _)) => + val errors = e.toList.map { r => + Json.obj( + "message" := s"Invalid schema: ${schemaKey.toSchemaUri} - ${r.message}", + "source" := v.source, + "path" := r.path + ) + } + ("ValidationError", errors, schemaKey.some, v.data.some) + } + FailureContext( + failureType = failureType, + errors = errors, + schema = schema, + data = data, + timestamp = timestamp, + componentName = processor.artifact, + componentVersion = processor.version + ) + } +} 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 351964385..8754eb477 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 @@ -58,7 +58,7 @@ object MiscEnrichments { case "headset" => "headset".asRight // AR/VR Headset case _ => val msg = "Not a valid platform" - AtomicError.ParseError(msg, field).asLeft + AtomicError.ParseError(msg, field, Option(platform)).asLeft } /** Make a String TSV safe */ diff --git a/modules/common/src/main/scala/com.snowplowanalytics.snowplow.enrich/common/utils/AtomicError.scala b/modules/common/src/main/scala/com.snowplowanalytics.snowplow.enrich/common/utils/AtomicError.scala index 4836ff483..ab360dcc7 100644 --- a/modules/common/src/main/scala/com.snowplowanalytics.snowplow.enrich/common/utils/AtomicError.scala +++ b/modules/common/src/main/scala/com.snowplowanalytics.snowplow.enrich/common/utils/AtomicError.scala @@ -15,24 +15,32 @@ import com.snowplowanalytics.iglu.client.validator.ValidatorReport sealed trait AtomicError { def message: String def field: String - def repr: String + def value: Option[String] + def keyword: String + // IMPORTANT: `value` should never be put in ValidatorReport def toValidatorReport: ValidatorReport = - ValidatorReport(message, Some(field), Nil, Some(repr)) + ValidatorReport(message, Some(field), Nil, Some(keyword)) } object AtomicError { + val source = "atomic_field" + val keywordParse = s"${source}_parse_error" + val keywordLength = s"${source}_length_exceeded" + case class ParseError( message: String, - field: String + field: String, + value: Option[String] ) extends AtomicError { - override def repr: String = "atomic_field_parse_error" + override def keyword: String = keywordParse } case class FieldLengthError( message: String, - field: String + field: String, + value: Option[String] ) extends AtomicError { - override def repr: String = "atomic_field_length_exceeded" + override def keyword: String = keywordLength } } 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 03b14e5e4..c791a9123 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 @@ -164,7 +164,7 @@ object ConversionUtils { val uuid = Try(UUID.fromString(str)).toOption.filter(check(str)) uuid match { case Some(_) => str.toLowerCase.asRight - case None => AtomicError.ParseError("Not a valid UUID", field).asLeft + case None => AtomicError.ParseError("Not a valid UUID", field, Option(str)).asLeft } } @@ -178,7 +178,7 @@ object ConversionUtils { Either .catchNonFatal { str.toInt; str } .leftMap { _ => - AtomicError.ParseError("Not a valid integer", field) + AtomicError.ParseError("Not a valid integer", field, Option(str)) } } @@ -326,7 +326,7 @@ object ConversionUtils { val stringToJInteger2: (String, String) => Either[AtomicError.ParseError, JInteger] = (field, str) => stringToJInteger(str).leftMap { e => - AtomicError.ParseError(e, field) + AtomicError.ParseError(e, field, Option(str)) } val stringToJBigDecimal: String => Either[String, JBigDecimal] = str => @@ -351,7 +351,7 @@ object ConversionUtils { val stringToJBigDecimal2: (String, String) => Either[AtomicError.ParseError, JBigDecimal] = (field, str) => stringToJBigDecimal(str).leftMap { e => - AtomicError.ParseError(e, field) + AtomicError.ParseError(e, field, Option(str)) } /** @@ -377,7 +377,7 @@ object ConversionUtils { } .leftMap { _ => val msg = "Cannot be converted to Double-like" - AtomicError.ParseError(msg, field) + AtomicError.ParseError(msg, field, Option(str)) } /** @@ -397,7 +397,7 @@ object ConversionUtils { jbigdec.doubleValue().some } } - .leftMap(_ => AtomicError.ParseError("Cannot be converted to Double", field)) + .leftMap(_ => AtomicError.ParseError("Cannot be converted to Double", field, Option(str))) /** Convert a java BigDecimal to a Double */ def jBigDecimalToDouble(field: String, f: JBigDecimal): Either[AtomicError.ParseError, Option[Double]] = @@ -405,7 +405,7 @@ object ConversionUtils { .catchNonFatal { Option(f).map(_.doubleValue) } - .leftMap(_ => AtomicError.ParseError("Cannot be converted to Double", field)) + .leftMap(_ => AtomicError.ParseError("Cannot be converted to Double", field, Option(f).map(_.toString))) /** Convert a java BigDecimal to a Double */ def jBigDecimalToDouble( @@ -431,7 +431,7 @@ object ConversionUtils { .catchNonFatal { d.map(dd => new JBigDecimal(dd)) } - .leftMap(_ => AtomicError.ParseError("Cannot be converted to java BigDecimal", field)) + .leftMap(_ => AtomicError.ParseError("Cannot be converted to java BigDecimal", field, d.map(_.toString))) /** * Converts a String to a Double with two decimal places. Used to honor schemas with @@ -466,7 +466,7 @@ object ConversionUtils { case "0" => (0.toByte: JByte).asRight case _ => val msg = "Cannot be converted to Boolean-like java.lang.Byte" - AtomicError.ParseError(msg, field).asLeft + AtomicError.ParseError(msg, field, Option(str)).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 791174120..3d058a136 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._ @@ -25,6 +25,8 @@ import com.snowplowanalytics.iglu.client.resolver.registries.RegistryLookup import com.snowplowanalytics.iglu.core.{SchemaCriterion, SchemaKey, SchemaVer, SelfDescribingData} import com.snowplowanalytics.iglu.core.circe.implicits._ +import com.snowplowanalytics.snowplow.enrich.common.enrichments.Failure + import com.snowplowanalytics.snowplow.badrows._ import com.snowplowanalytics.snowplow.enrich.common.outputs.EnrichedEvent @@ -44,27 +46,25 @@ 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], registryLookup: RegistryLookup[F] - ): F[ValidatedNel[FailureDetails.SchemaViolation, EventExtractResult]] = + ): IorT[F, NonEmptyList[Failure.SchemaViolation], EventExtractResult] = 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) - } + contexts <- extractAndValidateInputContexts(enriched, client, registryLookup) + unstruct <- extractAndValidateUnstructEvent(enriched, client, registryLookup) + } yield { + val validationInfoContexts = (contexts.flatMap(_.validationInfo) ::: unstruct.flatMap(_.validationInfo).toList).distinct + .map(_.toSdj) + EventExtractResult(contexts = contexts.map(_.sdj), + unstructEvent = unstruct.map(_.sdj), + validationInfoContexts = validationInfoContexts + ) + } /** * Extract unstructured event from event and validate against its schema @@ -79,20 +79,23 @@ object IgluUtils { enriched: EnrichedEvent, client: IgluCirceClient[F], registryLookup: RegistryLookup[F], - field: String = "ue_properties", + field: String = "unstruct", 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[Failure.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(unstruct, client, registryLookup) + unstructSDJ <- parseAndValidateSDJ(unstruct, client, registryLookup, field) } yield unstructSDJ.some + iorT.recoverWith { case errors => IorT.fromIor[F](Ior.Both(errors, None)) } case None => - EitherT.rightT[F, FailureDetails.SchemaViolation](none[SdjExtractResult]) - }).toValidated + IorT.rightT[F, NonEmptyList[Failure.SchemaViolation]](none[SdjExtractResult]) + } /** * Extract list of custom contexts from event and validate each against its schema @@ -101,7 +104,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, @@ -109,27 +112,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[Failure.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(_, client, registryLookup).toValidatedNel) - .sequence - .map(_.sequence.toEither) - ) - } yield contextsSDJ + contextsSdj <- contexts + .traverse( + parseAndValidateSDJ(_, client, registryLookup, field) + .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[Failure.SchemaViolation]](Nil) + } /** * Validate each context added by the enrichments against its schema @@ -138,22 +142,24 @@ 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]], registryLookup: RegistryLookup[F] - ): F[ValidatedNel[FailureDetails.SchemaViolation, Unit]] = + ): IorT[F, NonEmptyList[Failure.SchemaViolation], List[SelfDescribingData[Json]]] = checkList(client, sdjs, registryLookup) .leftMap( _.map { - case (schemaKey, clientError) => - val f: FailureDetails.SchemaViolation = FailureDetails.SchemaViolation.IgluError(schemaKey, clientError) - f + case (sdj, clientError) => + Failure.SchemaViolation( + schemaViolation = FailureDetails.SchemaViolation.IgluError(sdj.schema, clientError), + source = "derived_contexts", + data = sdj.data + ) } ) - .toValidated /** Used to extract .data for input custom contexts and input unstructured event */ private def extractInputData[F[_]: Monad: Clock]( @@ -162,34 +168,54 @@ object IgluUtils { expectedCriterion: SchemaCriterion, client: IgluCirceClient[F], registryLookup: RegistryLookup[F] - ): EitherT[F, FailureDetails.SchemaViolation, Json] = + ): EitherT[F, Failure.SchemaViolation, Json] = for { // Parse Json string with the SDJ json <- JsonUtils .extractJson(rawJson) - .leftMap(e => FailureDetails.SchemaViolation.NotJson(field, rawJson.some, e)) + .leftMap(e => + Failure.SchemaViolation( + schemaViolation = FailureDetails.SchemaViolation.NotJson(field, rawJson.some, e), + source = field, + data = rawJson.asJson + ) + ) .toEitherT[F] // Parse Json as SelfDescribingData[Json] (which contains the .data that we want) sdj <- SelfDescribingData .parse(json) - .leftMap(FailureDetails.SchemaViolation.NotIglu(json, _)) + .leftMap(e => + Failure.SchemaViolation( + schemaViolation = FailureDetails.SchemaViolation.NotIglu(json, e), + source = field, + data = json + ) + ) .toEitherT[F] // Check that the schema of SelfDescribingData[Json] is the expected one _ <- if (validateCriterion(sdj, expectedCriterion)) - EitherT.rightT[F, FailureDetails.SchemaViolation](sdj) + EitherT.rightT[F, Failure.SchemaViolation](sdj) else EitherT .leftT[F, SelfDescribingData[Json]]( - FailureDetails.SchemaViolation.CriterionMismatch(sdj.schema, expectedCriterion) + Failure.SchemaViolation( + schemaViolation = FailureDetails.SchemaViolation.CriterionMismatch(sdj.schema, expectedCriterion), + source = field, + data = sdj.data + ) ) // Check that the SDJ holding the .data is valid _ <- check(client, sdj, registryLookup) .leftMap { case (schemaKey, clientError) => - FailureDetails.SchemaViolation.IgluError(schemaKey, clientError) + Failure.SchemaViolation( + schemaViolation = FailureDetails.SchemaViolation.IgluError(schemaKey, clientError), + source = field, + data = sdj.data + ) } // Extract .data of SelfDescribingData[Json] - data <- EitherT.rightT[F, FailureDetails.SchemaViolation](sdj.data) + data <- EitherT.rightT[F, Failure.SchemaViolation](sdj.data) } yield data /** Check that the schema of a SDJ matches the expected one */ @@ -208,37 +234,53 @@ 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[(SelfDescribingData[Json], ClientError)], List[SelfDescribingData[Json]]] = + sdjs.map { sdj => + check(client, sdj, registryLookup) + .map(_ => List(sdj)) + .leftMap(e => NonEmptyList.one((sdj, e._2))) + .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[F[_]: Monad: Clock]( json: Json, client: IgluCirceClient[F], - registryLookup: RegistryLookup[F] - ): EitherT[F, FailureDetails.SchemaViolation, SdjExtractResult] = + registryLookup: RegistryLookup[F], + field: String + ): IorT[F, NonEmptyList[Failure.SchemaViolation], SdjExtractResult] = for { - sdj <- SelfDescribingData - .parse(json) - .leftMap(FailureDetails.SchemaViolation.NotIglu(json, _)) - .toEitherT[F] + sdj <- IorT + .fromEither[F](SelfDescribingData.parse(json)) + .leftMap[Failure.SchemaViolation](e => + Failure.SchemaViolation( + schemaViolation = FailureDetails.SchemaViolation.NotIglu(json, e), + source = field, + data = json.asJson + ) + ) + .leftMap(NonEmptyList.one) supersedingSchema <- check(client, sdj, registryLookup) .leftMap { case (schemaKey, clientError) => - FailureDetails.SchemaViolation - .IgluError(schemaKey, clientError): FailureDetails.SchemaViolation - + Failure.SchemaViolation( + schemaViolation = FailureDetails.SchemaViolation + .IgluError(schemaKey, clientError): FailureDetails.SchemaViolation, + source = field, + data = sdj.data + ) } + .leftMap(NonEmptyList.one) + .toIor validationInfo = supersedingSchema.map(s => ValidationInfo(sdj.schema, s)) sdjUpdated = replaceSchemaVersion(sdj, validationInfo) } yield SdjExtractResult(sdjUpdated, validationInfo) 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 b9ebd5c6b..545ad1ae3 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,20 +31,15 @@ object JsonUtils { /** Validates a String as correct JSON. */ val extractUnencJson: (String, String) => Either[AtomicError.ParseError, String] = - (field, str) => - validateAndReformatJson(str) - .leftMap { e => - AtomicError.ParseError(e, field) - } + (_, str) => Right(str) /** Decodes a Base64 (URL safe)-encoded String then validates it as correct JSON. */ val extractBase64EncJson: (String, String) => Either[AtomicError.ParseError, String] = (field, str) => ConversionUtils .decodeBase64Url(str) - .flatMap(validateAndReformatJson) .leftMap { e => - AtomicError.ParseError(e, field) + AtomicError.ParseError(e, field, Option(str)) } /** @@ -121,16 +116,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/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 16f08597c..5a3821f4b 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 @@ -37,7 +37,7 @@ import com.snowplowanalytics.iglu.client.{IgluCirceClient, Resolver} import com.snowplowanalytics.iglu.client.resolver.registries.Registry import com.snowplowanalytics.iglu.client.resolver.registries.JavaNetRegistryLookup -import com.snowplowanalytics.iglu.core.SelfDescribingData +import com.snowplowanalytics.iglu.core.{SchemaKey, SelfDescribingData} import com.snowplowanalytics.iglu.core.circe.implicits._ import com.snowplowanalytics.lrumap.CreateLruMap._ @@ -129,6 +129,27 @@ object SpecHelpers extends CatsEffect { .flatMap(SelfDescribingData.parse[Json]) .leftMap(err => s"Can't parse Json [$rawJson] as as SelfDescribingData, error: [$err]") + def listContexts(rawContexts: String): List[SelfDescribingData[Json]] = + jsonStringToSDJ(rawContexts) + .map(_.data.asArray.get.toList) + .flatMap(contexts => contexts.traverse(c => SelfDescribingData.parse[Json](c))) match { + case Left(err) => + throw new IllegalArgumentException(s"Couldn't list contexts schemas. Error: [$err]") + case Right(sdjs) => sdjs + } + + def listContextsSchemas(rawContexts: String): List[SchemaKey] = listContexts(rawContexts).map(_.schema) + + def getUnstructSchema(rawUnstruct: String): SchemaKey = + jsonStringToSDJ(rawUnstruct) + .map(_.data) + .flatMap(SelfDescribingData.parse[Json]) + .map(_.schema) match { + case Left(err) => + throw new IllegalArgumentException(s"Couldn't get unstruct event schema. Error: [$err]") + case Right(schema) => schema + } + implicit class MapOps[A, B](underlying: Map[A, B]) { def toOpt: Map[A, Option[B]] = underlying.map { case (a, b) => (a, Option(b)) } } @@ -148,6 +169,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/AtomicFieldsSpec.scala b/modules/common/src/test/scala/com.snowplowanalytics.snowplow.enrich.common/enrichments/AtomicFieldsSpec.scala new file mode 100644 index 000000000..1bfb44d14 --- /dev/null +++ b/modules/common/src/test/scala/com.snowplowanalytics.snowplow.enrich.common/enrichments/AtomicFieldsSpec.scala @@ -0,0 +1,62 @@ +/* + * Copyright (c) 2022-present Snowplow Analytics Ltd. + * All rights reserved. + * + * This software is made available by Snowplow Analytics, Ltd., + * under the terms of the Snowplow Limited Use License Agreement, Version 1.0 + * located at https://docs.snowplow.io/limited-use-license-1.0 + * BY INSTALLING, DOWNLOADING, ACCESSING, USING OR DISTRIBUTING ANY PORTION + * OF THE SOFTWARE, YOU AGREE TO THE TERMS OF SUCH LICENSE AGREEMENT. + */ +package com.snowplowanalytics.snowplow.enrich.common.enrichments + +import cats.data.NonEmptyList +import cats.syntax.option._ + +import io.circe.Json +import io.circe.syntax._ + +import com.snowplowanalytics.iglu.client.ClientError.ValidationError +import com.snowplowanalytics.iglu.client.validator.{ValidatorError, ValidatorReport} + +import com.snowplowanalytics.snowplow.badrows.FailureDetails + +import com.snowplowanalytics.snowplow.enrich.common.utils.AtomicError + +import org.specs2.mutable.Specification + +class AtomicFieldsSpec extends Specification { + + "errorsToSchemaViolation" should { + "convert AtomicErrors to SchemaViolation correctly" >> { + val error1 = AtomicError.ParseError("message1", "field1", Some("value1")) + val error2 = AtomicError.ParseError("message2", "field2", Some("")) + val error3 = AtomicError.FieldLengthError("message3", "field3", Some("value3")) + val error4 = AtomicError.ParseError("message4", "field4", None) + + val vrList = NonEmptyList( + ValidatorReport(error1.message, error1.field.some, Nil, AtomicError.keywordParse.some), + List( + ValidatorReport(error2.message, error2.field.some, Nil, AtomicError.keywordParse.some), + ValidatorReport(error3.message, error3.field.some, Nil, AtomicError.keywordLength.some), + ValidatorReport(error4.message, error4.field.some, Nil, AtomicError.keywordParse.some) + ) + ) + val expected = Failure.SchemaViolation( + schemaViolation = FailureDetails.SchemaViolation.IgluError( + schemaKey = AtomicFields.atomicSchema, + error = ValidationError(ValidatorError.InvalidData(vrList), None) + ), + source = AtomicError.source, + data = Json.obj( + error1.field := error1.value, + error2.field := error2.value, + error3.field := error3.value, + error4.field := Json.Null + ) + ) + val result = AtomicFields.errorsToSchemaViolation(NonEmptyList(error1, List(error2, error3, error4))) + result must beEqualTo(expected) + } + } +} diff --git a/modules/common/src/test/scala/com.snowplowanalytics.snowplow.enrich.common/enrichments/EnrichmentManagerSpec.scala b/modules/common/src/test/scala/com.snowplowanalytics.snowplow.enrich.common/enrichments/EnrichmentManagerSpec.scala index 7ea29cacb..fa1f350db 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 @@ -10,6 +10,8 @@ */ package com.snowplowanalytics.snowplow.enrich.common.enrichments +import java.time.Instant + import org.apache.commons.codec.digest.DigestUtils import org.specs2.mutable.Specification @@ -19,7 +21,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._ @@ -29,8 +31,10 @@ import io.circe.syntax._ import org.joda.time.DateTime import com.snowplowanalytics.snowplow.badrows._ +import com.snowplowanalytics.snowplow.badrows.{Failure => BadRowFailure} import com.snowplowanalytics.iglu.core.{SchemaCriterion, SchemaKey, SchemaVer, SelfDescribingData} +import com.snowplowanalytics.iglu.core.circe.implicits._ import com.snowplowanalytics.snowplow.enrich.common.QueryStringParameters import com.snowplowanalytics.snowplow.enrich.common.loaders._ @@ -42,7 +46,7 @@ import com.snowplowanalytics.snowplow.enrich.common.enrichments.registry.pii.{ PiiStrategyPseudonymize } import com.snowplowanalytics.snowplow.enrich.common.outputs.EnrichedEvent -import com.snowplowanalytics.snowplow.enrich.common.utils.ConversionUtils +import com.snowplowanalytics.snowplow.enrich.common.utils.{AtomicError, ConversionUtils} import com.snowplowanalytics.snowplow.enrich.common.enrichments.registry.{ CrossNavigationEnrichment, HttpHeaderExtractorEnrichment, @@ -89,13 +93,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" >> { @@ -127,13 +131,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 that contains 1 ValidationError for the atomic field and 1 ValidationError for the unstruct event" >> { @@ -167,29 +171,32 @@ class EnrichmentManagerSpec extends Specification with EitherMatchers with CatsE AcceptInvalid.featureFlags, IO.unit, SpecHelpers.registryLookup, - atomicFieldLimits + atomicFieldLimits, + emitIncomplete ) .value - .map(_ must beLeft.like { - case BadRow.SchemaViolations( - _, - Failure.SchemaViolations(_, - NonEmptyList(FailureDetails.SchemaViolation.IgluError(schemaKey1, clientError1), - List(FailureDetails.SchemaViolation.IgluError(schemaKey2, clientError2)) - ) - ), - _ + .map { + case Ior.Left( + BadRow.SchemaViolations( + _, + BadRowFailure.SchemaViolations(_, + NonEmptyList(FailureDetails.SchemaViolation.IgluError(schemaKey1, clientError1), + List(FailureDetails.SchemaViolation.IgluError(schemaKey2, clientError2)) + ) + ), + _ + ) ) => - schemaKey1 must beEqualTo(emailSentSchema) - clientError1.toString must contain( + schemaKey1 must beEqualTo(AtomicFields.atomicSchema) + clientError1.toString must contain("tr_tt") + clientError1.toString must contain("Cannot be converted to java.math.BigDecimal") + schemaKey2 must beEqualTo(emailSentSchema) + clientError2.toString must contain( "unallowedAdditionalField: is not defined in the schema and the schema does not allow additional properties" ) - schemaKey2 must beEqualTo(AtomicFields.atomicSchema) - clientError2.toString must contain("tr_tt") - clientError2.toString must contain("Cannot be converted to java.math.BigDecimal") case other => - ko(s"[$other] is not a SchemaViolations bad row with 2 IgluError") - }) + ko(s"[$other] is not a SchemaViolations bad row with 2 expected IgluError") + } } "return an EnrichmentFailures bad row if one of the enrichment (JS enrichment here) fails" >> { @@ -232,13 +239,14 @@ 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( + BadRowFailure.EnrichmentFailures( _, NonEmptyList( FailureDetails.EnrichmentFailure( @@ -249,13 +257,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 a SchemaViolations bad row containing one IgluError if one of the contexts added by the enrichments is invalid" >> { @@ -302,24 +308,26 @@ 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( + enriched.value map { + case Ior.Left( + BadRow.SchemaViolations( _, - Failure.SchemaViolations( + BadRowFailure.SchemaViolations( _, NonEmptyList( _: FailureDetails.SchemaViolation.IgluError, Nil ) ), - payload - ) if payload.enriched.derived_contexts.isDefined => - ok - case br => ko(s"[$br] is not a SchemaViolations bad row containing one IgluError and with derived_contexts defined") - }) + _ + ) + ) => + ok + case other => ko(s"[$other] is not a SchemaViolations bad row with one IgluError") + } } "emit an EnrichedEvent if everything goes well" >> { @@ -365,9 +373,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" >> { @@ -431,9 +443,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" >> { @@ -497,9 +513,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" >> { @@ -563,9 +583,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" >> { @@ -629,9 +653,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" >> { @@ -701,9 +729,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" >> { @@ -729,9 +761,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") + } } } } @@ -759,9 +795,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") + } } } } @@ -798,11 +838,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") + } } } @@ -825,12 +867,14 @@ 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) => + enriched.useragent must_== qs_ua + enriched.derived_contexts must contain("\"agentName\":\"Firefox\"") + case other => ko(s"[$other] is not an enriched event") } } @@ -851,10 +895,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") } } @@ -876,10 +922,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") } } @@ -902,12 +950,14 @@ 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) => + enriched.useragent must_== qs_ua + enriched.derived_contexts must contain("\"agentName\":\"%1$S\"") + case other => ko(s"[$other] is not an enriched event") } } @@ -948,10 +998,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") } } @@ -998,11 +1050,14 @@ 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) => + enriched.app_id must_== "test_app_id" + enriched.platform must_== "test_platform" + case other => ko(s"[$other] is not an enriched event") } } @@ -1152,20 +1207,646 @@ 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") + } + } + + "remove the invalid unstructured event and enrich the event if emitIncomplete is set to true" >> { + val script = + s""" + function process(event) { + return [ ${emailSent} ]; + }""" + val schemaKey = SchemaKey( + "com.snowplowanalytics.snowplow", + "javascript_script_config", + "jsonschema", + SchemaVer.Full(1, 0, 0) + ) + val enrichmentReg = EnrichmentRegistry[IO]( + javascriptScript = List( + JavascriptScriptEnrichment(schemaKey, script) + ) + ) + val invalidUeData = + """{ + "emailAddress": "hello@world.com", + "emailAddress2": "foo@bar.org", + "unallowedAdditionalField": "foo@bar.org" + }""" + val invalidUe = + s"""{ + "schema":"iglu:com.acme/email_sent/jsonschema/1-0-0", + "data": $invalidUeData + }""" + + val parameters = Map( + "e" -> "pp", + "tv" -> "js-0.13.1", + "p" -> "web", + "co" -> + s""" + { + "schema": "iglu:com.snowplowanalytics.snowplow/contexts/jsonschema/1-0-0", + "data": [ + $clientSession + ] + } + """, + "ue_pr" -> + s""" + { + "schema":"iglu:com.snowplowanalytics.snowplow/unstruct_event/jsonschema/1-0-0", + "data":$invalidUe + }""" + ).toOpt + val rawEvent = RawEvent(api, parameters, None, source, context) + def expectedDerivedContexts(enriched: EnrichedEvent): Boolean = { + val emailSentSDJ = SelfDescribingData.parse[Json](jparse(emailSent).toOption.get).toOption.get + SpecHelpers.listContexts(enriched.derived_contexts) match { + case List(SelfDescribingData(Failure.`failureSchemaKey`, feJson), `emailSentSDJ`) + if feJson.field("failureType") == "ValidationError".asJson && + feJson.field("errors") == Json.arr( + Json.obj( + "message" := "$.unallowedAdditionalField: is not defined in the schema and the schema does not allow additional properties", + "source" := "unstruct", + "path" := "$", + "keyword" := "additionalProperties", + "targets" := List("unallowedAdditionalField") + ) + ) && + feJson.field("schema") == emailSentSchema.asJson && + feJson.field("data") == jparse(invalidUeData).toOption.get => + true + case _ => false + } + } + + val enriched = EnrichmentManager.enrichEvent[IO]( + enrichmentReg, + client, + processor, + timestamp, + rawEvent, + AcceptInvalid.featureFlags, + IO.unit, + SpecHelpers.registryLookup, + atomicFieldLimits, + emitIncomplete = true + ) + + enriched.value.map { + case Ior.Both(_: BadRow.SchemaViolations, enriched) + if Option(enriched.unstruct_event).isEmpty && + SpecHelpers.listContextsSchemas(enriched.contexts) == List(clientSessionSchema) && + expectedDerivedContexts(enriched) => + ok + case other => ko(s"[$other] is not a SchemaViolations bad row and an enriched event without the unstructured event") + } + } + + "remove the invalid context and enrich the event if emitIncomplete is set to true" >> { + val script = + s""" + function process(event) { + return [ ${emailSent} ]; + }""" + val schemaKey = SchemaKey( + "com.snowplowanalytics.snowplow", + "javascript_script_config", + "jsonschema", + SchemaVer.Full(1, 0, 0) + ) + val enrichmentReg = EnrichmentRegistry[IO]( + javascriptScript = List( + JavascriptScriptEnrichment(schemaKey, script) + ) + ) + val invalidContextData = + """{ + "foo": "hello@world.com", + "emailAddress2": "foo@bar.org" + }""" + val invalidContext = + s"""{ + "schema":"iglu:com.acme/email_sent/jsonschema/1-0-0", + "data": $invalidContextData + }""" + + val parameters = Map( + "e" -> "pp", + "tv" -> "js-0.13.1", + "p" -> "web", + "co" -> + s""" + { + "schema": "iglu:com.snowplowanalytics.snowplow/contexts/jsonschema/1-0-0", + "data": [$invalidContext] + } + """, + "ue_pr" -> + s""" + { + "schema":"iglu:com.snowplowanalytics.snowplow/unstruct_event/jsonschema/1-0-0", + "data": $clientSession + }""" + ).toOpt + val rawEvent = RawEvent(api, parameters, None, source, context) + def expectedDerivedContexts(enriched: EnrichedEvent): Boolean = { + val emailSentSDJ = SelfDescribingData.parse[Json](jparse(emailSent).toOption.get).toOption.get + SpecHelpers.listContexts(enriched.derived_contexts) match { + case List(SelfDescribingData(Failure.`failureSchemaKey`, feJson), `emailSentSDJ`) + if feJson.field("failureType") == "ValidationError".asJson && + feJson.field("errors") == Json.arr( + Json.obj( + "message" := "$.emailAddress: is missing but it is required", + "source" := "contexts", + "path" := "$", + "keyword" := "required", + "targets" := List("emailAddress") + ), + Json.obj( + "message" := "$.foo: is not defined in the schema and the schema does not allow additional properties", + "source" := "contexts", + "path" := "$", + "keyword" := "additionalProperties", + "targets" := List("foo") + ) + ) && + feJson.field("schema") == emailSentSchema.asJson && + feJson.field("data") == jparse(invalidContextData).toOption.get => + true + case _ => false + } + } + + val enriched = EnrichmentManager.enrichEvent[IO]( + enrichmentReg, + client, + processor, + timestamp, + rawEvent, + AcceptInvalid.featureFlags, + IO.unit, + SpecHelpers.registryLookup, + atomicFieldLimits, + emitIncomplete = true + ) + enriched.value.map { + case Ior.Both(_: BadRow.SchemaViolations, enriched) + if Option(enriched.contexts).isEmpty && + SpecHelpers.getUnstructSchema(enriched.unstruct_event) == clientSessionSchema && + expectedDerivedContexts(enriched) => + ok + case other => ko(s"[$other] is not a SchemaViolations bad row and an enriched event with no input contexts") + } + } + + "remove one invalid context (out of 2) and enrich the event if emitIncomplete is set to true" >> { + val script = + s""" + function process(event) { + return [ ${emailSent} ]; + }""" + val schemaKey = SchemaKey( + "com.snowplowanalytics.snowplow", + "javascript_script_config", + "jsonschema", + SchemaVer.Full(1, 0, 0) + ) + val enrichmentReg = EnrichmentRegistry[IO]( + javascriptScript = List( + JavascriptScriptEnrichment(schemaKey, script) + ) + ) + val invalidContextData = + """ + { + "foo": "hello@world.com", + "emailAddress2": "foo@bar.org" + }""" + val invalidContext = + s""" + { + "schema":"iglu:com.acme/email_sent/jsonschema/1-0-0", + "data": $invalidContextData + }""" + + val parameters = Map( + "e" -> "pp", + "tv" -> "js-0.13.1", + "p" -> "web", + "co" -> + s""" + { + "schema": "iglu:com.snowplowanalytics.snowplow/contexts/jsonschema/1-0-0", + "data": [ + $invalidContext, + $clientSession + ] + } + """, + "ue_pr" -> + s""" + { + "schema":"iglu:com.snowplowanalytics.snowplow/unstruct_event/jsonschema/1-0-0", + "data": $clientSession + }""" + ).toOpt + val rawEvent = RawEvent(api, parameters, None, source, context) + def expectedDerivedContexts(enriched: EnrichedEvent): Boolean = { + val emailSentSDJ = SelfDescribingData.parse[Json](jparse(emailSent).toOption.get).toOption.get + SpecHelpers.listContexts(enriched.derived_contexts) match { + case List(SelfDescribingData(Failure.`failureSchemaKey`, feJson), `emailSentSDJ`) + if feJson.field("failureType") == "ValidationError".asJson && + feJson.field("errors") == Json.arr( + Json.obj( + "message" := "$.emailAddress: is missing but it is required", + "source" := "contexts", + "path" := "$", + "keyword" := "required", + "targets" := List("emailAddress") + ), + Json.obj( + "message" := "$.foo: is not defined in the schema and the schema does not allow additional properties", + "source" := "contexts", + "path" := "$", + "keyword" := "additionalProperties", + "targets" := List("foo") + ) + ) && + feJson.field("schema") == emailSentSchema.asJson && + feJson.field("data") == jparse(invalidContextData).toOption.get => + true + case _ => false + } + } + + val enriched = EnrichmentManager.enrichEvent[IO]( + enrichmentReg, + client, + processor, + timestamp, + rawEvent, + AcceptInvalid.featureFlags, + IO.unit, + SpecHelpers.registryLookup, + atomicFieldLimits, + emitIncomplete = true + ) + enriched.value.map { + case Ior.Both(_: BadRow.SchemaViolations, enriched) + if SpecHelpers.getUnstructSchema(enriched.unstruct_event) == clientSessionSchema && + SpecHelpers.listContextsSchemas(enriched.contexts) == List(clientSessionSchema) && + expectedDerivedContexts(enriched) => + ok + case other => ko(s"[$other] is not a SchemaViolations bad row and an enriched event with 1 input context") + } + } + + "return the enriched event after an enrichment exception if emitIncomplete is set to true" >> { + val script = + s""" + function process(event) { + throw "Javascript exception"; + return [ $emailSent ]; + }""" + val schemaKey = SchemaKey( + "com.snowplowanalytics.snowplow", + "javascript_script_config", + "jsonschema", + SchemaVer.Full(1, 0, 0) + ) + val enrichmentReg = EnrichmentRegistry[IO]( + yauaa = Some(YauaaEnrichment(None)), + javascriptScript = List( + JavascriptScriptEnrichment(schemaKey, script) + ) + ) + + val parameters = Map( + "e" -> "pp", + "tv" -> "js-0.13.1", + "p" -> "web", + "ue_pr" -> + s""" + { + "schema":"iglu:com.snowplowanalytics.snowplow/unstruct_event/jsonschema/1-0-0", + "data": $clientSession + }""" + ).toOpt + val rawEvent = RawEvent(api, parameters, None, source, context) + def expectedDerivedContexts(enriched: EnrichedEvent): Boolean = + SpecHelpers.listContexts(enriched.derived_contexts) match { + case List( + SelfDescribingData(Failure.`failureSchemaKey`, feJson), + SelfDescribingData(SchemaKey("nl.basjes", "yauaa_context", "jsonschema", _), _) + ) + if feJson.field("failureType") == "EnrichmentError: Javascript enrichment".asJson && + feJson.field("errors") == Json.arr( + Json.obj( + "message" := "Error during execution of JavaScript function: [Javascript exception in at line number 3 at column number 10]" + ) + ) && + feJson.field("schema") == JavascriptScriptEnrichment.supportedSchema.copy(addition = 0.some).asString.asJson && + feJson.field("data") == Json.Null => + true + case _ => false + } + + val enriched = EnrichmentManager.enrichEvent[IO]( + enrichmentReg, + client, + processor, + timestamp, + rawEvent, + AcceptInvalid.featureFlags, + IO.unit, + SpecHelpers.registryLookup, + atomicFieldLimits, + emitIncomplete = true + ) + enriched.value.map { + case Ior.Both(_: BadRow.EnrichmentFailures, enriched) + if SpecHelpers.getUnstructSchema(enriched.unstruct_event) == clientSessionSchema && + expectedDerivedContexts(enriched) => + ok + case other => ko(s"[$other] is not an EnrichmentFailures bad row and an enriched event") + } + } + + "return a SchemaViolations bad row in the Left in case of both a schema violation and an enrichment failure if emitIncomplete is set to true" >> { + val script = + s""" + function process(event) { + throw "Javascript exception"; + return [ $emailSent ]; + }""" + val schemaKey = SchemaKey( + "com.snowplowanalytics.snowplow", + "javascript_script_config", + "jsonschema", + SchemaVer.Full(1, 0, 0) + ) + val enrichmentReg = EnrichmentRegistry[IO]( + javascriptScript = List( + JavascriptScriptEnrichment(schemaKey, script) + ) + ) + + val parameters = Map( + "e" -> "pp", + "tv" -> "js-0.13.1", + "p" -> "web", + "tr_tt" -> "foo", + "ue_pr" -> + s""" + { + "schema":"iglu:com.snowplowanalytics.snowplow/unstruct_event/jsonschema/1-0-0", + "data": $clientSession + }""" + ).toOpt + val rawEvent = RawEvent(api, parameters, None, source, context) + def expectedDerivedContexts(enriched: EnrichedEvent): Boolean = + SpecHelpers.listContexts(enriched.derived_contexts) match { + case List( + SelfDescribingData(Failure.`failureSchemaKey`, validationError), + SelfDescribingData(Failure.`failureSchemaKey`, enrichmentError) + ) + if validationError.field("failureType") == "ValidationError".asJson && + validationError.field("errors") == Json.arr( + Json.obj( + "message" := "Cannot be converted to java.math.BigDecimal. Error : Character f is neither a decimal digit number, decimal point, nor \"e\" notation exponential mark.", + "source" := AtomicError.source, + "path" := "tr_tt", + "keyword" := AtomicError.keywordParse, + "targets" := Json.arr() + ) + ) && + validationError.field("schema") == AtomicFields.atomicSchema.asJson && + validationError.field("data") == Json.obj("tr_tt" := "foo") && + enrichmentError.field("failureType") == "EnrichmentError: Javascript enrichment".asJson && + enrichmentError.field("errors") == Json.arr( + Json.obj( + "message" := "Error during execution of JavaScript function: [Javascript exception in at line number 3 at column number 10]" + ) + ) && + enrichmentError.field("schema") == JavascriptScriptEnrichment.supportedSchema.copy(addition = 0.some).asString.asJson && + enrichmentError.field("data") == Json.Null => + true + case _ => false + } + + val enriched = EnrichmentManager.enrichEvent[IO]( + enrichmentReg, + client, + processor, + timestamp, + rawEvent, + AcceptInvalid.featureFlags, + IO.unit, + SpecHelpers.registryLookup, + atomicFieldLimits, + emitIncomplete = true + ) + enriched.value.map { + case Ior.Both(_: BadRow.SchemaViolations, enriched) if expectedDerivedContexts(enriched) => ok + case other => ko(s"[$other] doesn't have a SchemaViolations bad row in the Left") + } + } + + "remove an invalid enrichment context and return the enriched event if emitIncomplete is set to true" >> { + val invalidContextData = + """ + { + "foo": "hello@world.com", + "emailAddress2": "foo@bar.org" + }""" + val invalidContext = + s""" + { + "schema":"iglu:com.acme/email_sent/jsonschema/1-0-0", + "data": $invalidContextData + }""" + val script = + s""" + function process(event) { + return [ + $invalidContext + ]; + }""" + val schemaKey = SchemaKey( + "com.snowplowanalytics.snowplow", + "javascript_script_config", + "jsonschema", + SchemaVer.Full(1, 0, 0) + ) + val enrichmentReg = EnrichmentRegistry[IO]( + yauaa = Some(YauaaEnrichment(None)), + javascriptScript = List( + JavascriptScriptEnrichment(schemaKey, script) + ) + ) + + val parameters = Map( + "e" -> "pp", + "tv" -> "js-0.13.1", + "p" -> "web", + "ue_pr" -> + s""" + { + "schema":"iglu:com.snowplowanalytics.snowplow/unstruct_event/jsonschema/1-0-0", + "data": $clientSession + }""" + ).toOpt + val rawEvent = RawEvent(api, parameters, None, source, context) + def expectedDerivedContexts(enriched: EnrichedEvent): Boolean = + SpecHelpers.listContexts(enriched.derived_contexts) match { + case List( + SelfDescribingData(Failure.`failureSchemaKey`, feJson), + SelfDescribingData(SchemaKey("nl.basjes", "yauaa_context", "jsonschema", _), _) + ) + if feJson.field("failureType") == "ValidationError".asJson && + feJson.field("errors") == Json.arr( + Json.obj( + "message" := "$.emailAddress: is missing but it is required", + "source" := "derived_contexts", + "path" := "$", + "keyword" := "required", + "targets" := List("emailAddress") + ), + Json.obj( + "message" := "$.foo: is not defined in the schema and the schema does not allow additional properties", + "source" := "derived_contexts", + "path" := "$", + "keyword" := "additionalProperties", + "targets" := List("foo") + ) + ) && + feJson.field("schema") == emailSentSchema.asJson && + feJson.field("data") == jparse(invalidContextData).toOption.get => + true + case _ => false + } + + val enriched = EnrichmentManager.enrichEvent[IO]( + enrichmentReg, + client, + processor, + timestamp, + rawEvent, + AcceptInvalid.featureFlags, + IO.unit, + SpecHelpers.registryLookup, + atomicFieldLimits, + emitIncomplete = true + ) + enriched.value.map { + case Ior.Both(_: BadRow.SchemaViolations, enriched) + if SpecHelpers.getUnstructSchema(enriched.unstruct_event) == clientSessionSchema && + expectedDerivedContexts(enriched) => + ok + case other => ko(s"[$other] is not a SchemaViolations bad row and an enriched event without the faulty enrichment context") + } + } + + "return a bad row that contains validation errors only from ue if there is validation error in both ue and derived contexts when emitIncomplete is set to true" >> { + val invalidContext = + """ + { + "schema":"iglu:com.acme/email_sent/jsonschema/1-0-0", + "data": { + "foo": "hello@world.com", + "emailAddress2": "foo@bar.org" + } + }""" + val invalidUe = + """{ + "schema":"iglu:com.snowplowanalytics.snowplow/client_session/jsonschema/1-0-1", + "data": { + "unallowedAdditionalField": "foo@bar.org" + } + }""" + val script = + s""" + function process(event) { + return [ + $invalidContext + ]; + }""" + val schemaKey = SchemaKey( + "com.snowplowanalytics.snowplow", + "javascript_script_config", + "jsonschema", + SchemaVer.Full(1, 0, 0) + ) + val enrichmentReg = EnrichmentRegistry[IO]( + javascriptScript = List( + JavascriptScriptEnrichment(schemaKey, script) + ) + ) + val parameters = Map( + "e" -> "pp", + "tv" -> "js-0.13.1", + "p" -> "web", + "ue_pr" -> + s""" + { + "schema":"iglu:com.snowplowanalytics.snowplow/unstruct_event/jsonschema/1-0-0", + "data": $invalidUe + }""" + ).toOpt + val rawEvent = RawEvent(api, parameters, None, source, context) + val enriched = EnrichmentManager.enrichEvent[IO]( + enrichmentReg, + client, + processor, + timestamp, + rawEvent, + AcceptInvalid.featureFlags, + IO.unit, + SpecHelpers.registryLookup, + atomicFieldLimits, + emitIncomplete = true + ) + def expectedDerivedContexts(enriched: EnrichedEvent): Boolean = + SpecHelpers.listContextsSchemas(enriched.derived_contexts).count(_ == Failure.failureSchemaKey) == 2 + + def expectedBadRow(badRow: BadRow): Boolean = + badRow match { + case BadRow.SchemaViolations( + _, + BadRowFailure.SchemaViolations( + _, + NonEmptyList(FailureDetails.SchemaViolation.IgluError(`clientSessionSchema`, _), Nil) + ), + _ + ) => + true + case _ => false + } + + enriched.value.map { + case Ior.Both(badRow, enriched) + if Option(enriched.unstruct_event).isEmpty && + expectedDerivedContexts(enriched) && + expectedBadRow(badRow) => + ok + case other => ko(s"[$other] is not expected one") + } } } @@ -1630,7 +2311,7 @@ class EnrichmentManagerSpec extends Specification with EitherMatchers with CatsE "validateEnriched" should { "create a SchemaViolations bad row if an atomic field is oversized" >> { - val result = EnrichmentManager + EnrichmentManager .enrichEvent[IO]( enrichmentReg, client, @@ -1640,26 +2321,28 @@ class EnrichmentManagerSpec extends Specification with EitherMatchers with CatsE featureFlags = AcceptInvalid.featureFlags.copy(acceptInvalid = false), IO.unit, SpecHelpers.registryLookup, - atomicFieldLimits + atomicFieldLimits, + emitIncomplete ) - - result.value - .map(_ must beLeft.like { - case BadRow.SchemaViolations( - _, - Failure.SchemaViolations(_, NonEmptyList(FailureDetails.SchemaViolation.IgluError(schemaKey, clientError), Nil)), - _ + .value + .map { + case Ior.Left( + BadRow.SchemaViolations( + _, + BadRowFailure.SchemaViolations(_, NonEmptyList(FailureDetails.SchemaViolation.IgluError(schemaKey, clientError), Nil)), + _ + ) ) => schemaKey must beEqualTo(AtomicFields.atomicSchema) clientError.toString must contain("v_tracker") clientError.toString must contain("Field is longer than maximum allowed size") case other => ko(s"[$other] is not a SchemaViolations bad row with one IgluError") - }) + } } "not create a bad row if an atomic field is oversized and acceptInvalid is set to true" >> { - val result = EnrichmentManager + EnrichmentManager .enrichEvent[IO]( enrichmentReg, client, @@ -1669,11 +2352,14 @@ 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]) + .map { + case Ior.Right(_) => ok + case other => ko(s"[$other] is not an enriched event") + } } "return a SchemaViolations bad row containing both the atomic field length error and the invalid enrichment context error" >> { @@ -1716,27 +2402,95 @@ class EnrichmentManagerSpec extends Specification with EitherMatchers with CatsE AcceptInvalid.featureFlags, IO.unit, SpecHelpers.registryLookup, - atomicFieldLimits + atomicFieldLimits, + emitIncomplete ) .value - .map(_ must beLeft.like { - case BadRow.SchemaViolations( - _, - Failure.SchemaViolations(_, - NonEmptyList(FailureDetails.SchemaViolation.IgluError(schemaKey1, clientError1), - List(FailureDetails.SchemaViolation.IgluError(schemaKey2, clientError2)) - ) - ), - _ + .map { + case Ior.Left( + BadRow.SchemaViolations( + _, + BadRowFailure.SchemaViolations(_, + NonEmptyList(FailureDetails.SchemaViolation.IgluError(schemaKey1, clientError1), + List(FailureDetails.SchemaViolation.IgluError(schemaKey2, clientError2)) + ) + ), + _ + ) ) => - schemaKey1 must beEqualTo(AtomicFields.atomicSchema) - clientError1.toString must contain("v_tracker") - clientError1.toString must contain("Field is longer than maximum allowed size") - schemaKey2 must beEqualTo(emailSentSchema) - clientError2.toString must contain("emailAddress2: is missing but it is required") + schemaKey1 must beEqualTo(emailSentSchema) + clientError1.toString must contain("emailAddress2: is missing but it is required") + schemaKey2 must beEqualTo(AtomicFields.atomicSchema) + clientError2.toString must contain("v_tracker") + clientError2.toString must contain("Field is longer than maximum allowed size") case other => ko(s"[$other] is not a SchemaViolations bad row with 2 IgluError") - }) + } + } + + "remove an oversized atomic field if emitIncomplete is set to true" >> { + val enriched = EnrichmentManager + .enrichEvent[IO]( + enrichmentReg, + client, + processor, + timestamp, + RawEvent(api, fatBody, None, source, context), + featureFlags = AcceptInvalid.featureFlags.copy(acceptInvalid = false), + IO.unit, + SpecHelpers.registryLookup, + atomicFieldLimits, + emitIncomplete = true + ) + + enriched.value.map { + case Ior.Both(_: BadRow.SchemaViolations, enriched) if Option(enriched.v_tracker).isEmpty => ok + case other => ko(s"[$other] is not a SchemaViolations bad row and an enriched event without tracker version") + } + } + } + + "setDerivedContexts" should { + val sv = Failure.SchemaViolation( + schemaViolation = FailureDetails.SchemaViolation.NotJson("testField", "testValue".some, "testError"), + source = "testSource", + data = Json.obj("testKey" := "testValue") + ) + val ef = Failure.EnrichmentFailure( + FailureDetails.EnrichmentFailure( + None, + FailureDetails.EnrichmentFailureMessage.Simple("testError") + ) + ) + val emailSentSDJ = SelfDescribingData.parse[Json](jparse(emailSent).toOption.get).toOption.get + val timestamp = Instant.now() + "set derived contexts correctly if enrichment result is Ior.Left" >> { + val enriched = new EnrichedEvent() + val enrichmentResult = Ior.Left(NonEmptyList.of(NonEmptyList.of(sv, ef), NonEmptyList.of(sv, ef))) + EnrichmentManager.setDerivedContexts(enriched, enrichmentResult, timestamp, processor) + val schemas = SpecHelpers.listContextsSchemas(enriched.derived_contexts) + schemas.size must beEqualTo(4) + forall(schemas)(s => s must beEqualTo(Failure.failureSchemaKey)) + } + "set derived contexts correctly if enrichment result is Ior.Right" >> { + val enriched = new EnrichedEvent() + val enrichmentResult = Ior.Right(List(emailSentSDJ, emailSentSDJ)) + EnrichmentManager.setDerivedContexts(enriched, enrichmentResult, timestamp, processor) + val schemas = SpecHelpers.listContextsSchemas(enriched.derived_contexts) + schemas.size must beEqualTo(2) + forall(schemas)(s => s must beEqualTo(emailSentSchema)) + } + "set derived contexts correctly if enrichment result is Ior.Both" >> { + val enriched = new EnrichedEvent() + val enrichmentResult = Ior.Both( + NonEmptyList.of(NonEmptyList.of(sv, ef), NonEmptyList.of(sv, ef)), + List(emailSentSDJ, emailSentSDJ) + ) + EnrichmentManager.setDerivedContexts(enriched, enrichmentResult, timestamp, processor) + val schemas = SpecHelpers.listContextsSchemas(enriched.derived_contexts) + schemas.size must beEqualTo(6) + schemas.count(_ == Failure.failureSchemaKey) must beEqualTo(4) + schemas.count(_ == emailSentSchema) must beEqualTo(2) } } } @@ -1813,4 +2567,37 @@ object EnrichmentManagerSpec { "jsonschema", SchemaVer.Full(1, 0, 0) ) + + val emailSent = s"""{ + "schema": "${emailSentSchema.toSchemaUri}", + "data": { + "emailAddress": "hello@world.com", + "emailAddress2": "foo@bar.org" + } + }""" + + val clientSessionSchema = + SchemaKey( + "com.snowplowanalytics.snowplow", + "client_session", + "jsonschema", + SchemaVer.Full(1, 0, 1) + ) + + val clientSession = s"""{ + "schema": "${clientSessionSchema.toSchemaUri}", + "data": { + "sessionIndex": 1, + "storageMechanism": "LOCAL_STORAGE", + "firstEventId": "5c33fccf-6be5-4ce6-afb1-e34026a3ca75", + "sessionId": "21c2a0dd-892d-42d1-b156-3a9d4e147eef", + "previousSessionId": null, + "userId": "20d631b8-7837-49df-a73e-6da73154e6fd" + } + }""" + + implicit class JsonFieldGetter(json: Json) { + def field(f: String): Json = + json.hcursor.downField(f).as[Json].toOption.get + } } diff --git a/modules/common/src/test/scala/com.snowplowanalytics.snowplow.enrich.common/enrichments/FailureSpec.scala b/modules/common/src/test/scala/com.snowplowanalytics.snowplow.enrich.common/enrichments/FailureSpec.scala new file mode 100644 index 000000000..d8dee9940 --- /dev/null +++ b/modules/common/src/test/scala/com.snowplowanalytics.snowplow.enrich.common/enrichments/FailureSpec.scala @@ -0,0 +1,395 @@ +/* + * Copyright (c) 2014-present Snowplow Analytics Ltd. + * All rights reserved. + * + * This software is made available by Snowplow Analytics, Ltd., + * under the terms of the Snowplow Limited Use License Agreement, Version 1.0 + * located at https://docs.snowplow.io/limited-use-license-1.0 + * BY INSTALLING, DOWNLOADING, ACCESSING, USING OR DISTRIBUTING ANY PORTION + * OF THE SOFTWARE, YOU AGREE TO THE TERMS OF SUCH LICENSE AGREEMENT. + */ +package com.snowplowanalytics.snowplow.enrich.common.enrichments + +import java.time.Instant + +import scala.collection.immutable.SortedMap + +import cats.effect.testing.specs2.CatsEffect +import cats.effect.unsafe.implicits.global +import cats.effect.IO + +import cats.data.NonEmptyList +import cats.syntax.option._ + +import io.circe.syntax._ +import io.circe.Json + +import org.specs2.mutable.Specification +import org.specs2.matcher.ValidatedMatchers +import org.specs2.ScalaCheck + +import org.scalacheck.{Gen, Prop} + +import com.snowplowanalytics.snowplow.badrows.{FailureDetails, Processor} + +import com.snowplowanalytics.iglu.core.{ParseError, SchemaCriterion, SchemaKey, SchemaVer, SelfDescribingData} + +import com.snowplowanalytics.iglu.client.ClientError +import com.snowplowanalytics.iglu.client.validator.{ValidatorError, ValidatorReport} +import com.snowplowanalytics.iglu.client.resolver.LookupHistory +import com.snowplowanalytics.iglu.client.resolver.registries.RegistryLookup + +import com.snowplowanalytics.snowplow.enrich.common.utils.AtomicError + +import com.snowplowanalytics.snowplow.enrich.common.SpecHelpers + +class FailureSpec extends Specification with ValidatedMatchers with CatsEffect with ScalaCheck { + + val timestamp = Instant.now() + val processor = Processor("unit tests SCE", "v42") + val schemaKey = SchemaKey("com.snowplowanalytics", "test", "jsonschema", SchemaVer.Full(1, 0, 0)) + val schemaCriterion = SchemaCriterion.apply("com.snowplowanalytics", "test", "jsonschema", 1) + + "FailureEntityContext should be valid against its schema" >> { + implicit val registryLookup: RegistryLookup[IO] = SpecHelpers.registryLookup + + val genFeContext = for { + failureType <- Gen.alphaNumStr + jsonGen = Gen.oneOf( + Json.obj(), + Json.obj("test1" := "value1"), + Json.obj("test1" := "value1", "test2" := "value2"), + Json.obj("test1" := "value1", "test2" := "value2", "test3" := "value3") + ) + errors <- Gen.listOf(jsonGen) + data <- Gen.option(jsonGen) + schema <- Gen.option(Gen.const(schemaKey)) + } yield Failure.FailureContext( + failureType = failureType, + errors = errors, + schema = schema, + data = data, + timestamp = timestamp, + componentName = processor.artifact, + componentVersion = processor.version + ) + + Prop.forAll(genFeContext) { feContext: Failure.FailureContext => + val sdj = SelfDescribingData(schema = Failure.failureSchemaKey, data = feContext.asJson) + SpecHelpers.client + .check(sdj) + .value + .map(_ must beRight) + .unsafeRunSync() + } + } + + "fromEnrichmentFailure" should { + "convert InputData correctly" >> { + val ef = Failure.EnrichmentFailure( + enrichmentFailure = FailureDetails.EnrichmentFailure( + enrichment = FailureDetails + .EnrichmentInformation( + schemaKey = schemaKey, + identifier = "enrichmentId" + ) + .some, + message = FailureDetails.EnrichmentFailureMessage.InputData( + field = "testField", + value = "testValue".some, + expectation = "testExpectation" + ) + ) + ) + val result = Failure.fromEnrichmentFailure(ef, timestamp, processor) + val expected = Failure.FailureContext( + failureType = "EnrichmentError: enrichmentId", + errors = List( + Json.obj( + "message" := "testField - testExpectation", + "source" := "testField" + ) + ), + schema = schemaKey.some, + data = Json.obj("testField" := "testValue").some, + timestamp = timestamp, + componentName = processor.artifact, + componentVersion = processor.version + ) + result must beEqualTo(expected) + } + + "convert Simple correctly" >> { + val ef = Failure.EnrichmentFailure( + enrichmentFailure = FailureDetails.EnrichmentFailure( + enrichment = FailureDetails + .EnrichmentInformation( + schemaKey = schemaKey, + identifier = "enrichmentId" + ) + .some, + message = FailureDetails.EnrichmentFailureMessage.Simple(error = "testError") + ) + ) + val result = Failure.fromEnrichmentFailure(ef, timestamp, processor) + val expected = Failure.FailureContext( + failureType = "EnrichmentError: enrichmentId", + errors = List(Json.obj("message" := "testError")), + schema = schemaKey.some, + data = None, + timestamp = timestamp, + componentName = processor.artifact, + componentVersion = processor.version + ) + result must beEqualTo(expected) + } + } + + "fromSchemaViolation" should { + "convert NotJson correctly" >> { + val sv = Failure.SchemaViolation( + schemaViolation = FailureDetails.SchemaViolation.NotJson( + field = "testField", + value = "testValue".some, + error = "testError" + ), + source = "testSource", + data = "testData".asJson + ) + val fe = Failure.fromSchemaViolation(sv, timestamp, processor) + val expected = Failure.FailureContext( + failureType = "NotJSON", + errors = List( + Json.obj( + "message" := "testError", + "source" := "testSource" + ) + ), + schema = None, + data = Json.obj("testSource" := "testData").some, + timestamp = timestamp, + componentName = processor.artifact, + componentVersion = processor.version + ) + fe must beEqualTo(expected) + } + + "convert NotIglu correctly" >> { + val sv = Failure.SchemaViolation( + schemaViolation = FailureDetails.SchemaViolation.NotIglu( + json = Json.Null, + error = ParseError.InvalidSchema + ), + source = "testSource", + data = "testData".asJson + ) + val fe = Failure.fromSchemaViolation(sv, timestamp, processor) + val expected = Failure.FailureContext( + failureType = "NotIglu", + errors = List( + Json.obj( + "message" := "Invalid schema", + "source" := "testSource" + ) + ), + schema = None, + data = "testData".asJson.some, + timestamp = timestamp, + componentName = processor.artifact, + componentVersion = processor.version + ) + fe must beEqualTo(expected) + } + + "convert CriterionMismatch correctly" >> { + val sv = Failure.SchemaViolation( + schemaViolation = FailureDetails.SchemaViolation.CriterionMismatch( + schemaKey = schemaKey, + schemaCriterion = schemaCriterion + ), + source = "testSource", + data = "testData".asJson + ) + val fe = Failure.fromSchemaViolation(sv, timestamp, processor) + val expected = Failure.FailureContext( + failureType = "CriterionMismatch", + errors = List( + Json.obj( + "message" := "Unexpected schema: iglu:com.snowplowanalytics/test/jsonschema/1-0-0 does not match the criterion", + "source" := "testSource", + "criterion" := "iglu:com.snowplowanalytics/test/jsonschema/1-*-*" + ) + ), + schema = schemaKey.some, + data = "testData".asJson.some, + timestamp = timestamp, + componentName = processor.artifact, + componentVersion = processor.version + ) + fe must beEqualTo(expected) + } + + "convert ResolutionError correctly" >> { + val sv = Failure.SchemaViolation( + schemaViolation = FailureDetails.SchemaViolation.IgluError( + schemaKey = schemaKey, + error = ClientError.ResolutionError( + value = SortedMap( + "repo1" -> LookupHistory( + errors = Set.empty, + attempts = 1, + lastAttempt = timestamp + ), + "repo2" -> LookupHistory( + errors = Set.empty, + attempts = 2, + lastAttempt = timestamp + ) + ) + ) + ), + source = "testSource", + data = "testData".asJson + ) + val fe = Failure.fromSchemaViolation(sv, timestamp, processor) + val expected = Failure.FailureContext( + failureType = "ResolutionError", + errors = List( + Json.obj( + "message" := "Resolution error: schema iglu:com.snowplowanalytics/test/jsonschema/1-0-0 not found", + "source" := "testSource", + "lookupHistory" := Json.arr( + Json.obj("repository" := "repo1", "errors" := List.empty[String], "attempts" := 1, "lastAttempt" := timestamp), + Json.obj("repository" := "repo2", "errors" := List.empty[String], "attempts" := 2, "lastAttempt" := timestamp) + ) + ) + ), + schema = schemaKey.some, + data = "testData".asJson.some, + timestamp = timestamp, + componentName = processor.artifact, + componentVersion = processor.version + ) + fe must beEqualTo(expected) + } + + "convert InvalidData correctly" >> { + def createSv(schemaKey: SchemaKey) = { + val (source, targets1, targets2, keyword1, keyword2) = + if (schemaKey == AtomicFields.atomicSchema) + (AtomicError.source, Nil, Nil, AtomicError.keywordParse, AtomicError.keywordLength) + else + ("testSource", List("testTarget1"), List("testTarget2"), "testKeyword1", "testKeyword2") + Failure.SchemaViolation( + schemaViolation = FailureDetails.SchemaViolation.IgluError( + schemaKey = schemaKey, + error = ClientError.ValidationError( + error = ValidatorError.InvalidData( + messages = NonEmptyList.of( + ValidatorReport(message = "message1", path = "path1".some, targets = targets1, keyword = keyword1.some), + ValidatorReport(message = "message2", path = "path2".some, targets = targets2, keyword = keyword2.some) + ) + ), + supersededBy = None + ) + ), + source = source, + data = "testData".asJson + ) + } + + val svWithAtomicSchema = createSv(AtomicFields.atomicSchema) + val svWithOrdinarySchema = createSv(schemaKey) + val feWithAtomicSchema = Failure.fromSchemaViolation(svWithAtomicSchema, timestamp, processor) + val feWithOrdinarySchema = Failure.fromSchemaViolation(svWithOrdinarySchema, timestamp, processor) + val expectedWithAtomicSchema = Failure.FailureContext( + failureType = "ValidationError", + errors = List( + Json.obj("message" := "message1", + "source" := AtomicError.source, + "path" := "path1", + "keyword" := AtomicError.keywordParse, + "targets" := List.empty[String] + ), + Json.obj("message" := "message2", + "source" := AtomicError.source, + "path" := "path2", + "keyword" := AtomicError.keywordLength, + "targets" := List.empty[String] + ) + ), + schema = AtomicFields.atomicSchema.some, + data = "testData".asJson.some, + timestamp = timestamp, + componentName = processor.artifact, + componentVersion = processor.version + ) + val expectedWithOrdinarySchema = Failure.FailureContext( + failureType = "ValidationError", + errors = List( + Json.obj("message" := "message1", + "source" := "testSource", + "path" := "path1", + "keyword" := "testKeyword1", + "targets" := List("testTarget1") + ), + Json.obj("message" := "message2", + "source" := "testSource", + "path" := "path2", + "keyword" := "testKeyword2", + "targets" := List("testTarget2") + ) + ), + schema = schemaKey.some, + data = "testData".asJson.some, + timestamp = timestamp, + componentName = processor.artifact, + componentVersion = processor.version + ) + + feWithAtomicSchema must beEqualTo(expectedWithAtomicSchema) + feWithOrdinarySchema must beEqualTo(expectedWithOrdinarySchema) + } + + "convert InvalidSchema correctly" >> { + val sv = Failure.SchemaViolation( + schemaViolation = FailureDetails.SchemaViolation.IgluError( + schemaKey = schemaKey, + error = ClientError.ValidationError( + error = ValidatorError.InvalidSchema( + issues = NonEmptyList.of( + ValidatorError.SchemaIssue(path = "testPath1", message = "testMessage1"), + ValidatorError.SchemaIssue(path = "testPath2", message = "testMessage2") + ) + ), + supersededBy = None + ) + ), + source = "testSource", + data = "testData".asJson + ) + val fe = Failure.fromSchemaViolation(sv, timestamp, processor) + val expected = Failure.FailureContext( + failureType = "ValidationError", + errors = List( + Json.obj( + "message" := "Invalid schema: iglu:com.snowplowanalytics/test/jsonschema/1-0-0 - testMessage1", + "source" := "testSource", + "path" := "testPath1" + ), + Json.obj( + "message" := "Invalid schema: iglu:com.snowplowanalytics/test/jsonschema/1-0-0 - testMessage2", + "source" := "testSource", + "path" := "testPath2" + ) + ), + schema = schemaKey.some, + data = "testData".asJson.some, + timestamp = timestamp, + componentName = processor.artifact, + componentVersion = processor.version + ) + fe must beEqualTo(expected) + } + } +} diff --git a/modules/common/src/test/scala/com.snowplowanalytics.snowplow.enrich.common/enrichments/clientEnrichmentSpecs.scala b/modules/common/src/test/scala/com.snowplowanalytics.snowplow.enrich.common/enrichments/clientEnrichmentSpecs.scala index 01e5625ff..c393cc5d7 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,11 +19,11 @@ import com.snowplowanalytics.snowplow.enrich.common.utils.AtomicError class ExtractViewDimensionsSpec extends Specification with DataTables { val FieldName = "res" - def err: AtomicError = - AtomicError.ParseError("""Does not conform to regex (\d+)x(\d+)""", FieldName) + def err(value: String): AtomicError = + AtomicError.ParseError("""Does not conform to regex (\d+)x(\d+)""", FieldName, Option(value)) - def err2: AtomicError = - AtomicError.ParseError("Could not be converted to java.lang.Integer s", FieldName) + def err2(value: String): AtomicError = + AtomicError.ParseError("Could not be converted to java.lang.Integer s", FieldName, Option(value)) def is = s2""" Extracting screen dimensions (viewports, screen resolution etc) with extractViewDimensions should work $e1""" @@ -32,13 +32,13 @@ class ExtractViewDimensionsSpec extends Specification with DataTables { "SPEC NAME" || "INPUT VAL" | "EXPECTED OUTPUT" | "valid desktop" !! "1200x800" ! (1200, 800).asRight | "valid mobile" !! "76x128" ! (76, 128).asRight | - "invalid empty" !! "" ! err.asLeft | - "invalid null" !! null ! err.asLeft | - "invalid hex" !! "76xEE" ! err.asLeft | - "invalid negative" !! "1200x-17" ! err.asLeft | - "Arabic number" !! "٤٥٦٧x680" ! err.asLeft | - "number > int #1" !! "760x3389336768" ! err2.asLeft | - "number > int #2" !! "9989336768x1200" ! err2.asLeft |> { (_, input, expected) => + "invalid empty" !! "" ! err("").asLeft | + "invalid null" !! null ! err(null).asLeft | + "invalid hex" !! "76xEE" ! err("76xEE").asLeft | + "invalid negative" !! "1200x-17" ! err("1200x-17").asLeft | + "Arabic number" !! "٤٥٦٧x680" ! err("٤٥٦٧x680").asLeft | + "number > int #1" !! "760x3389336768" ! err2("760x3389336768").asLeft | + "number > int #2" !! "9989336768x1200" ! err2("9989336768x1200").asLeft |> { (_, input, expected) => ClientEnrichments.extractViewDimensions(FieldName, input) must_== expected } } 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 62a20a009..1b8ba8837 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 @@ -27,8 +27,8 @@ class ExtractEventTypeSpec extends Specification with DataTables { """ val FieldName = "e" - def err: AtomicError = - AtomicError.ParseError("Not a valid event type", FieldName) + def err(value: String): AtomicError = + AtomicError.ParseError("Not a valid event type", FieldName, Option(value)) def e1 = "SPEC NAME" || "INPUT VAL" | "EXPECTED OUTPUT" | @@ -45,10 +45,10 @@ class ExtractEventTypeSpec extends Specification with DataTables { def e2 = "SPEC NAME" || "INPUT VAL" | "EXPECTED OUTPUT" | - "null" !! null ! err | - "empty string" !! "" ! err | - "unrecognized #1" !! "e" ! err | - "unrecognized #2" !! "evnt" ! err |> { (_, input, expected) => + "null" !! null ! err(null) | + "empty string" !! "" ! err("") | + "unrecognized #1" !! "e" ! err("e") | + "unrecognized #2" !! "evnt" ! err("evnt") |> { (_, input, expected) => EventEnrichments.extractEventType(FieldName, input) must beLeft(expected) } @@ -58,9 +58,9 @@ class ExtractEventTypeSpec extends Specification with DataTables { def e3 = // format: off "SPEC NAME" || "INPUT VAL" | "EXPECTED OUTPUT" | - "None" !! None ! AtomicError.ParseError("Field not set", "collector_tstamp").asLeft | - "Negative timestamp" !! BCTstamp ! AtomicError.ParseError("Formatted as -0030-01-01 00:00:00.000 is not Redshift-compatible", "collector_tstamp").asLeft | - ">10k timestamp" !! FarAwayTstamp ! AtomicError.ParseError("Formatted as 11970-01-01 00:00:00.000 is not Redshift-compatible", "collector_tstamp").asLeft | + "None" !! None ! AtomicError.ParseError("Field not set", "collector_tstamp", None).asLeft | + "Negative timestamp" !! BCTstamp ! AtomicError.ParseError("Formatted as -0030-01-01 00:00:00.000 is not Redshift-compatible", "collector_tstamp", BCTstamp.map(_.toString)).asLeft | + ">10k timestamp" !! FarAwayTstamp ! AtomicError.ParseError("Formatted as 11970-01-01 00:00:00.000 is not Redshift-compatible", "collector_tstamp", FarAwayTstamp.map(_.toString)).asLeft | "Valid timestamp" !! SeventiesTstamp ! "1970-01-01 00:00:00.000".asRight |> { // format: on (_, input, expected) => @@ -69,11 +69,12 @@ class ExtractEventTypeSpec extends Specification with DataTables { def e4 = "SPEC NAME" || "INPUT VAL" | "EXPECTED OUTPUT" | - "Not long" !! (("f", "v")) ! AtomicError.ParseError("Not in the expected format: ms since epoch", "f").asLeft | + "Not long" !! (("f", "v")) ! AtomicError.ParseError("Not in the expected format: ms since epoch", "f", Some("v")).asLeft | "Too long" !! (("f", "1111111111111111")) ! AtomicError .ParseError( "Formatting as 37179-09-17 07:18:31.111 is not Redshift-compatible", - "f" + "f", + Some("1111111111111111") ) .asLeft | "Valid ts" !! (("f", "1")) ! "1970-01-01 00:00:00.001".asRight |> { (_, input, 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 ddf22f7aa..37e751fc9 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 @@ -37,8 +37,8 @@ class EtlVersionSpec extends MutSpecification { /** Tests the extractPlatform function. Uses DataTables. */ class ExtractPlatformSpec extends Specification with DataTables { val FieldName = "p" - def err: AtomicError = - AtomicError.ParseError("Not a valid platform", FieldName) + def err(value: String): AtomicError = + AtomicError.ParseError("Not a valid platform", FieldName, Option(value)) def is = s2""" Extracting platforms with extractPlatform should work $e1 @@ -55,9 +55,9 @@ class ExtractPlatformSpec extends Specification with DataTables { "valid games console" !! "cnsl" ! "cnsl".asRight | "valid iot (internet of things)" !! "iot" ! "iot".asRight | "valid headset" !! "headset" ! "headset".asRight | - "invalid empty" !! "" ! err.asLeft | - "invalid null" !! null ! err.asLeft | - "invalid platform" !! "ma" ! err.asLeft |> { (_, input, expected) => + "invalid empty" !! "" ! err("").asLeft | + "invalid null" !! null ! err(null).asLeft | + "invalid platform" !! "ma" ! err("ma").asLeft |> { (_, input, expected) => MiscEnrichments.extractPlatform(FieldName, input) must_== expected } } 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 8f0596eba..d9d8736ae 100644 --- a/modules/common/src/test/scala/com.snowplowanalytics.snowplow.enrich.common/utils/IgluUtilsSpec.scala +++ b/modules/common/src/test/scala/com.snowplowanalytics.snowplow.enrich.common/utils/IgluUtilsSpec.scala @@ -16,15 +16,19 @@ import org.specs2.matcher.ValidatedMatchers import cats.effect.testing.specs2.CatsEffect import io.circe.parser.parse +import io.circe.Json +import io.circe.syntax._ -import cats.data.NonEmptyList +import cats.data.{Ior, NonEmptyList} import com.snowplowanalytics.iglu.core.{SchemaKey, SchemaVer} import com.snowplowanalytics.iglu.client.ClientError.{ResolutionError, ValidationError} import com.snowplowanalytics.snowplow.badrows._ +import com.snowplowanalytics.snowplow.badrows.FailureDetails +import com.snowplowanalytics.snowplow.enrich.common.enrichments.Failure import com.snowplowanalytics.snowplow.enrich.common.outputs.EnrichedEvent import com.snowplowanalytics.snowplow.enrich.common.SpecHelpers import com.snowplowanalytics.snowplow.enrich.common.adapters.RawEvent @@ -42,7 +46,12 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect val processor = Processor("unit tests SCE", "v42") val enriched = new EnrichedEvent() + val unstructFieldName = "unstruct" + val contextsFieldName = "contexts" + val derivedContextsFieldName = "derived_contexts" + val notJson = "foo" + val jsonNotJson = notJson.asJson // Just jsonized version of the string val notIglu = """{"foo":"bar"}""" val unstructSchema = SchemaKey( @@ -94,11 +103,12 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect "emailAddress2": "foo2@bar.org" } }""" + val invalidEmailSentData = s"""{ + "emailAddress": "hello@world.com" + }""" val invalidEmailSent = s"""{ "schema": "${emailSentSchema.toSchemaUri}", - "data": { - "emailAddress": "hello@world.com" - } + "data": $invalidEmailSentData }""" val supersedingExample1 = s"""{ @@ -128,104 +138,165 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect "userId": "20d631b8-7837-49df-a73e-6da73154e6fd" } }""" + val noSchemaData = "{}" val noSchema = - """{"schema":"iglu:com.snowplowanalytics.snowplow/foo/jsonschema/1-0-0", "data": {}}""" + s"""{"schema":"iglu:com.snowplowanalytics.snowplow/foo/jsonschema/1-0-0", "data": $noSchemaData}""" "extractAndValidateUnstructEvent" should { "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" >> { + "return a FailureDetails.SchemaViolation.NotJson if unstruct_event does not contain a properly formatted JSON string" >> { val input = new EnrichedEvent input.setUnstruct_event(notJson) 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( + Failure.SchemaViolation(_: FailureDetails.SchemaViolation.NotJson, `unstructFieldName`, `jsonNotJson`), + _ + ), + None + ) => + ok + case other => ko(s"[$other] is not an error with NotJson") + } } - "return a SchemaViolation.NotIglu if unstruct_event contains a properly formatted JSON string that is not self-describing" >> { + "return a FailureDetails.SchemaViolation.NotIglu if unstruct_event contains a properly formatted JSON string that is not self-describing" >> { + val json = notIglu.toJson val input = new EnrichedEvent input.setUnstruct_event(notIglu) 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(Failure.SchemaViolation(_: FailureDetails.SchemaViolation.NotIglu, `unstructFieldName`, `json`), _), + None + ) => + ok + case other => ko(s"[$other] is not an error with NotIglu") + } } - "return a SchemaViolation.CriterionMismatch if unstruct_event contains a self-describing JSON but not with the expected schema for unstructured events" >> { + "return a FailureDetails.SchemaViolation.CriterionMismatch if unstruct_event contains a self-describing JSON but not with the expected schema for unstructured events" >> { + val json = noSchemaData.toJson val input = new EnrichedEvent input.setUnstruct_event(noSchema) 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( + Failure.SchemaViolation(_: FailureDetails.SchemaViolation.CriterionMismatch, `unstructFieldName`, `json`), + _ + ), + None + ) => + ok + case other => ko(s"[$other] is not an error with CriterionMismatch") + } } - "return a SchemaViolation.NotJson if the JSON in .data is not a JSON" >> { + "return a FailureDetails.SchemaViolation.NotJson if the JSON in .data is not a JSON" >> { val input = new EnrichedEvent - input.setUnstruct_event(buildUnstruct(notJson)) + val ue = buildUnstruct(notJson) + val ueJson = ue.asJson + input.setUnstruct_event(ue) 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( + Failure.SchemaViolation(_: FailureDetails.SchemaViolation.NotJson, `unstructFieldName`, `ueJson`), + _ + ), + None + ) => + ok + case other => ko(s"[$other] is not an error with NotJson") + } } - "return a SchemaViolation.IgluError containing a ValidationError if the JSON in .data is not self-describing" >> { + "return a FailureDetails.SchemaViolation.IgluError containing a ValidationError if the JSON in .data is not self-describing" >> { val input = new EnrichedEvent - input.setUnstruct_event(buildUnstruct(notIglu)) + val ue = buildUnstruct(notIglu) + val ueJson = notIglu.toJson + input.setUnstruct_event(ue) IgluUtils .extractAndValidateUnstructEvent(input, SpecHelpers.client, SpecHelpers.registryLookup) - .map(_ must beInvalid.like { - case FailureDetails.SchemaViolation.IgluError(_, ValidationError(_, _)) => ok - case ie: FailureDetails.SchemaViolation.IgluError => - ko(s"IgluError [$ie] is not ValidationError") - case err => ko(s"[$err] is not IgluError") - }) + .value + .map { + case Ior.Both(NonEmptyList(Failure.SchemaViolation(FailureDetails.SchemaViolation.IgluError(_, _: ValidationError), + `unstructFieldName`, + `ueJson` + ), + _ + ), + None + ) => + ok + case other => ko(s"[$other] is not expected one") + } } - "return a SchemaViolation.IgluError containing a ValidationError if the JSON in .data is not a valid SDJ" >> { + "return a FailureDetails.SchemaViolation.IgluError containing a ValidationError if the JSON in .data is not a valid SDJ" >> { val input = new EnrichedEvent + val json = invalidEmailSentData.toJson input.setUnstruct_event(buildUnstruct(invalidEmailSent)) IgluUtils .extractAndValidateUnstructEvent(input, SpecHelpers.client, SpecHelpers.registryLookup) - .map(_ must beInvalid.like { - case FailureDetails.SchemaViolation.IgluError(_, ValidationError(_, _)) => ok - case ie: FailureDetails.SchemaViolation.IgluError => - ko(s"IgluError [$ie] is not ValidationError") - case err => ko(s"[$err] is not IgluError") - }) + .value + .map { + case Ior.Both(NonEmptyList(Failure.SchemaViolation(FailureDetails.SchemaViolation.IgluError(_, _: ValidationError), + `unstructFieldName`, + `json` + ), + _ + ), + None + ) => + ok + case other => ko(s"[$other] is not expected one") + } } - "return a SchemaViolation.IgluError containing a ResolutionError if the schema of the SDJ in .data can't be resolved" >> { + "return a FailureDetails.SchemaViolation.IgluError containing a ResolutionError if the schema of the SDJ in .data can't be resolved" >> { val input = new EnrichedEvent + val json = noSchemaData.toJson input.setUnstruct_event(buildUnstruct(noSchema)) 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( + Failure.SchemaViolation(FailureDetails.SchemaViolation.IgluError(_, _: ResolutionError), `unstructFieldName`, `json`), + _ + ), + None + ) => + ok + case other => ko(s"[$other] is not expected one") + } } "return the extracted unstructured event if .data is a valid SDJ" >> { @@ -234,14 +305,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 +327,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,118 +360,180 @@ 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" >> { + "return a FailureDetails.SchemaViolation.NotJson if .contexts does not contain a properly formatted JSON string" >> { val input = new EnrichedEvent input.setContexts(notJson) 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(Failure.SchemaViolation(_: FailureDetails.SchemaViolation.NotJson, `contextsFieldName`, `jsonNotJson`), Nil), + Nil + ) => + ok + case other => ko(s"[$other] is not an error with NotJson") + } } - "return a SchemaViolation.NotIglu if .contexts contains a properly formatted JSON string that is not self-describing" >> { + "return a FailureDetails.SchemaViolation.NotIglu if .contexts contains a properly formatted JSON string that is not self-describing" >> { val input = new EnrichedEvent + val json = notIglu.toJson input.setContexts(notIglu) 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(Failure.SchemaViolation(_: FailureDetails.SchemaViolation.NotIglu, `contextsFieldName`, `json`), Nil), + Nil + ) => + ok + case other => ko(s"[$other] is not an error with NotIglu") + } } - "return a SchemaViolation.CriterionMismatch if .contexts contains a self-describing JSON but not with the right schema" >> { + "return a FailureDetails.SchemaViolation.CriterionMismatch if .contexts contains a self-describing JSON but not with the right schema" >> { val input = new EnrichedEvent + val json = noSchemaData.toJson input.setContexts(noSchema) 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( + Failure.SchemaViolation(_: FailureDetails.SchemaViolation.CriterionMismatch, `contextsFieldName`, `json`), + Nil + ), + Nil + ) => + ok + case other => ko(s"[$other] is not an error with CriterionMismatch") + } } - "return a SchemaViolation.IgluError containing a ValidationError if .data does not contain an array of JSON objects" >> { + "return a FailureDetails.SchemaViolation.IgluError containing a ValidationError if .data does not contain an array of JSON objects" >> { val input = new EnrichedEvent val notArrayContexts = s"""{"schema": "${inputContextsSchema.toSchemaUri}", "data": ${emailSent1}}""" + val json = emailSent1.toJson input.setContexts(notArrayContexts) IgluUtils .extractAndValidateInputContexts(input, SpecHelpers.client, SpecHelpers.registryLookup) - .map(_ must beInvalid.like { - case NonEmptyList(FailureDetails.SchemaViolation.IgluError(_, ValidationError(_, _)), Nil) => + .value + .map { + case Ior.Both( + NonEmptyList( + Failure.SchemaViolation(FailureDetails.SchemaViolation.IgluError(_, _: ValidationError), `contextsFieldName`, `json`), + Nil + ), + Nil + ) => ok - case NonEmptyList(ie: FailureDetails.SchemaViolation.IgluError, Nil) => - ko(s"IgluError [$ie] is not ValidationError") - case err => ko(s"[$err] is not one IgluError") - }) + case other => ko(s"[$other] is not expected one") + } } - "return a SchemaViolation.IgluError containing a ValidationError if .data contains one invalid context" >> { + "return a FailureDetails.SchemaViolation.IgluError containing a ValidationError if .data contains one invalid context" >> { val input = new EnrichedEvent + val json = invalidEmailSentData.toJson input.setContexts(buildInputContexts(List(invalidEmailSent))) IgluUtils .extractAndValidateInputContexts(input, SpecHelpers.client, SpecHelpers.registryLookup) - .map(_ must beInvalid.like { - case NonEmptyList(FailureDetails.SchemaViolation.IgluError(_, ValidationError(_, _)), Nil) => + .value + .map { + case Ior.Both( + NonEmptyList( + Failure.SchemaViolation(FailureDetails.SchemaViolation.IgluError(_, _: ValidationError), `contextsFieldName`, `json`), + Nil + ), + Nil + ) => ok - case NonEmptyList(ie: FailureDetails.SchemaViolation.IgluError, Nil) => - ko(s"IgluError [$ie] is not ValidationError") - case err => ko(s"[$err] is not one IgluError") - }) + case other => ko(s"[$other] is not expected one") + } } - "return a SchemaViolation.IgluError containing a ResolutionError if .data contains one context whose schema can't be resolved" >> { + "return a FailureDetails.SchemaViolation.IgluError containing a ResolutionError if .data contains one context whose schema can't be resolved" >> { val input = new EnrichedEvent + val json = noSchemaData.toJson input.setContexts(buildInputContexts(List(noSchema))) IgluUtils .extractAndValidateInputContexts(input, SpecHelpers.client, SpecHelpers.registryLookup) - .map(_ must beInvalid.like { - case NonEmptyList(FailureDetails.SchemaViolation.IgluError(_, ResolutionError(_)), Nil) => + .value + .map { + case Ior.Both( + NonEmptyList( + Failure.SchemaViolation(FailureDetails.SchemaViolation.IgluError(_, _: ResolutionError), `contextsFieldName`, `json`), + Nil + ), + Nil + ) => ok - case NonEmptyList(ie: FailureDetails.SchemaViolation.IgluError, Nil) => - ko(s"IgluError [$ie] is not ResolutionError") - case err => ko(s"[$err] is not one IgluError") - }) + case other => ko(s"[$other] is not expected one") + } } "return 2 expected failures for 2 invalid contexts" >> { val input = new EnrichedEvent + val invalidEmailSentJson = invalidEmailSentData.toJson + val noSchemaJson = noSchemaData.toJson input.setContexts(buildInputContexts(List(invalidEmailSent, noSchema))) 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( + Failure.SchemaViolation(FailureDetails.SchemaViolation.IgluError(_, _: ValidationError), + `contextsFieldName`, + `invalidEmailSentJson` + ), + List( + Failure.SchemaViolation(FailureDetails.SchemaViolation.IgluError(_, _: ResolutionError), + `contextsFieldName`, + `noSchemaJson` + ) + ) + ), + 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") + } } - "return an expected failure if one context is valid and the other invalid" >> { + "return an expected failure and an expected SDJ if one context is invalid and one is valid" >> { val input = new EnrichedEvent + val noSchemaJson = noSchemaData.toJson 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( + Failure.SchemaViolation(_: FailureDetails.SchemaViolation.IgluError, `contextsFieldName`, `noSchemaJson`), + Nil + ), + List(extract) + ) if extract.sdj.schema == emailSentSchema => + ok + case other => ko(s"[$other] is not one IgluError and one SDJ with schema $emailSentSchema") + } } "return the extracted SDJs for 2 valid input contexts" >> { @@ -402,12 +542,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 +557,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,30 +572,44 @@ 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 one expected SchemaViolation for one invalid context" >> { + val json = invalidEmailSentData.toJson val contexts = List( SpecHelpers.jsonStringToSDJ(invalidEmailSent).right.get ) IgluUtils .validateEnrichmentsContexts(SpecHelpers.client, contexts, SpecHelpers.registryLookup) - .map(_.toEither must beLeft.like { - case NonEmptyList(FailureDetails.SchemaViolation.IgluError(_, ValidationError(_, _)), Nil) => ok + .value + .map { + case Ior.Both( + NonEmptyList(Failure.SchemaViolation(FailureDetails.SchemaViolation.IgluError(_, _: ValidationError), + `derivedContextsFieldName`, + `json` + ), + Nil + ), + Nil + ) => + ok case other => ko(s"[$other] is not one ValidationError") - }) + } } "return two expected SchemaViolation for two invalid contexts" >> { + val invalidEmailSentJson = invalidEmailSentData.toJson + val noSchemaJson = noSchemaData.toJson val contexts = List( SpecHelpers.jsonStringToSDJ(invalidEmailSent).right.get, SpecHelpers.jsonStringToSDJ(noSchema).right.get @@ -461,16 +617,29 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect IgluUtils .validateEnrichmentsContexts(SpecHelpers.client, contexts, SpecHelpers.registryLookup) - .map(_.toEither must beLeft.like { - case NonEmptyList(FailureDetails.SchemaViolation.IgluError(_, ValidationError(_, _)), - List(FailureDetails.SchemaViolation.IgluError(_, ResolutionError(_))) + .value + .map { + case Ior.Both(NonEmptyList( + Failure.SchemaViolation(FailureDetails.SchemaViolation.IgluError(_, _: ValidationError), + `derivedContextsFieldName`, + `invalidEmailSentJson` + ), + List( + Failure.SchemaViolation(FailureDetails.SchemaViolation.IgluError(_, _: ResolutionError), + `derivedContextsFieldName`, + `noSchemaJson` + ) + ) + ), + Nil ) => ok case other => ko(s"[$other] is not one ValidationError and one ResolutionError") - }) + } } "return one expected SchemaViolation for one invalid context and one valid" >> { + val invalidEmailSentJson = invalidEmailSentData.toJson val contexts = List( SpecHelpers.jsonStringToSDJ(invalidEmailSent).right.get, SpecHelpers.jsonStringToSDJ(emailSent1).right.get @@ -478,13 +647,23 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect IgluUtils .validateEnrichmentsContexts(SpecHelpers.client, contexts, SpecHelpers.registryLookup) - .map(_.toEither must beLeft.like { - case NonEmptyList(FailureDetails.SchemaViolation.IgluError(_, ValidationError(_, _)), Nil) => ok - case other => ko(s"[$other] is not one ValidationError") - }) + .value + .map { + case Ior.Both(NonEmptyList( + Failure.SchemaViolation(FailureDetails.SchemaViolation.IgluError(_, _: ValidationError), + `derivedContextsFieldName`, + `invalidEmailSentJson` + ), + Nil + ), + List(sdj) + ) if sdj.schema == emailSentSchema => + ok + case other => ko(s"[$other] is not one ValidationError and one SDJ with schema $emailSentSchema") + } } - "not return any error for two valid contexts" >> { + "return 2 valid contexts" >> { val contexts = List( SpecHelpers.jsonStringToSDJ(emailSent1).right.get, SpecHelpers.jsonStringToSDJ(emailSent2).right.get @@ -492,7 +671,11 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect IgluUtils .validateEnrichmentsContexts(SpecHelpers.client, contexts, SpecHelpers.registryLookup) - .map(_.toEither must beRight) + .value + .map { + case Ior.Right(List(sdj1, sdj2)) if sdj1.schema == emailSentSchema && sdj2.schema == emailSentSchema => ok + case other => ko(s"[$other] doesn't contain 2 valid contexts with schema $emailSentSchema") + } } } @@ -507,10 +690,18 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect SpecHelpers.client, SpecHelpers.registryLookup ) - .map(_.toEither must beLeft.like { - case errors if errors.size == 1 => ok - case other => ko(s"[$other] is not one SchemaViolation") - }) + .value + .map { + case Ior.Both( + NonEmptyList( + _: Failure.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 one invalid context" >> { @@ -523,10 +714,18 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect SpecHelpers.client, SpecHelpers.registryLookup ) - .map(_.toEither must beLeft.like { - case errors if errors.size == 1 => ok - case other => ko(s"[$other] is not one SchemaViolation") - }) + .value + .map { + case Ior.Both( + NonEmptyList( + _: Failure.SchemaViolation, + Nil + ), + IgluUtils.EventExtractResult(Nil, None, Nil) + ) => + ok + case other => ko(s"[$other] isn't an error with SchemaViolation") + } } "return two SchemaViolation if the input event contains an invalid unstructured event and one invalid context" >> { @@ -540,10 +739,18 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect SpecHelpers.client, SpecHelpers.registryLookup ) - .map(_.toEither must beLeft.like { - case errors if errors.size == 2 => ok - case other => ko(s"[$other] is not two SchemaViolation") - }) + .value + .map { + case Ior.Both( + NonEmptyList( + _: Failure.SchemaViolation, + List(_: Failure.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" >> { @@ -557,17 +764,80 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect SpecHelpers.client, SpecHelpers.registryLookup ) - .map(_.toEither must beRight.like { - case IgluUtils.EventExtractResult(contexts, Some(unstructEvent), validationInfos) + .value + .map { + case Ior.Right(IgluUtils.EventExtractResult(contexts, Some(unstructEvent), validationInfos)) if contexts.size == 2 && validationInfos.isEmpty && (unstructEvent :: contexts).forall(_.schema == emailSentSchema) => ok - case res => + case other => ko( - s"[$res] is not a list with two extracted contexts and an option with the extracted unstructured event" + s"[$other] doesn't contain the two contexts and the unstructured event" ) - }) + } + } + + "return the SchemaViolation of the invalid context in the Left and the extracted unstructured event in the Right" >> { + val input = new EnrichedEvent + val invalidEmailSentJson = invalidEmailSentData.toJson + input.setUnstruct_event(buildUnstruct(emailSent1)) + input.setContexts(buildInputContexts(List(invalidEmailSent))) + + IgluUtils + .extractAndValidateInputJsons( + input, + SpecHelpers.client, + SpecHelpers.registryLookup + ) + .value + .map { + case Ior.Both( + NonEmptyList(Failure.SchemaViolation(FailureDetails.SchemaViolation.IgluError(_, _: ValidationError), + `contextsFieldName`, + `invalidEmailSentJson` + ), + _ + ), + extract + ) if extract.contexts.isEmpty && extract.unstructEvent.isDefined && extract.unstructEvent.get.schema == emailSentSchema => + ok + case other => + ko( + s"[$other] isn't one ValidationError and an unstructured event with schema $emailSentSchema" + ) + } + } + + "return the SchemaViolation of the invalid unstructured event in the Left and the valid context in the Right" >> { + val input = new EnrichedEvent + val invalidEmailSentJson = invalidEmailSentData.toJson + input.setUnstruct_event(buildUnstruct(invalidEmailSent)) + input.setContexts(buildInputContexts(List(emailSent1))) + + IgluUtils + .extractAndValidateInputJsons( + input, + SpecHelpers.client, + SpecHelpers.registryLookup + ) + .value + .map { + case Ior.Both( + NonEmptyList(Failure.SchemaViolation(FailureDetails.SchemaViolation.IgluError(_, _: ValidationError), + `unstructFieldName`, + `invalidEmailSentJson` + ), + _ + ), + extract + ) if extract.contexts.size == 1 && extract.contexts.head.schema == emailSentSchema && extract.unstructEvent.isEmpty => + ok + case other => + ko( + s"[$other] isn't one ValidationError and one context with schema $emailSentSchema" + ) + } } "return the extracted unstructured event and the extracted input contexts when schema is superseded by another schema" >> { @@ -575,12 +845,11 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect input.setUnstruct_event(buildUnstruct(supersedingExample1)) input.setContexts(buildInputContexts(List(supersedingExample1, supersedingExample2))) - val expectedValidationInfoContext = parse( + val expectedValidationInfoContext = """ { | "originalSchema" : "iglu:com.acme/superseding_example/jsonschema/1-0-0", | "validatedWith" : "1-0-1" - |}""".stripMargin - ).toOption.get + |}""".stripMargin.toJson IgluUtils .extractAndValidateInputJsons( @@ -588,19 +857,20 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect SpecHelpers.client, SpecHelpers.registryLookup ) - .map(_.toEither must beRight.like { - case IgluUtils.EventExtractResult(contexts, Some(unstructEvent), List(validationInfo)) + .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 res => + case other => ko( - s"[$res] is not a list with two extracted contexts and an option with the extracted unstructured event" + s"[$other] doesn't contain the two contexts and the unstructured event with the superseded schema" ) - }) + } } } @@ -609,4 +879,8 @@ class IgluUtilsSpec extends Specification with ValidatedMatchers with CatsEffect def buildInputContexts(sdjs: List[String] = List.empty[String]) = s"""{"schema": "${inputContextsSchema.toSchemaUri}", "data": [${sdjs.mkString(",")}]}""" + + implicit class StringToJson(str: String) { + def toJson: Json = parse(str).toOption.get + } } 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 1ec672f8f..06d8eea5b 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,7 +263,7 @@ class ValidateUuidSpec extends Specification with DataTables with ScalaCheck { def e2 = prop { (str: String) => ConversionUtils.validateUuid(FieldName, str) must beLeft( - AtomicError.ParseError("Not a valid UUID", FieldName) + AtomicError.ParseError("Not a valid UUID", FieldName, Option(str)) ) } } @@ -281,7 +281,7 @@ class ValidateIntegerSpec extends Specification { def e2 = { val str = "abc" ConversionUtils.validateInteger(FieldName, str) must beLeft( - AtomicError.ParseError("Not a valid integer", FieldName) + AtomicError.ParseError("Not a valid integer", FieldName, Some(str)) ) } } @@ -312,19 +312,19 @@ class StringToDoubleLikeSpec extends Specification with DataTables { """ val FieldName = "val" - def err: AtomicError = - AtomicError.ParseError("Cannot be converted to Double-like", FieldName) + def err(value: String): AtomicError = + AtomicError.ParseError("Cannot be converted to Double-like", FieldName, Option(value)) def e1 = "SPEC NAME" || "INPUT STR" | "EXPECTED" | - "Empty string" !! "" ! err | - "Number with commas" !! "19,999.99" ! err | - "Hexadecimal number" !! "0x54" ! err | - "Bad sci. notation" !! "-7.51E^9" ! err | - "German number" !! "1.000,3932" ! err | - "NaN" !! "NaN" ! err | - "English string" !! "hi & bye" ! err | - "Vietnamese name" !! "Trịnh Công Sơn" ! err |> { (_, str, expected) => + "Empty string" !! "" ! err("") | + "Number with commas" !! "19,999.99" ! err("19,999.99") | + "Hexadecimal number" !! "0x54" ! err("0x54") | + "Bad sci. notation" !! "-7.51E^9" ! err("-7.51E^9") | + "German number" !! "1.000,3932" ! err("1.000,3932") | + "NaN" !! "NaN" ! err("NaN") | + "English string" !! "hi & bye" ! err("hi & bye") | + "Vietnamese name" !! "Trịnh Công Sơn" ! err("Trịnh Công Sơn") |> { (_, str, expected) => ConversionUtils.stringToDoubleLike(FieldName, str) must beLeft(expected) } @@ -388,18 +388,18 @@ class StringToBooleanLikeJByteSpec extends Specification with DataTables { """ val FieldName = "val" - def err: AtomicError = - AtomicError.ParseError("Cannot be converted to Boolean-like java.lang.Byte", FieldName) + def err(value: String): AtomicError = + AtomicError.ParseError("Cannot be converted to Boolean-like java.lang.Byte", FieldName, Option(value)) def e1 = "SPEC NAME" || "INPUT STR" | "EXPECTED" | - "Empty string" !! "" ! err | - "Small number" !! "2" ! err | - "Negative number" !! "-1" ! err | - "Floating point number" !! "0.0" ! err | - "Large number" !! "19,999.99" ! err | - "Text #1" !! "a" ! err | - "Text #2" !! "0x54" ! err |> { (_, str, expected) => + "Empty string" !! "" ! err("") | + "Small number" !! "2" ! err("2") | + "Negative number" !! "-1" ! err("-1") | + "Floating point number" !! "0.0" ! err("0.0") | + "Large number" !! "19,999.99" ! err("19,999.99") | + "Text #1" !! "a" ! err("a") | + "Text #2" !! "0x54" ! err("0x54") |> { (_, str, expected) => ConversionUtils.stringToBooleanLikeJByte(FieldName, str) must beLeft(expected) } diff --git a/modules/kafka/src/main/resources/application.conf b/modules/kafka/src/main/resources/application.conf index d5d0bb10c..e1d6ee9b5 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 71d8178ce..c2c947bd6 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, 8), @@ -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, 8), None, diff --git a/modules/kinesis/src/it/resources/enrich/enrich-localstack-statsd.hocon b/modules/kinesis/src/it/resources/enrich/enrich-localstack-statsd.hocon new file mode 100644 index 000000000..2a3e4b3aa --- /dev/null +++ b/modules/kinesis/src/it/resources/enrich/enrich-localstack-statsd.hocon @@ -0,0 +1,60 @@ +{ + "license": { + "accept": true + } + + "input": { + "type": "Kinesis" + "appName": ${APP_NAME} + "streamName": ${STREAM_RAW} + "region": ${REGION} + "customEndpoint": ${LOCALSTACK_ENDPOINT} + "dynamodbCustomEndpoint": ${LOCALSTACK_ENDPOINT} + "cloudwatchCustomEndpoint": ${LOCALSTACK_ENDPOINT} + } + + "output": { + "good": { + "type": "Kinesis" + "streamName": ${STREAM_ENRICHED} + "region": ${REGION} + "customEndpoint": ${LOCALSTACK_ENDPOINT} + } + + "bad": { + "type": "Kinesis" + "streamName": ${STREAM_BAD} + "region": ${REGION} + "customEndpoint": ${LOCALSTACK_ENDPOINT} + } + + "incomplete": { + "type": "Kinesis" + "streamName": ${STREAM_INCOMPLETE} + "region": ${REGION} + "customEndpoint": ${LOCALSTACK_ENDPOINT} + } + } + + "monitoring": { + "metrics": { + "stdout": { + "period": "10 seconds" + } + "statsd": { + "hostname": statsd + "port": 8125 + "period": "10 seconds" + "tags": { + "env": test + } + "prefix": "snowplow.enrich." + } + "cloudwatch": true + } + } + + "telemetry": { + "disable": true + } +} diff --git a/modules/kinesis/src/it/resources/enrich/enrich-localstack.hocon b/modules/kinesis/src/it/resources/enrich/enrich-localstack.hocon index 569eca940..bf441a8f0 100644 --- a/modules/kinesis/src/it/resources/enrich/enrich-localstack.hocon +++ b/modules/kinesis/src/it/resources/enrich/enrich-localstack.hocon @@ -27,6 +27,13 @@ "region": ${REGION} "customEndpoint": ${LOCALSTACK_ENDPOINT} } + + "incomplete": { + "type": "Kinesis" + "streamName": ${STREAM_INCOMPLETE} + "region": ${REGION} + "customEndpoint": ${LOCALSTACK_ENDPOINT} + } } "monitoring": { diff --git a/modules/kinesis/src/it/scala/com/snowplowanalytics/snowplow/enrich/kinesis/Containers.scala b/modules/kinesis/src/it/scala/com/snowplowanalytics/snowplow/enrich/kinesis/Containers.scala index 55b9ec6d7..adfd10428 100644 --- a/modules/kinesis/src/it/scala/com/snowplowanalytics/snowplow/enrich/kinesis/Containers.scala +++ b/modules/kinesis/src/it/scala/com/snowplowanalytics/snowplow/enrich/kinesis/Containers.scala @@ -19,11 +19,13 @@ import org.slf4j.LoggerFactory import retry.syntax.all._ import retry.RetryPolicies +import cats.implicits._ + import cats.effect.{IO, Resource} import cats.effect.testing.specs2.CatsEffect -import org.testcontainers.containers.{BindMode, GenericContainer => JGenericContainer, Network} +import org.testcontainers.containers.{BindMode, Network} import org.testcontainers.containers.wait.strategy.Wait import org.testcontainers.containers.output.Slf4jLogConsumer @@ -34,47 +36,66 @@ import com.snowplowanalytics.snowplow.enrich.kinesis.generated.BuildInfo object Containers extends CatsEffect { - private val network = Network.newNetwork() + object Images { + case class DockerImage(image: String, tag: String) { + def toStr = s"$image:$tag" + } + val Localstack = DockerImage("localstack/localstack-light", "1.2.0") + val Enrich = DockerImage("snowplow/snowplow-enrich-kinesis", s"${BuildInfo.version}-distroless") + val MySQL = DockerImage("mysql", "8.0.31") + val HTTP = DockerImage("nginx", "1.23.2") + val Statsd = DockerImage("dblworks/statsd", "v0.10.2") // the official statsd/statsd size is monstrous + } - private val localstackPort = 4566 - private val localstackAlias = "localstack" + case class Localstack( + container: GenericContainer, + alias: String, + internalPort: Int, + mappedPort: Int + ) - val localstack = { + private val network = Network.newNetwork() + + def localstack: Resource[IO, Localstack] = Resource.make { + val port = 4566 val container = GenericContainer( - dockerImage = "localstack/localstack-light:1.2.0", - fileSystemBind = Seq( - GenericContainer.FileSystemBind( - "modules/kinesis/src/it/resources/localstack", - "/docker-entrypoint-initaws.d", - BindMode.READ_ONLY - ) - ), + dockerImage = Images.Localstack.toStr, env = Map( "AWS_ACCESS_KEY_ID" -> "foo", "AWS_SECRET_ACCESS_KEY" -> "bar" ), waitStrategy = Wait.forLogMessage(".*Ready.*", 1), - exposedPorts = Seq(localstackPort) + exposedPorts = Seq(port) ) container.underlyingUnsafeContainer.withNetwork(network) - container.underlyingUnsafeContainer.withNetworkAliases(localstackAlias) - container.container + val alias = "localstack" + container.underlyingUnsafeContainer.withNetworkAliases(alias) + + IO.blocking(container.start()) *> + IO( + Localstack( + container, + alias, + port, + container.container.getMappedPort(port) + ) + ) + } { + l => IO.blocking(l.container.stop()) } - def localstackMappedPort = localstack.getMappedPort(localstackPort) - def enrich( + localstack: Localstack, configPath: String, testName: String, - needsLocalstack: Boolean, enrichments: List[Enrichment], uuid: String = UUID.randomUUID().toString, waitLogMessage: String = "Running Enrich" - ): Resource[IO, JGenericContainer[_]] = { + ): Resource[IO, GenericContainer] = { val streams = KinesisConfig.getStreams(uuid) val container = GenericContainer( - dockerImage = s"snowplow/snowplow-enrich-kinesis:${BuildInfo.version}-distroless", + dockerImage = Images.Enrich.toStr, env = Map( "AWS_REGION" -> KinesisConfig.region, "AWS_ACCESS_KEY_ID" -> "foo", @@ -86,7 +107,8 @@ object Containers extends CatsEffect { "STREAM_RAW" -> streams.raw, "STREAM_ENRICHED" -> streams.enriched, "STREAM_BAD" -> streams.bad, - "LOCALSTACK_ENDPOINT" -> s"http://$localstackAlias:$localstackPort" + "STREAM_INCOMPLETE" -> streams.incomplete, + "LOCALSTACK_ENDPOINT" -> s"http://${localstack.alias}:${localstack.internalPort}" ), fileSystemBind = Seq( GenericContainer.FileSystemBind( @@ -112,16 +134,16 @@ object Containers extends CatsEffect { ) container.container.withNetwork(network) Resource.make ( - IO(startLocalstack(needsLocalstack, KinesisConfig.region, streams)) >> - IO(startContainerWithLogs(container.container, testName)) + createStreams(localstack, KinesisConfig.region, streams) *> + startContainerWithLogs(container, testName) )( - e => IO(e.stop()) + e => IO.blocking(e.stop()) ) } - def mysqlServer: Resource[IO, JGenericContainer[_]] = Resource.make { + def mysqlServer: Resource[IO, GenericContainer] = Resource.make { val container = GenericContainer( - dockerImage = "mysql:8.0.31", + dockerImage = Images.MySQL.toStr, fileSystemBind = Seq( GenericContainer.FileSystemBind( "modules/kinesis/src/it/resources/mysql", @@ -139,14 +161,14 @@ object Containers extends CatsEffect { ) container.underlyingUnsafeContainer.withNetwork(network) container.underlyingUnsafeContainer.withNetworkAliases("mysql") - IO(container.start()) >> IO.pure(container.container) + IO(container.start()) *> IO.pure(container) } { c => IO(c.stop()) } - def httpServer: Resource[IO, JGenericContainer[_]] = Resource.make { + def httpServer: Resource[IO, GenericContainer] = Resource.make { val container = GenericContainer( - dockerImage = "nginx:1.23.2", + dockerImage = Images.HTTP.toStr, fileSystemBind = Seq( GenericContainer.FileSystemBind( "modules/kinesis/src/it/resources/nginx/default.conf", @@ -168,23 +190,32 @@ object Containers extends CatsEffect { ) container.underlyingUnsafeContainer.withNetwork(network) container.underlyingUnsafeContainer.withNetworkAliases("api") - IO(container.start()) >> IO.pure(container.container) + IO.blocking(container.start()) *> IO.pure(container) } { - c => IO(c.stop()) + c => IO.blocking(c.stop()) + } + + def statsdServer: Resource[IO, GenericContainer] = Resource.make { + val container = GenericContainer(Images.Statsd.toStr) + container.underlyingUnsafeContainer.withNetwork(network) + container.underlyingUnsafeContainer.withNetworkAliases("statsd") + container.underlyingUnsafeContainer.addExposedPort(8126) + IO.blocking(container.start()) *> IO.pure(container) + } { + c => IO.blocking(c.stop()) } private def startContainerWithLogs( - container: JGenericContainer[_], + container: GenericContainer, loggerName: String - ): JGenericContainer[_] = { + ): IO[GenericContainer] = { val logger = LoggerFactory.getLogger(loggerName) val logs = new Slf4jLogConsumer(logger) - container.start() - container.followOutput(logs) - container + IO.blocking(container.start()) *> + IO(container.container.followOutput(logs)).as(container) } - def waitUntilStopped(container: JGenericContainer[_]): IO[Boolean] = { + def waitUntilStopped(container: GenericContainer): IO[Boolean] = { val retryPolicy = RetryPolicies.limitRetriesByCumulativeDelay( 5.minutes, RetryPolicies.capDelay[IO]( @@ -193,50 +224,32 @@ object Containers extends CatsEffect { ) ) - IO(container.isRunning()).retryingOnFailures( + IO(container.container.isRunning()).retryingOnFailures( _ => IO.pure(false), retryPolicy, (_, _) => IO.unit ) } - // synchronized so that start() isn't called by several threads at the same time. - // start() is blocking. - // Calling start() on an already started container has no effect. - private def startLocalstack( - needsLocalstack: Boolean, - region: String, - streams: KinesisConfig.Streams - ): Unit = synchronized { - if(needsLocalstack) { - localstack.start() - createStreams( - localstack, - localstackPort, - region, - streams - ) - } else () - } - private def createStreams( - localstack: JGenericContainer[_], - port: Int, + localstack: Localstack, region: String, streams: KinesisConfig.Streams - ): Unit = - List(streams.raw, streams.enriched, streams.bad).foreach { stream => - localstack.execInContainer( - "aws", - s"--endpoint-url=http://127.0.0.1:$port", - "kinesis", - "create-stream", - "--stream-name", - stream, - "--shard-count", - "1", - "--region", - region + ): IO[Unit] = + List(streams.raw, streams.enriched, streams.bad, streams.incomplete).traverse_ { stream => + IO.blocking( + localstack.container.execInContainer( + "aws", + s"--endpoint-url=http://127.0.0.1:${localstack.internalPort}", + "kinesis", + "create-stream", + "--stream-name", + stream, + "--shard-count", + "1", + "--region", + region + ) ) } } diff --git a/modules/kinesis/src/it/scala/com/snowplowanalytics/snowplow/enrich/kinesis/DockerPull.scala b/modules/kinesis/src/it/scala/com/snowplowanalytics/snowplow/enrich/kinesis/DockerPull.scala new file mode 100644 index 000000000..d38b20428 --- /dev/null +++ b/modules/kinesis/src/it/scala/com/snowplowanalytics/snowplow/enrich/kinesis/DockerPull.scala @@ -0,0 +1,35 @@ +/* + * Copyright (c) 2024-present Snowplow Analytics Ltd. All rights reserved. + * + * This program is licensed to you under the Snowplow Community License Version 1.0, + * and you may not use this file except in compliance with the Snowplow Community License Version 1.0. + * You may obtain a copy of the Snowplow Community License Version 1.0 at https://docs.snowplow.io/community-license-1.0 + */ +package com.snowplowanalytics.snowplow.enrich.kinesis + +import com.github.dockerjava.core.DockerClientBuilder +import com.github.dockerjava.api.command.PullImageResultCallback +import com.github.dockerjava.api.model.PullResponseItem + +object DockerPull { + + /** + * A blocking operation that runs on main thread to pull container image before `CatsResource` is + * created. This operation is then not counted towards test timeout. + */ + def pull(image: String, tag: String): Unit = + DockerClientBuilder + .getInstance() + .build() + .pullImageCmd(image) + .withTag(tag) + .withPlatform("linux/amd64") + .exec(new PullImageResultCallback() { + override def onNext(item: PullResponseItem) = { + println(s"$image: ${item.getStatus()}") + super.onNext(item) + } + }) + .awaitCompletion() + .onComplete() +} diff --git a/modules/kinesis/src/it/scala/com/snowplowanalytics/snowplow/enrich/kinesis/EnrichKinesisSpec.scala b/modules/kinesis/src/it/scala/com/snowplowanalytics/snowplow/enrich/kinesis/EnrichKinesisSpec.scala index 67fca4aca..eb1ce325e 100644 --- a/modules/kinesis/src/it/scala/com/snowplowanalytics/snowplow/enrich/kinesis/EnrichKinesisSpec.scala +++ b/modules/kinesis/src/it/scala/com/snowplowanalytics/snowplow/enrich/kinesis/EnrichKinesisSpec.scala @@ -15,34 +15,46 @@ import java.util.UUID import scala.concurrent.duration._ import cats.effect.IO +import cats.effect.kernel.Resource -import cats.effect.testing.specs2.CatsEffect +import cats.effect.testing.specs2.CatsResource -import org.specs2.mutable.Specification -import org.specs2.specification.AfterAll +import org.specs2.mutable.SpecificationLike +import org.specs2.specification.BeforeAll import com.snowplowanalytics.snowplow.enrich.kinesis.enrichments._ + import com.snowplowanalytics.snowplow.enrich.common.fs2.test.CollectorPayloadGen -class EnrichKinesisSpec extends Specification with AfterAll with CatsEffect { +import com.snowplowanalytics.snowplow.enrich.kinesis.Containers.Localstack + +class EnrichKinesisSpec extends CatsResource[IO, Localstack] with SpecificationLike with BeforeAll { override protected val Timeout = 10.minutes - def afterAll: Unit = Containers.localstack.stop() + override def beforeAll(): Unit = { + DockerPull.pull(Containers.Images.Localstack.image, Containers.Images.Localstack.tag) + DockerPull.pull(Containers.Images.MySQL.image, Containers.Images.MySQL.tag) + DockerPull.pull(Containers.Images.HTTP.image, Containers.Images.HTTP.tag) + DockerPull.pull(Containers.Images.Statsd.image, Containers.Images.Statsd.tag) + super.beforeAll() + } + + override val resource: Resource[IO, Localstack] = Containers.localstack "enrich-kinesis" should { - "be able to parse the minimal config" in { + "be able to parse the minimal config" in withResource { localstack => Containers.enrich( + localstack, configPath = "config/config.kinesis.minimal.hocon", testName = "minimal", - needsLocalstack = false, enrichments = Nil ).use { e => - IO(e.getLogs must contain("Running Enrich")) + IO(e.container.getLogs must contain("Running Enrich")) } } - "emit the correct number of enriched events and bad rows" in { + "emit the correct number of enriched events, bad rows and incomplete events" in withResource { localstack => import utils._ val testName = "count" @@ -52,13 +64,13 @@ class EnrichKinesisSpec extends Specification with AfterAll with CatsEffect { val resources = for { _ <- Containers.enrich( + localstack, configPath = "modules/kinesis/src/it/resources/enrich/enrich-localstack.hocon", - testName = "count", - needsLocalstack = true, + testName = testName, enrichments = Nil, uuid = uuid ) - enrichPipe <- mkEnrichPipe(Containers.localstackMappedPort, uuid) + enrichPipe <- mkEnrichPipe(localstack.mappedPort, uuid) } yield enrichPipe val input = CollectorPayloadGen.generate[IO](nbGood, nbBad) @@ -66,15 +78,61 @@ class EnrichKinesisSpec extends Specification with AfterAll with CatsEffect { resources.use { enrich => for { output <- enrich(input).compile.toList - (good, bad) = parseOutput(output, testName) + (good, bad, incomplete) = parseOutput(output, testName) + } yield { + good.size.toLong must beEqualTo(nbGood) + bad.size.toLong must beEqualTo(nbBad) + incomplete.size.toLong must beEqualTo(nbBad) + } + } + } + + "send the metrics to StatsD" in withResource { localstack => + import utils._ + + val testName = "statsd" + val nbGood = 100l + val nbBad = 10l + val uuid = UUID.randomUUID().toString + + val resources = for { + statsd <- Containers.statsdServer + statsdHost = statsd.container.getHost() + statsdAdminPort = statsd.container.getMappedPort(8126) + statsdAdmin <- mkStatsdAdmin(statsdHost, statsdAdminPort) + _ <- Containers.enrich( + localstack, + configPath = "modules/kinesis/src/it/resources/enrich/enrich-localstack-statsd.hocon", + testName = testName, + enrichments = Nil, + uuid = uuid + ) + enrichPipe <- mkEnrichPipe(localstack.mappedPort, uuid) + } yield (enrichPipe, statsdAdmin) + + val input = CollectorPayloadGen.generate[IO](nbGood, nbBad) + + resources.use { case (enrich, statsdAdmin) => + for { + output <- enrich(input).compile.toList + (good, bad, incomplete) = parseOutput(output, testName) + counters <- statsdAdmin.getCounters + gauges <- statsdAdmin.getGauges } yield { good.size.toLong must beEqualTo(nbGood) bad.size.toLong must beEqualTo(nbBad) + incomplete.size.toLong must beEqualTo(nbBad) + counters must contain(s"'snowplow.enrich.raw;env=test': ${nbGood + nbBad}") + counters must contain(s"'snowplow.enrich.good;env=test': $nbGood") + counters must contain(s"'snowplow.enrich.bad;env=test': $nbBad") + counters must contain(s"'snowplow.enrich.invalid_enriched;env=test': 0") + counters must contain(s"'snowplow.enrich.incomplete;env=test': $nbBad") + gauges must contain(s"'snowplow.enrich.latency;env=test': ") } } } - "run the enrichments and attach their context" in { + "run the enrichments and attach their context" in withResource { localstack => import utils._ val testName = "enrichments" @@ -94,13 +152,13 @@ class EnrichKinesisSpec extends Specification with AfterAll with CatsEffect { _ <- Containers.mysqlServer _ <- Containers.httpServer _ <- Containers.enrich( + localstack, configPath = "modules/kinesis/src/it/resources/enrich/enrich-localstack.hocon", - testName = "enrichments", - needsLocalstack = true, + testName = testName, enrichments = enrichments, uuid = uuid ) - enrichPipe <- mkEnrichPipe(Containers.localstackMappedPort, uuid) + enrichPipe <- mkEnrichPipe(localstack.mappedPort, uuid) } yield enrichPipe val input = CollectorPayloadGen.generate[IO](nbGood) @@ -108,32 +166,33 @@ class EnrichKinesisSpec extends Specification with AfterAll with CatsEffect { resources.use { enrich => for { output <- enrich(input).compile.toList - (good, bad) = parseOutput(output, testName) + (good, bad, incomplete) = parseOutput(output, testName) } yield { good.size.toLong must beEqualTo(nbGood) good.map { enriched => enriched.derived_contexts.data.map(_.schema) must containTheSameElementsAs(enrichmentsContexts) } bad.size.toLong must beEqualTo(0l) + incomplete.size.toLong must beEqualTo(0l) } } } - "shutdown when it receives a SIGTERM" in { + "shutdown when it receives a SIGTERM" in withResource { localstack => Containers.enrich( + localstack, configPath = "modules/kinesis/src/it/resources/enrich/enrich-localstack.hocon", testName = "stop", - needsLocalstack = true, enrichments = Nil, waitLogMessage = "enrich.metrics" ).use { enrich => for { _ <- IO(println("stop - Sending signal")) - _ <- IO(enrich.getDockerClient().killContainerCmd(enrich.getContainerId()).withSignal("TERM").exec()) + _ <- IO(enrich.container.getDockerClient().killContainerCmd(enrich.container.getContainerId()).withSignal("TERM").exec()) _ <- Containers.waitUntilStopped(enrich) } yield { - enrich.isRunning() must beFalse - enrich.getLogs() must contain("Enrich stopped") + enrich.container.isRunning() must beFalse + enrich.container.getLogs() must contain("Enrich stopped") } } } diff --git a/modules/kinesis/src/it/scala/com/snowplowanalytics/snowplow/enrich/kinesis/KinesisConfig.scala b/modules/kinesis/src/it/scala/com/snowplowanalytics/snowplow/enrich/kinesis/KinesisConfig.scala index cf65222a7..91d279af9 100644 --- a/modules/kinesis/src/it/scala/com/snowplowanalytics/snowplow/enrich/kinesis/KinesisConfig.scala +++ b/modules/kinesis/src/it/scala/com/snowplowanalytics/snowplow/enrich/kinesis/KinesisConfig.scala @@ -59,6 +59,9 @@ object KinesisConfig { Some(URI.create(getEndpoint(localstackPort))) ) + def incompleteStreamConfig(localstackPort: Int, streamName: String) = + enrichedStreamConfig(localstackPort, streamName) + val monitoring = Monitoring( None, MetricsReporters(None, None, false) @@ -67,8 +70,8 @@ object KinesisConfig { private def getEndpoint(localstackPort: Int): String = s"http://$endpoint:$localstackPort" - case class Streams(raw: String, enriched: String, bad: String) + case class Streams(raw: String, enriched: String, bad: String, incomplete: String) def getStreams(uuid: String): Streams = - Streams(s"raw-$uuid", s"enriched-$uuid", s"bad-1-$uuid") + Streams(s"raw-$uuid", s"enriched-$uuid", s"bad-1-$uuid", s"incomplete-$uuid") } diff --git a/modules/kinesis/src/it/scala/com/snowplowanalytics/snowplow/enrich/kinesis/utils.scala b/modules/kinesis/src/it/scala/com/snowplowanalytics/snowplow/enrich/kinesis/utils.scala index ad3b63508..dd5a19775 100644 --- a/modules/kinesis/src/it/scala/com/snowplowanalytics/snowplow/enrich/kinesis/utils.scala +++ b/modules/kinesis/src/it/scala/com/snowplowanalytics/snowplow/enrich/kinesis/utils.scala @@ -10,7 +10,11 @@ */ package com.snowplowanalytics.snowplow.enrich.kinesis +import java.io._ +import java.net._ + import scala.concurrent.duration._ +import scala.jdk.CollectionConverters._ import cats.data.Validated @@ -34,6 +38,7 @@ object utils extends CatsEffect { object OutputRow { final case class Good(event: Event) extends OutputRow final case class Bad(badRow: BadRow) extends OutputRow + final case class Incomplete(incomplete: Event) extends OutputRow } def mkEnrichPipe( @@ -46,9 +51,12 @@ object utils extends CatsEffect { } yield { val enriched = asGood(outputStream(KinesisConfig.enrichedStreamConfig(localstackPort, streams.enriched))) val bad = asBad(outputStream(KinesisConfig.badStreamConfig(localstackPort, streams.bad))) + val incomplete = asIncomplete(outputStream(KinesisConfig.incompleteStreamConfig(localstackPort, streams.incomplete))) collectorPayloads => - enriched.merge(bad) + enriched + .merge(bad) + .merge(incomplete) .interruptAfter(3.minutes) .concurrently(collectorPayloads.evalMap(bytes => rawSink(List(bytes)))) } @@ -81,13 +89,46 @@ object utils extends CatsEffect { } } - def parseOutput(output: List[OutputRow], testName: String): (List[Event], List[BadRow]) = { + private def asIncomplete(source: Stream[IO, Array[Byte]]): Stream[IO, OutputRow.Incomplete] = + source.map { bytes => + OutputRow.Incomplete { + val s = new String(bytes) + Event.parse(s) match { + case Validated.Valid(e) => e + case Validated.Invalid(e) => + throw new RuntimeException(s"Can't parse incomplete event [$s]. Error: $e") + } + } + } + + def parseOutput(output: List[OutputRow], testName: String): (List[Event], List[BadRow], List[Event]) = { val good = output.collect { case OutputRow.Good(e) => e} println(s"[$testName] Bad rows:") val bad = output.collect { case OutputRow.Bad(b) => println(s"[$testName] ${b.compact}") b } - (good, bad) + val incomplete = output.collect { case OutputRow.Incomplete(i) => i} + (good, bad, incomplete) + } + + trait StatsdAdmin { + def get(metricType: String): IO[String] + def getCounters = get("counters") + def getGauges = get("gauges") + } + + def mkStatsdAdmin(host: String, port: Int): Resource[IO, StatsdAdmin] = { + for { + socket <- Resource.make(IO.blocking(new Socket(host, port)))(s => IO(s.close())) + toStatsd <- Resource.make(IO(new PrintWriter(socket.getOutputStream(), true)))(pw => IO(pw.close())) + fromStatsd <- Resource.make(IO(new BufferedReader(new InputStreamReader(socket.getInputStream()))))(br => IO(br.close())) + } yield new StatsdAdmin { + def get(metricType: String): IO[String] = + for { + _ <- IO.blocking(toStatsd.println(metricType)) + stats <- IO.blocking(fromStatsd.lines().iterator().asScala.takeWhile(!_.toLowerCase().contains("end")).mkString("\n")) + } yield stats + } } } diff --git a/modules/kinesis/src/main/resources/application.conf b/modules/kinesis/src/main/resources/application.conf index 05145dbd2..7b14a4dc4 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 350342327..ec6e9e12f 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, 8), @@ -173,7 +185,8 @@ class ConfigSpec extends Specification with CatsEffect { 500, 5242880, None - ) + ), + None ), io.Concurrency(256, 8), 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, diff --git a/project/BuildSettings.scala b/project/BuildSettings.scala index ee55dfbb7..e6ca52360 100644 --- a/project/BuildSettings.scala +++ b/project/BuildSettings.scala @@ -197,7 +197,10 @@ object BuildSettings { // Project commonFs2ProjectSettings ++ buildSettings ++ // Tests - scoverageSettings ++ noParallelTestExecution ++ addExampleConfToTestCp + scoverageSettings ++ noParallelTestExecution ++ addExampleConfToTestCp ++ Seq( + Test / fork := true, + Test / javaOptions := Seq("-Dnashorn.args=--language=es6") + ) } lazy val awsUtilsBuildSettings = { diff --git a/project/Dependencies.scala b/project/Dependencies.scala index 73283ae08..3cc643e41 100644 --- a/project/Dependencies.scala +++ b/project/Dependencies.scala @@ -99,6 +99,7 @@ object Dependencies { val specs2CE = "1.5.0" val scalacheck = "1.17.0" val testcontainers = "0.40.10" + val dockerJava = "3.3.6" val parserCombinators = "2.1.1" val sentry = "1.7.30" @@ -163,6 +164,7 @@ object Dependencies { val eventGen = "com.snowplowanalytics" %% "snowplow-event-generator-core" % V.eventGen % Test val parserCombinators = "org.scala-lang.modules" %% "scala-parser-combinators" % V.parserCombinators % Test val testContainersIt = "com.dimafeng" %% "testcontainers-scala-core" % V.testcontainers % IntegrationTest + val dockerJavaIt = "com.github.docker-java" % "docker-java" % V.dockerJava % IntegrationTest val kinesisSdk = "com.amazonaws" % "aws-java-sdk-kinesis" % V.awsSdk val dynamodbSdk = "com.amazonaws" % "aws-java-sdk-dynamodb" % V.awsSdk