Skip to content

Commit

Permalink
Allow multiple javascript enrichments (close #868)
Browse files Browse the repository at this point in the history
  • Loading branch information
istreeter authored and spenes committed Feb 12, 2024
1 parent b5e32e7 commit fad74df
Show file tree
Hide file tree
Showing 4 changed files with 234 additions and 14 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -211,7 +211,7 @@ object EnrichmentManager {
_ <- getHttpHeaderContexts // Execute header extractor enrichment
_ <- getYauaaContext[F](registry.yauaa, raw.context.headers) // Runs YAUAA enrichment (gets info thanks to user agent)
_ <- extractSchemaFields[F](unstructEvent) // Extract the event vendor/name/format/version
_ <- getJsScript[F](registry.javascriptScript) // Execute the JavaScript scripting enrichment
_ <- registry.javascriptScript.traverse(getJsScript[F](_)) // Execute the JavaScript scripting enrichment
_ <- getCurrency[F](raw.context.timestamp, registry.currencyConversion) // Finalize the currency conversion
_ <- getWeatherContext[F](registry.weather) // Fetch weather context
_ <- geoLocation[F](registry.ipLookups) // Execute IP lookup enrichment
Expand Down Expand Up @@ -240,7 +240,7 @@ object EnrichmentManager {
_ <- getYauaaContext[F](registry.yauaa, raw.context.headers) // Runs YAUAA enrichment (gets info thanks to user agent)
_ <- extractSchemaFields[F](unstructEvent) // Extract the event vendor/name/format/version
_ <- geoLocation[F](registry.ipLookups) // Execute IP lookup enrichment
_ <- getJsScript[F](registry.javascriptScript) // Execute the JavaScript scripting enrichment
_ <- registry.javascriptScript.traverse(getJsScript[F](_)) // Execute the JavaScript scripting enrichment
_ <- sqlContexts // Derive some contexts with custom SQL Query enrichment
_ <- apiContexts // Derive some contexts with custom API Request enrichment
// format: on
Expand Down Expand Up @@ -671,16 +671,12 @@ object EnrichmentManager {

// Execute the JavaScript scripting enrichment
def getJsScript[F[_]: Applicative](
javascriptScript: Option[JavascriptScriptEnrichment]
javascriptScript: JavascriptScriptEnrichment
): EStateT[F, Unit] =
EStateT.fromEither {
case (event, derivedContexts) =>
javascriptScript match {
case Some(jse) =>
ME.formatContexts(derivedContexts).foreach(c => event.derived_contexts = c)
jse.process(event).leftMap(NonEmptyList.one)
case None => Nil.asRight
}
ME.formatContexts(derivedContexts).foreach(c => event.derived_contexts = c)
javascriptScript.process(event).leftMap(NonEmptyList.one)
}

def headerContexts[F[_]: Applicative, A](
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -143,7 +143,7 @@ object EnrichmentRegistry {
enrichment <- EitherT.right(c.enrichment[F](blockingEC))
registry <- er
} yield registry.copy(ipLookups = enrichment.some)
case c: JavascriptScriptConf => er.map(_.copy(javascriptScript = c.enrichment.some))
case c: JavascriptScriptConf => er.map(v => v.copy(javascriptScript = v.javascriptScript :+ c.enrichment))
case c: RefererParserConf =>
for {
enrichment <- c.enrichment[F]
Expand Down Expand Up @@ -246,7 +246,7 @@ final case class EnrichmentRegistry[F[_]](
httpHeaderExtractor: Option[HttpHeaderExtractorEnrichment] = None,
iab: Option[IabEnrichment] = None,
ipLookups: Option[IpLookupsEnrichment[F]] = None,
javascriptScript: Option[JavascriptScriptEnrichment] = None,
javascriptScript: List[JavascriptScriptEnrichment] = Nil,
refererParser: Option[RefererParserEnrichment] = None,
uaParser: Option[UaParserEnrichment[F]] = None,
userAgentUtils: Option[UserAgentUtilsEnrichment] = None,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -160,7 +160,7 @@ class EnrichmentManagerSpec extends Specification with EitherMatchers with CatsE
val jsEnrichConf =
JavascriptScriptEnrichment.parse(config, schemaKey).toOption.get
val jsEnrich = JavascriptScriptEnrichment(jsEnrichConf.schemaKey, jsEnrichConf.rawFunction)
val enrichmentReg = EnrichmentRegistry[IO](javascriptScript = Some(jsEnrich))
val enrichmentReg = EnrichmentRegistry[IO](javascriptScript = List(jsEnrich))

val parameters = Map(
"e" -> "pp",
Expand Down Expand Up @@ -230,7 +230,7 @@ class EnrichmentManagerSpec extends Specification with EitherMatchers with CatsE
val jsEnrichConf =
JavascriptScriptEnrichment.parse(config, schemaKey).toOption.get
val jsEnrich = JavascriptScriptEnrichment(jsEnrichConf.schemaKey, jsEnrichConf.rawFunction)
val enrichmentReg = EnrichmentRegistry[IO](javascriptScript = Some(jsEnrich))
val enrichmentReg = EnrichmentRegistry[IO](javascriptScript = List(jsEnrich))

val parameters = Map(
"e" -> "pp",
Expand Down Expand Up @@ -876,7 +876,7 @@ class EnrichmentManagerSpec extends Specification with EitherMatchers with CatsE
SchemaVer.Full(1, 0, 0)
)
val enrichmentReg = EnrichmentRegistry[IO](
javascriptScript = Some(JavascriptScriptEnrichment(schemaKey, script)),
javascriptScript = List(JavascriptScriptEnrichment(schemaKey, script)),
httpHeaderExtractor = Some(HttpHeaderExtractorEnrichment(".*"))
)

Expand All @@ -903,6 +903,57 @@ class EnrichmentManagerSpec extends Specification with EitherMatchers with CatsE
}
}

"run multiple JavaScript enrichments" >> {
val script1 =
"""
function process(event) {
event.setApp_id("test_app_id");
return [];
}"""

val script2 =
"""
function process(event) {
event.setPlatform("test_platform");
return [];
}"""

val schemaKey = SchemaKey(
"com.snowplowanalytics.snowplow",
"javascript_script_config",
"jsonschema",
SchemaVer.Full(1, 0, 0)
)
val enrichmentReg = EnrichmentRegistry[IO](
javascriptScript = List(
JavascriptScriptEnrichment(schemaKey, script1),
JavascriptScriptEnrichment(schemaKey, script2)
)
)

val parameters = Map(
"e" -> "pp",
"tv" -> "js-0.13.1",
"p" -> "web"
).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
)
enriched.value.map { e =>
(e.map(_.app_id) must beRight("test_app_id")) and
(e.map(_.platform) must beRight("test_platform"))
}
}

"emit an EnrichedEvent with superseded schemas" >> {
val expectedContexts = jparse(
"""
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,173 @@
/*
* 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

import org.specs2.Specification

import cats.effect.IO
import cats.effect.testing.specs2.CatsEffect

import io.circe.Json
import io.circe.literal._

import com.snowplowanalytics.snowplow.enrich.common.enrichments.registry.EnrichmentConf
import com.snowplowanalytics.snowplow.enrich.common.utils.ConversionUtils
import com.snowplowanalytics.snowplow.enrich.common.SpecHelpers

class EnrichmentRegistrySpec extends Specification with CatsEffect {
import EnrichmentRegistrySpec._

def is = s2"""
EnrichmentRegistry should parse array of enrichments without any JS enrichment correctly $noJSEnrichment
EnrichmentRegistry should parse array of enrichments with single JS enrichment correctly $singleJSEnrichment
EnrichmentRegistry should parse array of enrichments with multiple JS enrichments correctly $multipleJSEnrichments
"""

def noJSEnrichment =
EnrichmentRegistry
.parse[IO](
enrichmentConfig(),
SpecHelpers.client,
localMode = false,
SpecHelpers.registryLookup
)
.map { res =>
val jsConfs = res.getOrElse(List.empty).filter {
case _: EnrichmentConf.JavascriptScriptConf => true
case _ => false
}
jsConfs.size must beEqualTo(0)
}

def singleJSEnrichment = {
val jsEnrichments = List(jsEnrichment())
EnrichmentRegistry
.parse[IO](
enrichmentConfig(jsEnrichments),
SpecHelpers.client,
localMode = false,
SpecHelpers.registryLookup
)
.map { res =>
val jsConfs = res.getOrElse(List.empty).filter {
case _: EnrichmentConf.JavascriptScriptConf => true
case _ => false
}
jsConfs.size must beEqualTo(1)
}
}

def multipleJSEnrichments = {
val jsReturns = List("return1", "return2")
val jsEnrichments = jsReturns.map(jsEnrichment)
EnrichmentRegistry
.parse[IO](
enrichmentConfig(jsEnrichments),
SpecHelpers.client,
localMode = false,
SpecHelpers.registryLookup
)
.map { res =>
val jsConfs = res.getOrElse(List.empty).flatMap {
case e: EnrichmentConf.JavascriptScriptConf => Some(e)
case _ => None
}
jsReturns.zip(jsConfs).forall {
case (jsReturn, jsConf) => jsConf.rawFunction should contain(jsReturn)
}
}
}
}

object EnrichmentRegistrySpec {

def jsEnrichment(jsReturn: String = "defaultReturn"): Json = {
val script = s"""
function process(event) {
return $jsReturn;
}
"""

json"""{
"schema": "iglu:com.snowplowanalytics.snowplow/javascript_script_config/jsonschema/1-0-0",
"data": {
"vendor": "com.snowplowanalytics.snowplow",
"name": "javascript_script_config",
"enabled": true,
"parameters": {
"script": ${ConversionUtils.encodeBase64Url(script)}
}
}
}"""
}

// Vendor and name are intentionally tweaked in the first enrichment
// to test that we are no longer validating them (users were confused about such validation)
def enrichmentConfig(additionals: List[Json] = List.empty) = {
val enrichmentArr = json"""[
{
"schema": "iglu:com.snowplowanalytics.snowplow/anon_ip/jsonschema/1-0-0",
"data": {
"vendor": "com.snowplowanalytics.snowplow_custom",
"name": "anon_ip_custom",
"enabled": true,
"parameters": {
"anonOctets": 1
}
}
},
{
"schema": "iglu:com.snowplowanalytics.snowplow/campaign_attribution/jsonschema/1-0-0",
"data": {
"vendor": "com.snowplowanalytics.snowplow",
"name": "campaign_attribution",
"enabled": true,
"parameters": {
"mapping": "static",
"fields": {
"mktMedium": ["utm_medium", "medium"],
"mktSource": ["utm_source", "source"],
"mktTerm": ["utm_term", "legacy_term"],
"mktContent": ["utm_content"],
"mktCampaign": ["utm_campaign", "cid", "legacy_campaign"]
}
}
}
},
{
"schema": "iglu:com.snowplowanalytics.snowplow/user_agent_utils_config/jsonschema/1-0-0",
"data": {
"vendor": "com.snowplowanalytics.snowplow",
"name": "user_agent_utils_config",
"enabled": true,
"parameters": {}
}
},
{
"schema": "iglu:com.snowplowanalytics.snowplow/referer_parser/jsonschema/2-0-0",
"data": {
"vendor": "com.snowplowanalytics.snowplow",
"name": "referer_parser",
"enabled": true,
"parameters": {
"internalDomains": ["www.subdomain1.snowplowanalytics.com"],
"database": "referer-tests.json",
"uri": "http://snowplow.com"
}
}
}
]""".asArray.map(_.toList).getOrElse(List.empty) ::: additionals
json"""{
"schema": "iglu:com.snowplowanalytics.snowplow/enrichments/jsonschema/1-0-0",
"data": $enrichmentArr
}"""
}
}

0 comments on commit fad74df

Please sign in to comment.