diff --git a/config/enrichments/cross_navigation_config.json b/config/enrichments/cross_navigation_config.json new file mode 100644 index 000000000..6c0850f2b --- /dev/null +++ b/config/enrichments/cross_navigation_config.json @@ -0,0 +1,9 @@ +{ + "schema": "iglu:com.snowplowanalytics.snowplow.enrichments/cross_navigation_config/jsonschema/1-0-0", + + "data": { + "enabled": false, + "vendor": "com.snowplowanalytics.snowplow.enrichments", + "name": "cross_navigation_config" + } +} 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 aec4d8479..9517a8d80 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 @@ -35,7 +35,7 @@ import com.snowplowanalytics.snowplow.enrich.common.{EtlPipeline, QueryStringPar import com.snowplowanalytics.snowplow.enrich.common.adapters.RawEvent import com.snowplowanalytics.snowplow.enrich.common.enrichments.{EventEnrichments => EE} import com.snowplowanalytics.snowplow.enrich.common.enrichments.{MiscEnrichments => ME} -import com.snowplowanalytics.snowplow.enrich.common.enrichments.registry._ +import com.snowplowanalytics.snowplow.enrich.common.enrichments.registry.{CrossNavigationEnrichment => CNE, _} import com.snowplowanalytics.snowplow.enrich.common.enrichments.registry.apirequest.ApiRequestEnrichment import com.snowplowanalytics.snowplow.enrich.common.enrichments.registry.pii.PiiPseudonymizerEnrichment import com.snowplowanalytics.snowplow.enrich.common.enrichments.registry.sqlquery.SqlQueryEnrichment @@ -205,7 +205,7 @@ object EnrichmentManager { _ <- getRefererUri[F](registry.refererParser) // Potentially set the referrer details and URL components qsMap <- extractQueryString[F](pageUri, raw.source.encoding) // Parse the page URI's querystring _ <- setCampaign[F](qsMap, registry.campaignAttribution) // Marketing attribution - _ <- getCrossDomain[F](qsMap) // Cross-domain tracking + _ <- getCrossDomain[F](qsMap, registry.crossNavigation) // Cross-domain tracking _ <- setEventFingerprint[F](raw.parameters, registry.eventFingerprint) // This enrichment cannot fail _ <- getCookieContexts // Execute cookie extractor enrichment _ <- getHttpHeaderContexts // Execute header extractor enrichment @@ -232,7 +232,7 @@ object EnrichmentManager { _ <- getRefererUri[F](registry.refererParser) // Potentially set the referrer details and URL components qsMap <- extractQueryString[F](pageUri, raw.source.encoding) // Parse the page URI's querystring _ <- setCampaign[F](qsMap, registry.campaignAttribution) // Marketing attribution - _ <- getCrossDomain[F](qsMap) // Cross-domain tracking + _ <- getCrossDomain[F](qsMap, registry.crossNavigation) // Cross-domain tracking _ <- setEventFingerprint[F](raw.parameters, registry.eventFingerprint) // This enrichment cannot fail _ <- getCookieContexts // Execute cookie extractor enrichment _ <- getHttpHeaderContexts // Execute header extractor enrichment @@ -607,18 +607,30 @@ object EnrichmentManager { } def getCrossDomain[F[_]: Applicative]( - pageQsMap: Option[QueryStringParameters] + pageQsMap: Option[QueryStringParameters], + crossNavEnrichment: Option[CNE] ): EStateT[F, Unit] = EStateT.fromEither { case (event, _) => pageQsMap match { case Some(qsMap) => - val crossDomainParseResult = WPE.parseCrossDomain(qsMap) - for ((maybeRefrDomainUserid, maybeRefrDvceTstamp) <- crossDomainParseResult.toOption) { - maybeRefrDomainUserid.foreach(event.refr_domain_userid = _) - maybeRefrDvceTstamp.foreach(event.refr_dvce_tstamp = _) - } - crossDomainParseResult.bimap(NonEmptyList.one(_), _ => Nil) + CNE + .parseCrossDomain(qsMap) + .bimap( + err => + crossNavEnrichment match { + case Some(cn) => NonEmptyList.one(cn.addEnrichmentInfo(err)) + case None => NonEmptyList.one(err) + }, + crossNavMap => { + crossNavMap.duid.foreach(event.refr_domain_userid = _) + crossNavMap.tstamp.foreach(event.refr_dvce_tstamp = _) + crossNavEnrichment match { + case Some(_) => crossNavMap.getCrossNavigationContext + case None => Nil + } + } + ) case None => Nil.asRight } } diff --git a/modules/common/src/main/scala/com.snowplowanalytics.snowplow.enrich/common/enrichments/EnrichmentRegistry.scala b/modules/common/src/main/scala/com.snowplowanalytics.snowplow.enrich/common/enrichments/EnrichmentRegistry.scala index a59219bb3..79758be78 100644 --- a/modules/common/src/main/scala/com.snowplowanalytics.snowplow.enrich/common/enrichments/EnrichmentRegistry.scala +++ b/modules/common/src/main/scala/com.snowplowanalytics.snowplow.enrich/common/enrichments/EnrichmentRegistry.scala @@ -161,6 +161,7 @@ object EnrichmentRegistry { registry <- er } yield registry.copy(weather = enrichment.some) case c: YauaaConf => er.map(_.copy(yauaa = c.enrichment.some)) + case c: CrossNavigationConf => er.map(_.copy(crossNavigation = c.enrichment.some)) } } @@ -224,6 +225,8 @@ object EnrichmentRegistry { PiiPseudonymizerEnrichment.parse(enrichmentConfig, schemaKey).map(_.some) case "iab_spiders_and_robots_enrichment" => IabEnrichment.parse(enrichmentConfig, schemaKey, localMode).map(_.some) + case "cross_navigation_config" => + CrossNavigationEnrichment.parse(enrichmentConfig, schemaKey).map(_.some) case _ => Option.empty[EnrichmentConf].validNel // Enrichment is not recognized } @@ -248,5 +251,6 @@ final case class EnrichmentRegistry[F[_]]( uaParser: Option[UaParserEnrichment[F]] = None, userAgentUtils: Option[UserAgentUtilsEnrichment] = None, weather: Option[WeatherEnrichment[F]] = None, - yauaa: Option[YauaaEnrichment] = None + yauaa: Option[YauaaEnrichment] = None, + crossNavigation: Option[CrossNavigationEnrichment] = None ) diff --git a/modules/common/src/main/scala/com.snowplowanalytics.snowplow.enrich/common/enrichments/registry/CrossNavigationEnrichment.scala b/modules/common/src/main/scala/com.snowplowanalytics.snowplow.enrich/common/enrichments/registry/CrossNavigationEnrichment.scala new file mode 100644 index 000000000..443075822 --- /dev/null +++ b/modules/common/src/main/scala/com.snowplowanalytics.snowplow.enrich/common/enrichments/registry/CrossNavigationEnrichment.scala @@ -0,0 +1,228 @@ +/* + * 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.enrichments.registry + +import java.time.format.DateTimeFormatter + +import cats.data.ValidatedNel +import cats.syntax.either._ +import cats.syntax.option._ +import cats.syntax.traverse._ + +import io.circe.Json +import io.circe.syntax._ + +import com.snowplowanalytics.iglu.core.{SchemaCriterion, SchemaKey, SchemaVer, SelfDescribingData} +import com.snowplowanalytics.snowplow.badrows.FailureDetails +import com.snowplowanalytics.snowplow.enrich.common.enrichments.{EventEnrichments => EE} +import com.snowplowanalytics.snowplow.enrich.common.enrichments.registry.EnrichmentConf.CrossNavigationConf +import com.snowplowanalytics.snowplow.enrich.common.utils.{ConversionUtils => CU} +import com.snowplowanalytics.snowplow.enrich.common.QueryStringParameters + +/** + * Companion object to create an instance of CrossNavigationEnrichment + * from the configuration. + */ +object CrossNavigationEnrichment extends ParseableEnrichment { + + type CrossNavTransformation = String => Either[FailureDetails.EnrichmentFailure, Option[String]] + + val supportedSchema = SchemaCriterion( + "com.snowplowanalytics.snowplow.enrichments", + "cross_navigation_config", + "jsonschema", + 1, + 0 + ) + + val outputSchema = SchemaKey( + "com.snowplowanalytics.snowplow", + "cross_navigation", + "jsonschema", + SchemaVer.Full(1, 0, 0) + ) + + /** + * Creates a CrossNavigationConf instance from a Json. + * @param config The cross_navigation_config enrichment JSON + * @param schemaKey provided for the enrichment, must be supported by this enrichment + * @return a CrossNavigation configuration + */ + override def parse( + config: Json, + schemaKey: SchemaKey, + localMode: Boolean = false + ): ValidatedNel[String, CrossNavigationConf] = + (for { + _ <- isParseable(config, schemaKey) + } yield CrossNavigationConf(schemaKey)).toValidatedNel + + /** + * Extract the referrer domain user ID and timestamp from the "_sp={{DUID}}.{{TSTAMP}}" + * portion of the querystring + * + * @param qsMap The querystring parameters + * @return Validation boxing a pair of optional strings corresponding to the two fields + */ + def parseCrossDomain(qsMap: QueryStringParameters): Either[FailureDetails.EnrichmentFailure, CrossDomainMap] = + qsMap.toMap + .map { case (k, v) => (k, v.getOrElse("")) } + .get("_sp") match { + case Some("") => CrossDomainMap.empty.asRight + case Some(sp) => CrossDomainMap.makeCrossDomainMap(sp) + case None => CrossDomainMap.empty.asRight + } + + case class CrossDomainMap(domainMap: Map[String, Option[String]]) { + + /** + * Gets the cross navigation parameters as self-describing JSON. + * + * @param cnMap The map of cross navigation data + * @return the cross navigation context wrapped in a List + */ + def getCrossNavigationContext: List[SelfDescribingData[Json]] = + domainMap match { + case m: Map[String, Option[String]] if m.isEmpty => Nil + case m: Map[String, Option[String]] if m.get(CrossDomainMap.domainUserIdFieldName).flatten == None => Nil + case m: Map[String, Option[String]] if m.get(CrossDomainMap.timestampFieldName).flatten == None => Nil + case _ => + List( + SelfDescribingData( + CrossNavigationEnrichment.outputSchema, + finalizeCrossNavigationMap.asJson + ) + ) + } + + def duid: Option[String] = domainMap.get(CrossDomainMap.domainUserIdFieldName).flatten + + def tstamp: Option[String] = domainMap.get(CrossDomainMap.timestampFieldName).flatten + + /** + * Finalizes the cross navigation map by reformatting its timestamp key + * + * @param inputMap A Map of cross navigation properties + * @return The finalized Map + */ + private def finalizeCrossNavigationMap: Map[String, Option[String]] = + domainMap + .map { + case ("timestamp", t) => ("timestamp" -> CrossDomainMap.reformatTstamp(t)) + case kvPair => kvPair + } + } + + object CrossDomainMap { + val domainUserIdFieldName = "domain_user_id" + val timestampFieldName = "timestamp" + val CrossNavProps: List[(String, CrossNavTransformation)] = + List( + (domainUserIdFieldName, CU.fixTabsNewlines(_).asRight), + (timestampFieldName, extractTstamp), + ("session_id", Option(_: String).filter(_.trim.nonEmpty).asRight), + ("user_id", decodeWithFailure), + ("source_id", decodeWithFailure), + ("source_platform", Option(_: String).filter(_.trim.nonEmpty).asRight), + ("reason", decodeWithFailure) + ) + + /** + * Parses the QueryString into a Map + * @param sp QueryString + * @return either a map of query string parameters or enrichment failure + */ + def makeCrossDomainMap(sp: String): Either[FailureDetails.EnrichmentFailure, CrossDomainMap] = { + val values = sp + .split("\\.", -1) + .padTo( + CrossNavProps.size, + "" + ) + .toList + val result = + if (values.size == CrossNavProps.size) + values + .zip(CrossNavProps) + .map { + case (value, (propName, f)) => f(value).map(propName -> _) + } + .sequence + .map(_.toMap) + else Map.empty[String, Option[String]].asRight + result.map(CrossDomainMap(_)) + } + + def empty: CrossDomainMap = CrossDomainMap(Map.empty) + + /** + * Wrapper around CU.decodeBase64Url. + * If passed an empty string returns Right(None). + * + * @param str The string to decode + * @return either the decoded string or enrichment failure + */ + private def decodeWithFailure(str: String): Either[FailureDetails.EnrichmentFailure, Option[String]] = + CU.decodeBase64Url(str) match { + case Right(r) => Option(r).filter(_.trim.nonEmpty).asRight + case Left(msg) => + FailureDetails + .EnrichmentFailure( + None, + FailureDetails.EnrichmentFailureMessage.Simple(msg) + ) + .asLeft + } + + /** + * Wrapper around EE.extractTimestamp + * If passed an empty string returns Right(None). + * + * @param str The string to extract the timestamp from + * @return either the extracted timestamp or enrichment failure + */ + private def extractTstamp(str: String): Either[FailureDetails.EnrichmentFailure, Option[String]] = + str match { + case "" => None.asRight + case s => EE.extractTimestamp("sp_dtm", s).map(_.some) + } + + /** + * Converts a timestamp to an ISO-8601 format + * + * @param tstamp The timestamp expected as output of EE.extractTimestamp + * @return ISO-8601 timestamp + */ + private def reformatTstamp(tstamp: Option[String]): Option[String] = { + val pFormatter = DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss.SSS") + val formatter = DateTimeFormatter.ISO_DATE_TIME + tstamp.map(t => formatter.format(pFormatter.parse(t)).replaceAll(" ", "T") + "Z") + } + } +} + +/** + * Enrichment adding cross navigation context + */ +final case class CrossNavigationEnrichment(schemaKey: SchemaKey) extends Enrichment { + private val enrichmentInfo = + FailureDetails.EnrichmentInformation(schemaKey, "cross-navigation").some + + /** + * Given an EnrichmentFailure, returns one with the cross-navigation + * enrichment information added. + * @param failure The input enrichment failure + * @return the EnrichmentFailure with cross-navigation enrichment information + */ + def addEnrichmentInfo(failure: FailureDetails.EnrichmentFailure): FailureDetails.EnrichmentFailure = + failure.copy(enrichment = enrichmentInfo) + +} diff --git a/modules/common/src/main/scala/com.snowplowanalytics.snowplow.enrich/common/enrichments/registry/EnrichmentConf.scala b/modules/common/src/main/scala/com.snowplowanalytics.snowplow.enrich/common/enrichments/registry/EnrichmentConf.scala index 3b5cd9444..fd8cd0cb6 100644 --- a/modules/common/src/main/scala/com.snowplowanalytics.snowplow.enrich/common/enrichments/registry/EnrichmentConf.scala +++ b/modules/common/src/main/scala/com.snowplowanalytics.snowplow.enrich/common/enrichments/registry/EnrichmentConf.scala @@ -245,4 +245,8 @@ object EnrichmentConf { ) extends EnrichmentConf { def enrichment: YauaaEnrichment = YauaaEnrichment(cacheSize) } + + final case class CrossNavigationConf(schemaKey: SchemaKey) extends EnrichmentConf { + def enrichment: CrossNavigationEnrichment = CrossNavigationEnrichment(schemaKey) + } } diff --git a/modules/common/src/main/scala/com.snowplowanalytics.snowplow.enrich/common/enrichments/web/PageEnrichments.scala b/modules/common/src/main/scala/com.snowplowanalytics.snowplow.enrich/common/enrichments/web/PageEnrichments.scala index be369cb95..b7c6838cf 100644 --- a/modules/common/src/main/scala/com.snowplowanalytics.snowplow.enrich/common/enrichments/web/PageEnrichments.scala +++ b/modules/common/src/main/scala/com.snowplowanalytics.snowplow.enrich/common/enrichments/web/PageEnrichments.scala @@ -13,13 +13,10 @@ package com.snowplowanalytics.snowplow.enrich.common.enrichments.web import java.net.URI import cats.syntax.either._ -import cats.syntax.option._ import com.snowplowanalytics.snowplow.badrows._ import com.snowplowanalytics.snowplow.enrich.common.utils.{ConversionUtils => CU} -import com.snowplowanalytics.snowplow.enrich.common.enrichments.EventEnrichments -import com.snowplowanalytics.snowplow.enrich.common.QueryStringParameters /** Holds enrichments related to the web page URL, and the document object contained in the page. */ object PageEnrichments { @@ -44,26 +41,4 @@ object PageEnrichments { FailureDetails.EnrichmentFailureMessage.Simple(f) ) ) - - /** - * Extract the referrer domain user ID and timestamp from the "_sp={{DUID}}.{{TSTAMP}}" - * portion of the querystring - * @param qsMap The querystring parameters - * @return Validation boxing a pair of optional strings corresponding to the two fields - */ - def parseCrossDomain(qsMap: QueryStringParameters): Either[FailureDetails.EnrichmentFailure, (Option[String], Option[String])] = - qsMap.toMap - .map { case (k, v) => (k, v.getOrElse("")) } - .get("_sp") match { - case Some("") => (None, None).asRight - case Some(sp) => - val crossDomainElements = sp.split("\\.") - val duid = CU.makeTsvSafe(crossDomainElements(0)).some - val tstamp = crossDomainElements.lift(1) match { - case Some(spDtm) => EventEnrichments.extractTimestamp("sp_dtm", spDtm).map(_.some) - case None => None.asRight - } - tstamp.map(duid -> _) - case None => (None -> None).asRight - } } 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 a933b010f..8eceb9023 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 @@ -21,16 +21,19 @@ import cats.effect.testing.specs2.CatsEffect import cats.implicits._ import cats.data.NonEmptyList +import io.circe.Json import io.circe.literal._ import io.circe.parser.{parse => jparse} +import io.circe.syntax._ import org.joda.time.DateTime import com.snowplowanalytics.snowplow.badrows._ import com.snowplowanalytics.snowplow.badrows.FailureDetails.EnrichmentFailureMessage -import com.snowplowanalytics.iglu.core.{SchemaCriterion, SchemaKey, SchemaVer} +import com.snowplowanalytics.iglu.core.{SchemaCriterion, SchemaKey, SchemaVer, SelfDescribingData} +import com.snowplowanalytics.snowplow.enrich.common.QueryStringParameters import com.snowplowanalytics.snowplow.enrich.common.loaders._ import com.snowplowanalytics.snowplow.enrich.common.adapters.RawEvent import com.snowplowanalytics.snowplow.enrich.common.enrichments.registry.pii.{ @@ -42,13 +45,13 @@ import com.snowplowanalytics.snowplow.enrich.common.enrichments.registry.pii.{ import com.snowplowanalytics.snowplow.enrich.common.outputs.EnrichedEvent import com.snowplowanalytics.snowplow.enrich.common.utils.ConversionUtils import com.snowplowanalytics.snowplow.enrich.common.enrichments.registry.{ + CrossNavigationEnrichment, HttpHeaderExtractorEnrichment, IabEnrichment, JavascriptScriptEnrichment, YauaaEnrichment } import com.snowplowanalytics.snowplow.enrich.common.AcceptInvalid - import com.snowplowanalytics.snowplow.enrich.common.SpecHelpers import com.snowplowanalytics.snowplow.enrich.common.SpecHelpers._ @@ -1063,6 +1066,324 @@ class EnrichmentManagerSpec extends Specification with EitherMatchers with CatsE } } + "getCrossDomain" should { + val schemaKey = SchemaKey( + CrossNavigationEnrichment.supportedSchema.vendor, + CrossNavigationEnrichment.supportedSchema.name, + CrossNavigationEnrichment.supportedSchema.format, + SchemaVer.Full(1, 0, 0) + ) + + "do nothing if none query string parameters - crossNavigation enabled" >> { + val crossNavigationEnabled = Some(new CrossNavigationEnrichment(schemaKey)) + val qsMap: Option[QueryStringParameters] = None + val input = new EnrichedEvent() + val inputState = EnrichmentManager.Accumulation(input, Nil, Nil) + EnrichmentManager + .getCrossDomain[IO]( + qsMap, + crossNavigationEnabled + ) + .runS(inputState) + .map( + _ must beLike { + case acc: EnrichmentManager.Accumulation => + val p = EnrichedEvent.toPartiallyEnrichedEvent(acc.event) + (p.refr_domain_userid must beNone) and + (p.refr_dvce_tstamp must beNone) and + (acc.errors must beEmpty) and + (acc.contexts must beEmpty) + } + ) + } + + "do nothing if none query string parameters - crossNavigation disabled" >> { + val crossNavigationDisabled = None + val qsMap: Option[QueryStringParameters] = None + val input = new EnrichedEvent() + val inputState = EnrichmentManager.Accumulation(input, Nil, Nil) + EnrichmentManager + .getCrossDomain[IO]( + qsMap, + crossNavigationDisabled + ) + .runS(inputState) + .map( + _ must beLike { + case acc: EnrichmentManager.Accumulation => + val p = EnrichedEvent.toPartiallyEnrichedEvent(acc.event) + (p.refr_domain_userid must beNone) and + (p.refr_dvce_tstamp must beNone) and + (acc.errors must beEmpty) and + (acc.contexts must beEmpty) + } + ) + } + + "do nothing if _sp is empty - crossNavigation enabled" >> { + val crossNavigationEnabled = Some(new CrossNavigationEnrichment(schemaKey)) + val qsMap: Option[QueryStringParameters] = Some(List(("_sp" -> Some("")))) + val input = new EnrichedEvent() + val inputState = EnrichmentManager.Accumulation(input, Nil, Nil) + EnrichmentManager + .getCrossDomain[IO]( + qsMap, + crossNavigationEnabled + ) + .runS(inputState) + .map( + _ must beLike { + case acc: EnrichmentManager.Accumulation => + val p = EnrichedEvent.toPartiallyEnrichedEvent(acc.event) + (p.refr_domain_userid must beNone) and + (p.refr_dvce_tstamp must beNone) and + (acc.errors must beEmpty) and + (acc.contexts must beEmpty) + } + ) + } + + "do nothing if _sp is empty - crossNavigation disabled" >> { + val crossNavigationDisabled = None + val qsMap: Option[QueryStringParameters] = Some(List(("_sp" -> Some("")))) + val input = new EnrichedEvent() + val inputState = EnrichmentManager.Accumulation(input, Nil, Nil) + EnrichmentManager + .getCrossDomain[IO]( + qsMap, + crossNavigationDisabled + ) + .runS(inputState) + .map( + _ must beLike { + case acc: EnrichmentManager.Accumulation => + val p = EnrichedEvent.toPartiallyEnrichedEvent(acc.event) + (p.refr_domain_userid must beNone) and + (p.refr_dvce_tstamp must beNone) and + (acc.errors must beEmpty) and + (acc.contexts must beEmpty) + } + ) + } + + "add atomic props and ctx with original _sp format and cross navigation enabled" >> { + val crossNavigationEnabled = Some(new CrossNavigationEnrichment(schemaKey)) + val qsMap: Option[QueryStringParameters] = Some( + List( + ("_sp" -> Some("abc.1697175843762")) + ) + ) + val expectedRefrDuid = Some("abc") + val expectedRefrTstamp = Some("2023-10-13 05:44:03.762") + val expectedCtx: List[SelfDescribingData[Json]] = List( + SelfDescribingData( + CrossNavigationEnrichment.outputSchema, + Map( + "domain_user_id" -> Some("abc"), + "timestamp" -> Some("2023-10-13T05:44:03.762Z"), + "session_id" -> None, + "user_id" -> None, + "source_id" -> None, + "source_platform" -> None, + "reason" -> None + ).asJson + ) + ) + val input = new EnrichedEvent() + val inputState = EnrichmentManager.Accumulation(input, Nil, Nil) + EnrichmentManager + .getCrossDomain[IO]( + qsMap, + crossNavigationEnabled + ) + .runS(inputState) + .map( + _ must beLike { + case acc: EnrichmentManager.Accumulation => + val p = EnrichedEvent.toPartiallyEnrichedEvent(acc.event) + (p.refr_domain_userid must beEqualTo(expectedRefrDuid)) and + (p.refr_dvce_tstamp must beEqualTo(expectedRefrTstamp)) and + (acc.errors must beEmpty) and + (acc.contexts must beEqualTo(expectedCtx)) + } + ) + } + + "add atomic props but no ctx with original _sp format and cross navigation disabled" >> { + val crossNavigationDisabled = None + val qsMap: Option[QueryStringParameters] = Some( + List( + ("_sp" -> Some("abc.1697175843762")) + ) + ) + val expectedRefrDuid = Some("abc") + val expectedRefrTstamp = Some("2023-10-13 05:44:03.762") + val input = new EnrichedEvent() + val inputState = EnrichmentManager.Accumulation(input, Nil, Nil) + EnrichmentManager + .getCrossDomain[IO]( + qsMap, + crossNavigationDisabled + ) + .runS(inputState) + .map( + _ must beLike { + case acc: EnrichmentManager.Accumulation => + val p = EnrichedEvent.toPartiallyEnrichedEvent(acc.event) + (p.refr_domain_userid must beEqualTo(expectedRefrDuid)) and + (p.refr_dvce_tstamp must beEqualTo(expectedRefrTstamp)) and + (acc.errors must beEmpty) and + (acc.contexts must beEmpty) + } + ) + } + + "add atomic props and ctx with extended _sp format and cross navigation enabled" >> { + val crossNavigationEnabled = Some(new CrossNavigationEnrichment(schemaKey)) + val qsMap: Option[QueryStringParameters] = Some( + List( + ("_sp" -> Some("abc.1697175843762.176ff68a-4769-4566-ad0e-3792c1c8148f.dGVzdGVy.c29tZVNvdXJjZUlk.web.dGVzdGluZ19yZWFzb24")) + ) + ) + val expectedRefrDuid = Some("abc") + val expectedRefrTstamp = Some("2023-10-13 05:44:03.762") + val expectedCtx: List[SelfDescribingData[Json]] = List( + SelfDescribingData( + CrossNavigationEnrichment.outputSchema, + Map( + "domain_user_id" -> Some("abc"), + "timestamp" -> Some("2023-10-13T05:44:03.762Z"), + "session_id" -> Some("176ff68a-4769-4566-ad0e-3792c1c8148f"), + "user_id" -> Some("tester"), + "source_id" -> Some("someSourceId"), + "source_platform" -> Some("web"), + "reason" -> Some("testing_reason") + ).asJson + ) + ) + val input = new EnrichedEvent() + val inputState = EnrichmentManager.Accumulation(input, Nil, Nil) + EnrichmentManager + .getCrossDomain[IO]( + qsMap, + crossNavigationEnabled + ) + .runS(inputState) + .map( + _ must beLike { + case acc: EnrichmentManager.Accumulation => + val p = EnrichedEvent.toPartiallyEnrichedEvent(acc.event) + (p.refr_domain_userid must beEqualTo(expectedRefrDuid)) and + (p.refr_dvce_tstamp must beEqualTo(expectedRefrTstamp)) and + (acc.errors must beEmpty) and + (acc.contexts must beEqualTo(expectedCtx)) + } + ) + } + + "add atomic props but no ctx with extended _sp format and cross navigation disabled" >> { + val crossNavigationDisabled = None + val qsMap: Option[QueryStringParameters] = Some( + List( + ("_sp" -> Some("abc.1697175843762.176ff68a-4769-4566-ad0e-3792c1c8148f.dGVzdGVy.c29tZVNvdXJjZUlk.web.dGVzdGluZ19yZWFzb24")) + ) + ) + val expectedRefrDuid = Some("abc") + val expectedRefrTstamp = Some("2023-10-13 05:44:03.762") + val input = new EnrichedEvent() + val inputState = EnrichmentManager.Accumulation(input, Nil, Nil) + EnrichmentManager + .getCrossDomain[IO]( + qsMap, + crossNavigationDisabled + ) + .runS(inputState) + .map( + _ must beLike { + case acc: EnrichmentManager.Accumulation => + val p = EnrichedEvent.toPartiallyEnrichedEvent(acc.event) + (p.refr_domain_userid must beEqualTo(expectedRefrDuid)) and + (p.refr_dvce_tstamp must beEqualTo(expectedRefrTstamp)) and + (acc.errors must beEmpty) and + (acc.contexts must beEmpty) + } + ) + } + + "error with info if parsing failed and cross navigation is enabled" >> { + val crossNavigationEnabled = Some(new CrossNavigationEnrichment(schemaKey)) + // causing a parsing failure by providing invalid tstamp + val qsMap: Option[QueryStringParameters] = Some( + List( + ("_sp" -> Some("abc.some_invalid_timestamp_value")) + ) + ) + val input = new EnrichedEvent() + val expectedFail = FailureDetails.EnrichmentFailure( + FailureDetails + .EnrichmentInformation( + schemaKey, + "cross-navigation" + ) + .some, + FailureDetails.EnrichmentFailureMessage.InputData( + "sp_dtm", + "some_invalid_timestamp_value".some, + "not in the expected format: ms since epoch" + ) + ) + val inputState = EnrichmentManager.Accumulation(input, Nil, Nil) + EnrichmentManager + .getCrossDomain[IO]( + qsMap, + crossNavigationEnabled + ) + .runS(inputState) + .map( + _ must beLike { + case acc: EnrichmentManager.Accumulation => + (acc.errors must not beEmpty) and + (acc.errors must beEqualTo(List(expectedFail))) and + (acc.contexts must beEmpty) + } + ) + } + + "error without info if parsing failed and cross navigation is disabled" >> { + val crossNavigationDisabled = None + // causing a parsing failure by providing invalid tstamp + val qsMap: Option[QueryStringParameters] = Some( + List( + ("_sp" -> Some("abc.some_invalid_timestamp_value")) + ) + ) + val input = new EnrichedEvent() + val expectedFail = FailureDetails.EnrichmentFailure( + None, + FailureDetails.EnrichmentFailureMessage.InputData( + "sp_dtm", + "some_invalid_timestamp_value".some, + "not in the expected format: ms since epoch" + ) + ) + val inputState = EnrichmentManager.Accumulation(input, Nil, Nil) + EnrichmentManager + .getCrossDomain[IO]( + qsMap, + crossNavigationDisabled + ) + .runS(inputState) + .map( + _ must beLike { + case acc: EnrichmentManager.Accumulation => + (acc.errors must not beEmpty) and + (acc.errors must beEqualTo(List(expectedFail))) and + (acc.contexts must beEmpty) + } + ) + } + } + "getIabContext" should { "return no context if useragent is null" >> { val input = new EnrichedEvent() diff --git a/modules/common/src/test/scala/com.snowplowanalytics.snowplow.enrich.common/enrichments/registry/CrossNavigationEnrichmentSpec.scala b/modules/common/src/test/scala/com.snowplowanalytics.snowplow.enrich.common/enrichments/registry/CrossNavigationEnrichmentSpec.scala new file mode 100644 index 000000000..db830ff43 --- /dev/null +++ b/modules/common/src/test/scala/com.snowplowanalytics.snowplow.enrich.common/enrichments/registry/CrossNavigationEnrichmentSpec.scala @@ -0,0 +1,286 @@ +/* + * 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.enrichments.registry + +import cats.syntax.either._ +import cats.syntax.option._ + +import com.snowplowanalytics.iglu.core.{SchemaKey, SchemaVer, SelfDescribingData} +import com.snowplowanalytics.snowplow.badrows._ + +import io.circe.Json +import io.circe.syntax._ + +import org.specs2.mutable.Specification +import org.specs2.matcher.EitherMatchers + +class CrossNavigationEnrichmentSpec extends Specification with EitherMatchers { + import CrossNavigationEnrichment._ + + val schemaKey = SchemaKey( + CrossNavigationEnrichment.supportedSchema.vendor, + CrossNavigationEnrichment.supportedSchema.name, + CrossNavigationEnrichment.supportedSchema.format, + SchemaVer.Full(1, 0, 0) + ) + + "makeCrossDomainMap" should { + "return expected Map on original format" >> { + val input = "abc.1697398398279" + val expectedOut: Map[String, Option[String]] = Map( + "domain_user_id" -> Some("abc"), + "timestamp" -> Some("2023-10-15 19:33:18.279"), + "session_id" -> None, + "user_id" -> None, + "source_id" -> None, + "source_platform" -> None, + "reason" -> None + ) + val result = CrossDomainMap.makeCrossDomainMap(input).map(_.domainMap) + result must beEqualTo(expectedOut.asRight) + } + + "return expected Map on original format when missing timestamp" >> { + val input = "abc" + val expectedOut: Map[String, Option[String]] = Map( + "domain_user_id" -> Some("abc"), + "timestamp" -> None, + "session_id" -> None, + "user_id" -> None, + "source_id" -> None, + "source_platform" -> None, + "reason" -> None + ) + val result = CrossDomainMap.makeCrossDomainMap(input).map(_.domainMap) + result must beEqualTo(expectedOut.asRight) + } + + "return expected Map on original format when missing duid" >> { + val input = ".1697398398279" + val expectedOut: Map[String, Option[String]] = Map( + "domain_user_id" -> None, + "timestamp" -> Some("2023-10-15 19:33:18.279"), + "session_id" -> None, + "user_id" -> None, + "source_id" -> None, + "source_platform" -> None, + "reason" -> None + ) + val result = CrossDomainMap.makeCrossDomainMap(input).map(_.domainMap) + result must beEqualTo(expectedOut.asRight) + } + + "return expected Map on extended format" >> { + val input = "abc.1697175843762.176ff68a-4769-4566-ad0e-3792c1c8148f.dGVzdGVy.c29tZVNvdXJjZUlk.web.dGVzdGluZ19yZWFzb24" + val expectedOut: Map[String, Option[String]] = Map( + "domain_user_id" -> Some("abc"), + "timestamp" -> Some("2023-10-13 05:44:03.762"), + "session_id" -> Some("176ff68a-4769-4566-ad0e-3792c1c8148f"), + "user_id" -> Some("tester"), + "source_id" -> Some("someSourceId"), + "source_platform" -> Some("web"), + "reason" -> Some("testing_reason") + ) + val result = CrossDomainMap.makeCrossDomainMap(input).map(_.domainMap) + result must beEqualTo(expectedOut.asRight) + } + + "return expected Map on extended format when missing timestamp" >> { + val input = "abc..176ff68a-4769-4566-ad0e-3792c1c8148f.dGVzdGVy.c29tZVNvdXJjZUlk.web.dGVzdGluZ19yZWFzb24" + val expectedOut: Map[String, Option[String]] = Map( + "domain_user_id" -> Some("abc"), + "timestamp" -> None, + "session_id" -> Some("176ff68a-4769-4566-ad0e-3792c1c8148f"), + "user_id" -> Some("tester"), + "source_id" -> Some("someSourceId"), + "source_platform" -> Some("web"), + "reason" -> Some("testing_reason") + ) + val result = CrossDomainMap.makeCrossDomainMap(input).map(_.domainMap) + result must beEqualTo(expectedOut.asRight) + } + + "return expected Map on extended format when missing duid" >> { + val input = "..176ff68a-4769-4566-ad0e-3792c1c8148f.dGVzdGVy.c29tZVNvdXJjZUlk.web.dGVzdGluZ19yZWFzb24" + val expectedOut: Map[String, Option[String]] = Map( + "domain_user_id" -> None, + "timestamp" -> None, + "session_id" -> Some("176ff68a-4769-4566-ad0e-3792c1c8148f"), + "user_id" -> Some("tester"), + "source_id" -> Some("someSourceId"), + "source_platform" -> Some("web"), + "reason" -> Some("testing_reason") + ) + val result = CrossDomainMap.makeCrossDomainMap(input).map(_.domainMap) + result must beEqualTo(expectedOut.asRight) + } + + "handle variations of extended format 1" >> { + val input = "abc.1697175843762..dGVzdGVy..web" + val expectedOut: Map[String, Option[String]] = Map( + "domain_user_id" -> Some("abc"), + "timestamp" -> Some("2023-10-13 05:44:03.762"), + "session_id" -> None, + "user_id" -> Some("tester"), + "source_id" -> None, + "source_platform" -> Some("web"), + "reason" -> None + ) + val result = CrossDomainMap.makeCrossDomainMap(input).map(_.domainMap) + result must beEqualTo(expectedOut.asRight) + } + + "handle variations of extended format 2" >> { + val input = "abc..176ff68a-4769-4566-ad0e-3792c1c8148f.." + val expectedOut: Map[String, Option[String]] = Map( + "domain_user_id" -> Some("abc"), + "timestamp" -> None, + "session_id" -> Some("176ff68a-4769-4566-ad0e-3792c1c8148f"), + "user_id" -> None, + "source_id" -> None, + "source_platform" -> None, + "reason" -> None + ) + val result = CrossDomainMap.makeCrossDomainMap(input).map(_.domainMap) + result must beEqualTo(expectedOut.asRight) + } + + "handle variations of extended format 3" >> { + val input = "abc.1697175843762....." + val expectedOut: Map[String, Option[String]] = Map( + "domain_user_id" -> Some("abc"), + "timestamp" -> Some("2023-10-13 05:44:03.762"), + "session_id" -> None, + "user_id" -> None, + "source_id" -> None, + "source_platform" -> None, + "reason" -> None + ) + val result = CrossDomainMap.makeCrossDomainMap(input).map(_.domainMap) + result must beEqualTo(expectedOut.asRight) + } + + "return empty map on invalid format" >> { + val input = "abc.1697175843762.176ff68a-4769-4566-ad0e-3792c1c8148f.dGVzdGVy.c29tZVNvdXJjZUlk.web.dGVzdGluZ19yZWFzb24...foo..bar..." + val expectedOut = Map.empty[String, Option[String]] + val result = CrossDomainMap.makeCrossDomainMap(input).map(_.domainMap) + result must beEqualTo(expectedOut.asRight) + } + + "return failure on invalid timestamp" >> { + val input = "abc.not-timestamp.176ff68a-4769-4566-ad0e-3792c1c8148f." + val expectedOut = FailureDetails.EnrichmentFailure( + None, + FailureDetails.EnrichmentFailureMessage.InputData( + "sp_dtm", + "not-timestamp".some, + "not in the expected format: ms since epoch" + ) + ) + val result = CrossDomainMap.makeCrossDomainMap(input).map(_.domainMap) + result must beEqualTo(expectedOut.asLeft) + } + + "return failure on incompatible timestamp" >> { + val input = "abc.1111111111111111.176ff68a-4769-4566-ad0e-3792c1c8148f." + val expectedOut = FailureDetails.EnrichmentFailure( + None, + FailureDetails.EnrichmentFailureMessage.InputData( + "sp_dtm", + "1111111111111111".some, + "formatting as 37179-09-17 07:18:31.111 is not Redshift-compatible" + ) + ) + val result = CrossDomainMap.makeCrossDomainMap(input).map(_.domainMap) + result must beEqualTo(expectedOut.asLeft) + } + } + + "getCrossNavigationContext" should { + "return Nil if input is empty Map" >> { + val input = Map.empty[String, Option[String]] + val expectedOut: List[SelfDescribingData[Json]] = Nil + val result = CrossDomainMap(input).getCrossNavigationContext + result must beEqualTo(expectedOut) + } + + "return Nil if missing domain_user_id (required)" >> { + val input: Map[String, Option[String]] = Map( + "timestamp" -> Some("2023-10-13 05:44:03.762") + ) + val expectedOut: List[SelfDescribingData[Json]] = Nil + val result = CrossDomainMap(input).getCrossNavigationContext + result must beEqualTo(expectedOut) + } + + "return Nil if missing timestamp (required)" >> { + val input: Map[String, Option[String]] = Map( + "domain_user_id" -> Some("abcd") + ) + val expectedOut: List[SelfDescribingData[Json]] = Nil + val result = CrossDomainMap(input).getCrossNavigationContext + result must beEqualTo(expectedOut) + } + + "return List of SelfDescribingData" >> { + val input: Map[String, Option[String]] = Map( + "domain_user_id" -> Some("abc"), + "timestamp" -> Some("2023-10-13 05:44:03.762"), + "session_id" -> Some("176ff68a-4769-4566-ad0e-3792c1c8148f"), + "user_id" -> Some("tester"), + "source_id" -> Some("someSourceId"), + "source_platform" -> Some("web"), + "reason" -> Some("testing_reason") + ) + val expectedOut: List[SelfDescribingData[Json]] = List( + SelfDescribingData( + CrossNavigationEnrichment.outputSchema, + Map( + "domain_user_id" -> Some("abc"), + "timestamp" -> Some("2023-10-13T05:44:03.762Z"), + "session_id" -> Some("176ff68a-4769-4566-ad0e-3792c1c8148f"), + "user_id" -> Some("tester"), + "source_id" -> Some("someSourceId"), + "source_platform" -> Some("web"), + "reason" -> Some("testing_reason") + ).asJson + ) + ) + val result = CrossDomainMap(input).getCrossNavigationContext + result must beEqualTo(expectedOut) + } + } + + "addEnrichmentInfo" should { + val cne = new CrossNavigationEnrichment(schemaKey) + + "add the cross-navigation enrichment info" >> { + val input = FailureDetails.EnrichmentFailure( + None, + FailureDetails.EnrichmentFailureMessage.InputData( + "some_field", + Some("some_value"), + "some error message" + ) + ) + val expectedOut = FailureDetails.EnrichmentFailure( + FailureDetails.EnrichmentInformation(schemaKey, "cross-navigation").some, + FailureDetails.EnrichmentFailureMessage.InputData( + "some_field", + Some("some_value"), + "some error message" + ) + ) + val result = cne.addEnrichmentInfo(input) + result must beEqualTo(expectedOut) + } + } +} diff --git a/modules/common/src/test/scala/com.snowplowanalytics.snowplow.enrich.common/enrichments/registry/EnrichmentConfigsSpec.scala b/modules/common/src/test/scala/com.snowplowanalytics.snowplow.enrich.common/enrichments/registry/EnrichmentConfigsSpec.scala index 1105ed22a..56b591c9d 100644 --- a/modules/common/src/test/scala/com.snowplowanalytics.snowplow.enrich.common/enrichments/registry/EnrichmentConfigsSpec.scala +++ b/modules/common/src/test/scala/com.snowplowanalytics.snowplow.enrich.common/enrichments/registry/EnrichmentConfigsSpec.scala @@ -474,4 +474,21 @@ class EnrichmentConfigsSpec extends Specification with ValidatedMatchers with Da result must beInvalid } } + + "Parsing an cross_navigation_config JSON" should { + "successfully construct a CrossNavigationEnrichment" in { + val crossNavJson = json"""{ + "enabled": true + }""" + val schemaKey = SchemaKey( + "com.snowplowanalytics.snowplow.enrichments", + "cross_navigation_config", + "jsonschema", + SchemaVer.Full(1, 0, 0) + ) + val expected = CrossNavigationConf(schemaKey) + val result = CrossNavigationEnrichment.parse(crossNavJson, schemaKey, false) + result must beValid(expected) + } + } } diff --git a/modules/common/src/test/scala/com.snowplowanalytics.snowplow.enrich.common/enrichments/web/ParseCrossDomainSpec.scala b/modules/common/src/test/scala/com.snowplowanalytics.snowplow.enrich.common/enrichments/web/ParseCrossDomainSpec.scala index bf54611a3..0468e005f 100644 --- a/modules/common/src/test/scala/com.snowplowanalytics.snowplow.enrich.common/enrichments/web/ParseCrossDomainSpec.scala +++ b/modules/common/src/test/scala/com.snowplowanalytics.snowplow.enrich.common/enrichments/web/ParseCrossDomainSpec.scala @@ -11,16 +11,18 @@ package com.snowplowanalytics.snowplow.enrich.common.enrichments.web import cats.syntax.option._ -import org.specs2.Specification -import org.specs2.matcher.DataTables import com.snowplowanalytics.snowplow.badrows._ +import com.snowplowanalytics.snowplow.enrich.common.enrichments.registry.CrossNavigationEnrichment + +import org.specs2.Specification +import org.specs2.matcher.DataTables class ParseCrossDomainSpec extends Specification with DataTables { def is = s2""" - parseCrossDomain should return None when the querystring is empty $e1 - parseCrossDomain should return None when the querystring contains no _sp parameter $e2 - parseCrossDomain should return None when the querystring contains _sp parameter without value $e3 + parseCrossDomain should return empty Map when the querystring is empty $e1 + parseCrossDomain should return empty Map when the querystring contains no _sp parameter $e2 + parseCrossDomain should return empty Map when the querystring contains _sp parameter without value $e3 parseCrossDomain should return a failure when the _sp timestamp is unparseable $e4 parseCrossDomain should successfully extract the domain user ID when available $e5 parseCrossDomain should successfully extract the domain user ID and timestamp when available $e6 @@ -28,13 +30,15 @@ class ParseCrossDomainSpec extends Specification with DataTables { """ def e1 = - PageEnrichments.parseCrossDomain(Nil) must beRight((None, None)) + CrossNavigationEnrichment.parseCrossDomain(Nil).map(_.domainMap) must beRight(Map.empty[String, Option[String]]) def e2 = - PageEnrichments.parseCrossDomain(List(("foo" -> Some("bar")))) must beRight((None, None)) + CrossNavigationEnrichment.parseCrossDomain(List(("foo" -> Some("bar")))).map(_.domainMap) must beRight( + Map.empty[String, Option[String]] + ) def e3 = - PageEnrichments.parseCrossDomain(List(("_sp" -> None))) must beRight((None, None)) + CrossNavigationEnrichment.parseCrossDomain(List(("_sp" -> None))).map(_.domainMap) must beRight(Map.empty[String, Option[String]]) def e4 = { val expected = FailureDetails.EnrichmentFailure( @@ -45,17 +49,35 @@ class ParseCrossDomainSpec extends Specification with DataTables { "not in the expected format: ms since epoch" ) ) - PageEnrichments.parseCrossDomain(List(("_sp" -> Some("abc.not-a-timestamp")))) must beLeft(expected) + CrossNavigationEnrichment.parseCrossDomain(List(("_sp" -> Some("abc.not-a-timestamp")))).map(_.domainMap) must beLeft(expected) } def e5 = - PageEnrichments.parseCrossDomain(List(("_sp" -> Some("abc")))) must beRight(("abc".some, None)) + CrossNavigationEnrichment.parseCrossDomain(List(("_sp" -> Some("abc")))).map(_.domainMap) must beRight( + Map( + "domain_user_id" -> "abc".some, + "timestamp" -> None, + "session_id" -> None, + "user_id" -> None, + "source_id" -> None, + "source_platform" -> None, + "reason" -> None + ) + ) def e6 = - PageEnrichments.parseCrossDomain(List(("_sp" -> Some("abc.1426245561368")))) must beRight( - ("abc".some, "2015-03-13 11:19:21.368".some) + CrossNavigationEnrichment.parseCrossDomain(List(("_sp" -> Some("abc.1426245561368")))).map(_.domainMap) must beRight( + Map( + "domain_user_id" -> "abc".some, + "timestamp" -> "2015-03-13 11:19:21.368".some, + "session_id" -> None, + "user_id" -> None, + "source_id" -> None, + "source_platform" -> None, + "reason" -> None + ) ) def e7 = - PageEnrichments.parseCrossDomain(List(("_sp" -> Some("")))) must beRight(None -> None) + CrossNavigationEnrichment.parseCrossDomain(List(("_sp" -> Some("")))).map(_.domainMap) must beRight(Map.empty[String, Option[String]]) }