From 87636e90ea46615b8464681ad058dec5d47356ed Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Piotr=20Poniedzia=C5=82ek?= Date: Thu, 25 May 2023 15:57:29 +0200 Subject: [PATCH 01/14] Add azure module with kafka + blob storage implementation --- build.sbt | 8 ++ .../rdbloader/azure/AzureBlobStorage.scala | 106 ++++++++++++++++++ .../rdbloader/azure/KafkaConsumer.scala | 60 ++++++++++ .../rdbloader/azure/KafkaProducer.scala | 72 ++++++++++++ .../rdbloader/common/cloud/BlobStorage.scala | 6 +- project/BuildSettings.scala | 4 + project/Dependencies.scala | 10 ++ 7 files changed, 263 insertions(+), 3 deletions(-) create mode 100644 modules/azure/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/azure/AzureBlobStorage.scala create mode 100644 modules/azure/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/azure/KafkaConsumer.scala create mode 100644 modules/azure/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/azure/KafkaProducer.scala diff --git a/build.sbt b/build.sbt index 1d73386c2..1f8195e59 100755 --- a/build.sbt +++ b/build.sbt @@ -29,6 +29,7 @@ lazy val root = project.in(file(".")) .aggregate( aws, gcp, + azure, common, commonTransformerStream, loader, @@ -60,6 +61,13 @@ lazy val gcp: Project = project .dependsOn(common % "compile->compile;test->test") .enablePlugins(BuildInfoPlugin) +lazy val azure: Project = project + .in(file("modules/azure")) + .settings(BuildSettings.azureBuildSettings) + .settings(libraryDependencies ++= Dependencies.azureDependencies) + .dependsOn(common % "compile->compile;test->test") + .enablePlugins(BuildInfoPlugin) + lazy val commonTransformerStream = project .in(file("modules/common-transformer-stream")) .settings(BuildSettings.commonStreamTransformerBuildSettings) diff --git a/modules/azure/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/azure/AzureBlobStorage.scala b/modules/azure/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/azure/AzureBlobStorage.scala new file mode 100644 index 000000000..4cf250b05 --- /dev/null +++ b/modules/azure/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/azure/AzureBlobStorage.scala @@ -0,0 +1,106 @@ +/* + * Copyright (c) 2012-2023 Snowplow Analytics Ltd. All rights reserved. + * + * This program is licensed to you under the Apache License Version 2.0, + * and you may not use this file except in compliance with the Apache License Version 2.0. + * You may obtain a copy of the Apache License Version 2.0 at + * http://www.apache.org/licenses/LICENSE-2.0. + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the Apache License Version 2.0 is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the Apache License Version 2.0 for the specific language governing permissions and + * limitations there under. + */ +package com.snowplowanalytics.snowplow.rdbloader.aws + +import blobstore.azure.{AzureBlob, AzureStore} +import cats.effect._ +import cats.implicits._ +import fs2.{Pipe, Stream} +import blobstore.url.{Authority, Path, Url} +import blobstore.url.exception.{MultipleUrlValidationException, Throwables} +import com.azure.identity.DefaultAzureCredentialBuilder +import com.azure.storage.blob.BlobServiceClientBuilder +import com.snowplowanalytics.snowplow.rdbloader.common.cloud.BlobStorage +import com.snowplowanalytics.snowplow.rdbloader.common.cloud.BlobStorage.{Folder, Key} + +class AzureBlobStorage[F[_]: Async] private (store: AzureStore[F]) extends BlobStorage[F] { + + override def list(folder: Folder, recursive: Boolean): Stream[F, BlobStorage.BlobObject] = { + val (authority, path) = BlobStorage.splitPath(folder) + Authority + .parse(authority) + .fold( + errors => Stream.raiseError[F](new MultipleUrlValidationException(errors)), + authority => + // TODO + // fs2-blobstore uses 'authority' as a container name https://github.com/fs2-blobstore/fs2-blobstore/blob/a95a8a43ed6d4b7cfac73d85aa52356028256110/azure/src/main/scala/blobstore/azure/AzureStore.scala#L294 + // is that correct...? It seems 'authority' is already provided when we build client with 'endpoint' method and value like "{accountName}.blob.core.windows.net" + store + .list(Url("https", authority, Path(path)), recursive) + .map { url: Url[AzureBlob] => + val bucketName = url.authority.show + val keyPath = url.path.relative.show + val key = BlobStorage.Key.coerce(s"https://$bucketName/$keyPath") + BlobStorage.BlobObject(key, url.path.representation.size.getOrElse(0L)) + } + ) + } + + override def put(key: Key, overwrite: Boolean): Pipe[F, Byte, Unit] = { + val (authority, path) = BlobStorage.splitKey(key) + Authority + .parse(authority) + .fold( + errors => _ => Stream.raiseError[F](new MultipleUrlValidationException(errors)), + authority => store.put(Url("https", authority, Path(path)), overwrite) + ) + } + + override def get(key: Key): F[Either[Throwable, String]] = { + val (authority, path) = BlobStorage.splitKey(key) + Authority + .parse(authority) + .fold( + errors => Async[F].delay(new MultipleUrlValidationException(errors).asLeft[String]), + authority => + store + .get(Url("https", authority, Path(path)), 1024) + .compile + .to(Array) + .map(array => new String(array)) + .attempt + ) + } + + override def keyExists(key: Key): F[Boolean] = { + val (authority, path) = BlobStorage.splitKey(key) + Authority + .parse(authority) + .fold( + errors => Async[F].raiseError(new MultipleUrlValidationException(errors)), + authority => store.list(Url("https", authority, Path(path))).compile.toList.map(_.nonEmpty) + ) + } +} + +object AzureBlobStorage { + + def create[F[_]: Async](): Resource[F, BlobStorage[F]] = + createStore().map(new AzureBlobStorage(_)) + + private def createStore[F[_]: Async](): Resource[F, AzureStore[F]] = { + val credentials = new DefaultAzureCredentialBuilder().build + val client = new BlobServiceClientBuilder() + // TODO Do we need to pass 'endpoint' here? + // .endpoint("https://{accountName}.blob.core.windows.net") + .credential(credentials) + .buildAsyncClient() + + AzureStore + .builder[F](client) + .build + .fold(errors => Resource.raiseError(errors.reduce(Throwables.collapsingSemigroup)), Resource.pure) + } +} diff --git a/modules/azure/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/azure/KafkaConsumer.scala b/modules/azure/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/azure/KafkaConsumer.scala new file mode 100644 index 000000000..5d982d431 --- /dev/null +++ b/modules/azure/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/azure/KafkaConsumer.scala @@ -0,0 +1,60 @@ +/* + * Copyright (c) 2012-2023 Snowplow Analytics Ltd. All rights reserved. + * + * This program is licensed to you under the Apache License Version 2.0, + * and you may not use this file except in compliance with the Apache License Version 2.0. + * You may obtain a copy of the Apache License Version 2.0 at + * http://www.apache.org/licenses/LICENSE-2.0. + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the Apache License Version 2.0 is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the Apache License Version 2.0 for the specific language governing permissions and + * limitations there under. + */ +package com.snowplowanalytics.snowplow.rdbloader.azure + +import cats.effect.{Async, Resource} +import cats.implicits._ +import com.snowplowanalytics.snowplow.rdbloader.common.cloud.Queue +import com.snowplowanalytics.snowplow.rdbloader.common.cloud.Queue.Consumer +import fs2.kafka.{CommittableConsumerRecord, ConsumerSettings, KafkaConsumer => Fs2KafkaConsumer} +import org.typelevel.log4cats.Logger + +import java.nio.charset.StandardCharsets + +object KafkaConsumer { + + private final case class KafkaMessage[F[_]](record: CommittableConsumerRecord[F, String, Array[Byte]]) extends Queue.Consumer.Message[F] { + override def content: String = new String(record.record.value, StandardCharsets.UTF_8) + override def ack: F[Unit] = record.offset.commit + } + + def consumer[F[_]: Async: Logger]( + bootstrapServers: String, + topicName: String, + consumerConf: Map[String, String] + ): Resource[F, Queue.Consumer[F]] = { + val consumerSettings = + ConsumerSettings[F, String, Array[Byte]] + .withBootstrapServers(bootstrapServers) + .withProperties(consumerConf) + .withEnableAutoCommit(false) + .withProperties( + ("key.deserializer", "org.apache.kafka.common.serialization.StringDeserializer"), + ("value.deserializer", "org.apache.kafka.common.serialization.ByteArrayDeserializer") + ) + + Fs2KafkaConsumer[F] + .resource(consumerSettings) + .evalMap { consumer => + consumer.subscribeTo(topicName) *> Async[F].pure { + new Queue.Consumer[F] { + override def read: fs2.Stream[F, Consumer.Message[F]] = + consumer.records + .map(KafkaMessage(_)) + } + } + } + } +} diff --git a/modules/azure/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/azure/KafkaProducer.scala b/modules/azure/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/azure/KafkaProducer.scala new file mode 100644 index 000000000..b54507f76 --- /dev/null +++ b/modules/azure/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/azure/KafkaProducer.scala @@ -0,0 +1,72 @@ +/* + * Copyright (c) 2012-2023 Snowplow Analytics Ltd. All rights reserved. + * + * This program is licensed to you under the Apache License Version 2.0, + * and you may not use this file except in compliance with the Apache License Version 2.0. + * You may obtain a copy of the Apache License Version 2.0 at + * http://www.apache.org/licenses/LICENSE-2.0. + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the Apache License Version 2.0 is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the Apache License Version 2.0 for the specific language governing permissions and + * limitations there under. + */ +package com.snowplowanalytics.snowplow.rdbloader.azure + +import cats.effect.{Async, Resource} +import cats.implicits._ +import com.snowplowanalytics.snowplow.rdbloader.common.cloud.Queue +import fs2.kafka.{KafkaProducer => Fs2KafkaProducer, ProducerRecord, ProducerSettings} +import org.typelevel.log4cats.Logger + +import java.nio.charset.StandardCharsets +import java.util.UUID + +object KafkaProducer { + + def producer[F[_]: Async: Logger]( + bootstrapServers: String, + topicName: String, + producerConf: Map[String, String] + ): Resource[F, Queue.Producer[F]] = { + val producerSettings = + ProducerSettings[F, String, Array[Byte]] + .withBootstrapServers(bootstrapServers) + .withProperties(producerConf) + .withProperties( + ("key.serializer", "org.apache.kafka.common.serialization.StringSerializer"), + ("value.serializer", "org.apache.kafka.common.serialization.ByteArraySerializer") + ) + + Fs2KafkaProducer[F].resource(producerSettings).map { producer => + new Queue.Producer[F] { + override def send(message: String): F[Unit] = + producer + .produceOne_(toProducerRecord(topicName, message)) + .flatten + .void + } + } + } + + def chunkProducer[F[_]: Async: Logger]( + bootstrapServers: String, + topicName: String, + producerConf: Map[String, String] + ): Resource[F, Queue.ChunkProducer[F]] = + producer(bootstrapServers, topicName, producerConf) + .map { producer => + new Queue.ChunkProducer[F] { + override def send(messages: List[String]): F[Unit] = + messages.traverse_(producer.send) + } + } + + private def toProducerRecord(topicName: String, message: String): ProducerRecord[String, Array[Byte]] = + ProducerRecord( + topicName, + UUID.randomUUID().toString, + message.getBytes(StandardCharsets.UTF_8) + ) +} diff --git a/modules/common/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/common/cloud/BlobStorage.scala b/modules/common/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/common/cloud/BlobStorage.scala index 4e55f66eb..1d85a736e 100644 --- a/modules/common/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/common/cloud/BlobStorage.scala +++ b/modules/common/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/common/cloud/BlobStorage.scala @@ -88,7 +88,7 @@ object BlobStorage { object Folder extends tag.Tagger[BlobStorageFolderTag] { def parse(s: String): Either[String, Folder] = s match { - case _ if !correctlyPrefixed(s) => s"Bucket name $s doesn't start with s3:// s3a:// s3n:// or gs:// prefix".asLeft + case _ if !correctlyPrefixed(s) => s"Bucket name $s doesn't start with s3:// s3a:// s3n:// gs:// or https:// prefix".asLeft case _ if s.length > 1024 => "Key length cannot be more than 1024 symbols".asLeft case _ => coerce(s).asRight } @@ -117,7 +117,7 @@ object BlobStorage { * Extract `xx://path/run=YYYY-MM-dd-HH-mm-ss/atomic-events` part from Set of prefixes that can be * used in config.yml In the end it won't affect how blob storage is accessed */ - val supportedPrefixes = Set("s3", "s3n", "s3a", "gs") + val supportedPrefixes = Set("s3", "s3n", "s3a", "gs", "https") private def correctlyPrefixed(s: String): Boolean = supportedPrefixes.foldLeft(false) { (result, prefix) => @@ -146,7 +146,7 @@ object BlobStorage { fixPrefix(s).asInstanceOf[Key] def parse(s: String): Either[String, Key] = s match { - case _ if !correctlyPrefixed(s) => s"Bucket name $s doesn't start with s3:// s3a:// s3n:// or gs:// prefix".asLeft + case _ if !correctlyPrefixed(s) => s"Bucket name $s doesn't start with s3:// s3a:// s3n:// gs:// https:// prefix".asLeft case _ if s.length > 1024 => "Key length cannot be more than 1024 symbols".asLeft case _ if s.endsWith("/") => "Blob storage key cannot have trailing slash".asLeft case _ => coerce(s).asRight diff --git a/project/BuildSettings.scala b/project/BuildSettings.scala index 8deedc3ff..f651ee73f 100644 --- a/project/BuildSettings.scala +++ b/project/BuildSettings.scala @@ -180,6 +180,10 @@ object BuildSettings { lazy val gcpBuildSettings = { buildSettings } + + lazy val azureBuildSettings = { + buildSettings + } lazy val commonBuildSettings = { Seq( diff --git a/project/Dependencies.scala b/project/Dependencies.scala index ee1292125..771f44cdb 100644 --- a/project/Dependencies.scala +++ b/project/Dependencies.scala @@ -32,6 +32,7 @@ object Dependencies { val fs2Blobstore = "0.9.7" val fs2Cron = "0.7.2" val fs2PubSub = "0.21.0" + val fs2Kafka = "3.0.0" val secretManager = "2.7.0" val gcpStorage = "2.16.0" val doobie = "1.0.0-RC2" @@ -99,11 +100,14 @@ object Dependencies { val cron4sCirce = ("com.github.alonsodomin.cron4s" %% "cron4s-circe" % V.cron4sCirce) .exclude("io.circe", "circe-core_2.12") // cron4s-circe lacks circe 0.13 support val fs2 = "co.fs2" %% "fs2-core" % V.fs2 + val fs2Kafka = "com.github.fd4s" %% "fs2-kafka" % V.fs2Kafka val fs2Kinesis = ("io.laserdisc" %% "fs2-aws-kinesis" % V.fs2Aws) .exclude("com.amazonaws", "amazon-kinesis-producer") .exclude("software.amazon.kinesis", "amazon-kinesis-client") val fs2BlobstoreS3 = "com.github.fs2-blobstore" %% "s3" % V.fs2Blobstore val fs2BlobstoreGCS = "com.github.fs2-blobstore" %% "gcs" % V.fs2Blobstore + val fs2BlobstoreAzure = "com.github.fs2-blobstore" %% "azure" % V.fs2Blobstore + val azureIdentity = "com.azure" % "azure-identity" % "1.9.0" val fs2Cron = "eu.timepit" %% "fs2-cron-cron4s" % V.fs2Cron val fs2PubSub = "com.permutive" %% "fs2-google-pubsub-grpc" % V.fs2PubSub val secretManager = "com.google.cloud" % "google-cloud-secretmanager" % V.secretManager @@ -206,6 +210,12 @@ object Dependencies { gcpStorage ) + val azureDependencies = Seq( + fs2BlobstoreAzure, + azureIdentity, + fs2Kafka + ) + val commonDependencies = Seq( decline, analyticsSdk, From d7b3ecf3e04a537e6d751adc6597e85a00cca5cf Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Piotr=20Poniedzia=C5=82ek?= Date: Fri, 26 May 2023 16:24:29 +0200 Subject: [PATCH 02/14] Add transformer-kafka module --- build.sbt | 11 ++ .../transformer.kafka.config.minimal.hocon | 13 ++ .../transformer.kafka.config.reference.hocon | 153 ++++++++++++++++++ .../rdbloader/azure/KafkaConsumer.scala | 5 +- .../rdbloader/azure/KafkaProducer.scala | 5 +- .../transformer/stream/common/Config.scala | 67 +++++++- .../common/processing/TestApplication.scala | 1 + .../src/main/resources/application.conf | 38 +++++ .../stream/kafka/KafkaCheckpointer.scala | 44 +++++ .../transformer/stream/kafka/Main.scala | 89 ++++++++++ .../transformer/stream/kafka/ConfigSpec.scala | 140 ++++++++++++++++ project/BuildSettings.scala | 10 ++ project/Dependencies.scala | 5 + 13 files changed, 571 insertions(+), 10 deletions(-) create mode 100644 config/transformer/azure/transformer.kafka.config.minimal.hocon create mode 100644 config/transformer/azure/transformer.kafka.config.reference.hocon create mode 100644 modules/transformer-kafka/src/main/resources/application.conf create mode 100644 modules/transformer-kafka/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/kafka/KafkaCheckpointer.scala create mode 100644 modules/transformer-kafka/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/kafka/Main.scala create mode 100644 modules/transformer-kafka/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/kafka/ConfigSpec.scala diff --git a/build.sbt b/build.sbt index 1f8195e59..c2b4c3435 100755 --- a/build.sbt +++ b/build.sbt @@ -39,6 +39,7 @@ lazy val root = project.in(file(".")) transformerBatch, transformerKinesis, transformerPubsub, + transformerKafka ) lazy val common: Project = project @@ -177,3 +178,13 @@ lazy val transformerPubsubDistroless = project .settings(excludeDependencies ++= Dependencies.commonStreamTransformerExclusions) .dependsOn(commonTransformerStream % "compile->compile;test->test;runtime->runtime", gcp % "compile->compile;test->test") .enablePlugins(JavaAppPackaging, SnowplowDistrolessDockerPlugin, BuildInfoPlugin) + +lazy val transformerKafka = project + .in(file("modules/transformer-kafka")) + .settings(BuildSettings.transformerKafkaBuildSettings) + .settings(addCompilerPlugin(Dependencies.betterMonadicFor)) + .settings(libraryDependencies ++= Dependencies.transformerKafkaDependencies) + .settings(excludeDependencies ++= Dependencies.commonStreamTransformerExclusions) + .dependsOn(commonTransformerStream % "compile->compile;test->test;runtime->runtime", azure % "compile->compile;test->test;runtime->runtime") + .enablePlugins(JavaAppPackaging, SnowplowDockerPlugin, BuildInfoPlugin) + diff --git a/config/transformer/azure/transformer.kafka.config.minimal.hocon b/config/transformer/azure/transformer.kafka.config.minimal.hocon new file mode 100644 index 000000000..f1c944be4 --- /dev/null +++ b/config/transformer/azure/transformer.kafka.config.minimal.hocon @@ -0,0 +1,13 @@ +{ + "input": { + "topicName": "enriched" + "bootstrapServers": "localhost:9092" + } + "output": { + "path": "https://accountName.blob.core.windows.net/transformed/" + } + "queue": { + "topicName": "loaderTopic" + "bootstrapServers": "localhost:9092" + } +} diff --git a/config/transformer/azure/transformer.kafka.config.reference.hocon b/config/transformer/azure/transformer.kafka.config.reference.hocon new file mode 100644 index 000000000..00cd2f70f --- /dev/null +++ b/config/transformer/azure/transformer.kafka.config.reference.hocon @@ -0,0 +1,153 @@ +{ + "input": { + "type": "kafka" + + # Name of the Kafka topic to read from + "topicName": "enriched" + + # 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 Consumer configuration + # See https://kafka.apache.org/documentation/#consumerconfigs for all properties + "consumerConf": { + "enable.auto.commit": "false" + "auto.offset.reset" : "earliest" + "group.id": "transformer" + } + } + + # Path to transformed archive + "output": { + # Path to transformer output + "path": "https://accountName.blob.core.windows.net/transformed/", + # Transformer output compression, GZIP or NONE + # Optional, default value GZIP + "compression": "GZIP", + + # Optional section specifying details about badrows output. When unspecified, badrows are written as files under 'output.path' URI + "bad": { + + # Type of output sink. Either 'kafka' or 'file'. Optional, default value 'file'. When 'file', badrows are written as files under 'output.path' URI + "type": "kafka", + + # Name of the Kafka topic to write to + "topicName": "bad" + + # 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" + } + } + } + + # Frequency to emit loading finished message - 5,10,15,20,30,60 etc minutes + # Optional, default value 10 minutes + "windowing": "10 minutes" + + # Kafka topic used to communicate with Loader + "queue": { + "type": "kafka", + + # Name of the Kafka topic to write to + "topicName": "loaderTopic" + + # 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" + } + } + + "formats": { + # Optional. Denotes output file format. + # Possible values are 'json' and 'parquet'. Default value 'json'. + "fileFormat": "json" + } + + # Events will be validated against given criterias and + # bad row will be created if validation is not successful + "validations": { + "minimumTimestamp": "2021-11-18T11:00:00.00Z" + } + + # Observability and reporting options + "monitoring": { + # Optional, for tracking runtime exceptions + "sentry": { + "dsn": "http://sentry.acme.com" + } + # Optional. How metrics are reported + "metrics": { + # Optional. Send metrics to a StatsD server (e.g. on localhost) + "statsd": { + "hostname": "localhost" + "port": 8125 + "period": "1 minute" + # Optional. Any key-value pairs to be tagged on every StatsD metric + "tags": { + "app": transformer + } + # Optional. Override the default metric prefix + # "prefix": "snowplow.transformer." + } + # Optional. Log to stdout using Slf4j (logger name: transformer.metrics) + "stdout": { + "period": "1 minute" + # Optional. Override the default metric prefix + # "prefix": "snowplow.transformer." + } + } + } + + # Optional. Configure telemetry + # All the fields are optional + "telemetry": { + # Set to true to disable telemetry + "disable": false + # Interval for the heartbeat event + "interval": 15 minutes + # HTTP method used to send the heartbeat event + "method": "POST" + # URI of the collector receiving the heartbeat event + "collectorUri": "collector-g.snowplowanalytics.com" + # Port of the collector receiving the heartbeat event + "collectorPort": 443 + # Whether to use https or not + "secure": true + # Identifier intended to tie events together across modules, + # infrastructure and apps when used consistently + "userProvidedId": "my_pipeline" + # ID automatically generated upon running a modules deployment script + # Intended to identify each independent module, and the infrastructure it controls + "autoGeneratedId": "hfy67e5ydhtrd" + # Unique identifier for the VM instance + # Unique for each instance of the app running within a module + "instanceId": "665bhft5u6udjf" + # Name of the terraform module that deployed the app + "moduleName": "transformer-kafka-ce" + # Version of the terraform module that deployed the app + "moduleVersion": "1.0.0" + } + + # Optional. Enable features that are still in beta, or which are here to enable smoother upgrades + "featureFlags": { + # Read/write in the legacy version 1 shredding complete message format. + # This should be enabled during upgrade from older versions of the loader. + "legacyMessageFormat": false + + # When enabled, event's atomic fields are truncated (based on the length limits from the atomic JSON schema) before transformation. + # Optional, default "false". + "truncateAtomicFields": false + } +} diff --git a/modules/azure/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/azure/KafkaConsumer.scala b/modules/azure/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/azure/KafkaConsumer.scala index 5d982d431..44741bc7a 100644 --- a/modules/azure/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/azure/KafkaConsumer.scala +++ b/modules/azure/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/azure/KafkaConsumer.scala @@ -19,18 +19,17 @@ import cats.implicits._ import com.snowplowanalytics.snowplow.rdbloader.common.cloud.Queue import com.snowplowanalytics.snowplow.rdbloader.common.cloud.Queue.Consumer import fs2.kafka.{CommittableConsumerRecord, ConsumerSettings, KafkaConsumer => Fs2KafkaConsumer} -import org.typelevel.log4cats.Logger import java.nio.charset.StandardCharsets object KafkaConsumer { - private final case class KafkaMessage[F[_]](record: CommittableConsumerRecord[F, String, Array[Byte]]) extends Queue.Consumer.Message[F] { + final case class KafkaMessage[F[_]](record: CommittableConsumerRecord[F, String, Array[Byte]]) extends Queue.Consumer.Message[F] { override def content: String = new String(record.record.value, StandardCharsets.UTF_8) override def ack: F[Unit] = record.offset.commit } - def consumer[F[_]: Async: Logger]( + def consumer[F[_]: Async]( bootstrapServers: String, topicName: String, consumerConf: Map[String, String] diff --git a/modules/azure/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/azure/KafkaProducer.scala b/modules/azure/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/azure/KafkaProducer.scala index b54507f76..427866989 100644 --- a/modules/azure/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/azure/KafkaProducer.scala +++ b/modules/azure/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/azure/KafkaProducer.scala @@ -18,14 +18,13 @@ import cats.effect.{Async, Resource} import cats.implicits._ import com.snowplowanalytics.snowplow.rdbloader.common.cloud.Queue import fs2.kafka.{KafkaProducer => Fs2KafkaProducer, ProducerRecord, ProducerSettings} -import org.typelevel.log4cats.Logger import java.nio.charset.StandardCharsets import java.util.UUID object KafkaProducer { - def producer[F[_]: Async: Logger]( + def producer[F[_]: Async]( bootstrapServers: String, topicName: String, producerConf: Map[String, String] @@ -50,7 +49,7 @@ object KafkaProducer { } } - def chunkProducer[F[_]: Async: Logger]( + def chunkProducer[F[_]: Async]( bootstrapServers: String, topicName: String, producerConf: Map[String, String] diff --git a/modules/common-transformer-stream/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/common/Config.scala b/modules/common-transformer-stream/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/common/Config.scala index b34d77aa3..e73192c0f 100644 --- a/modules/common-transformer-stream/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/common/Config.scala +++ b/modules/common-transformer-stream/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/common/Config.scala @@ -84,6 +84,12 @@ object Config { throw new IllegalArgumentException(s"Subscription format $subscription invalid") } } + + final case class Kafka( + topicName: String, + bootstrapServers: String, + consumerConf: Map[String, String] + ) extends StreamInput } sealed trait Output { @@ -112,6 +118,14 @@ object Config { bad: Bad ) extends Output + final case class AzureBlobStorage( + path: URI, + compression: Compression, + bufferSize: Int, + maxRecordsPerFile: Long, + bad: Bad + ) extends Output + sealed trait Bad object Bad { @@ -152,6 +166,12 @@ object Config { throw new IllegalArgumentException(s"Subscription format $topic invalid") } } + + final case class Kafka( + topicName: String, + bootstrapServers: String, + producerConf: Map[String, String] + ) extends Queue } } @@ -178,6 +198,12 @@ object Config { throw new IllegalArgumentException(s"Subscription format $topic invalid") } } + + final case class Kafka( + topicName: String, + bootstrapServers: String, + producerConf: Map[String, String] + ) extends QueueConfig } final case class Monitoring(sentry: Option[TransformerConfig.Sentry], metrics: MetricsReporters) @@ -232,8 +258,15 @@ object Config { cur.as[StreamInput.Kinesis] case Right("pubsub") => cur.as[StreamInput.Pubsub] + case Right("kafka") => + cur.as[StreamInput.Kafka] case Right(other) => - Left(DecodingFailure(s"Shredder input type $other is not supported yet. Supported types: 'kinesis', 'pubsub'", typeCur.history)) + Left( + DecodingFailure( + s"Shredder input type $other is not supported yet. Supported types: 'kinesis', 'pubsub', 'kafka'", + typeCur.history + ) + ) case Left(DecodingFailure(_, List(CursorOp.DownField("type")))) => Left(DecodingFailure("Cannot find 'type' string in transformer configuration", typeCur.history)) case Left(other) => @@ -247,6 +280,9 @@ object Config { implicit val streamInputPubsubConfigDecoder: Decoder[StreamInput.Pubsub] = deriveDecoder[StreamInput.Pubsub] + implicit val streamInputKafkaConfigDecoder: Decoder[StreamInput.Kafka] = + deriveDecoder[StreamInput.Kafka] + implicit val outputConfigDecoder: Decoder[Output] = Decoder.instance { cur => val pathCur = cur.downField("path") @@ -255,9 +291,14 @@ object Config { cur.as[Output.S3] case Right("gs") => cur.as[Output.GCS] + case Right("https") => + cur.as[Output.AzureBlobStorage] case Right(other) => Left( - DecodingFailure(s"Output type $other is not supported yet. Supported types: 's3', 's3a', 's3n', and 'gs'", pathCur.history) + DecodingFailure( + s"Output type $other is not supported yet. Supported types: 's3', 's3a', 's3n', 'gs', 'https'", + pathCur.history + ) ) case Left(DecodingFailure(_, List(CursorOp.DownField("type")))) => Left(DecodingFailure("Cannot find 'path' string in output configuration", pathCur.history)) @@ -272,6 +313,9 @@ object Config { implicit val pubsubBadOutputConfigDecoder: Decoder[Output.Bad.Queue.Pubsub] = deriveDecoder[Output.Bad.Queue.Pubsub] + implicit val kafkaBadOutputConfigDecoder: Decoder[Output.Bad.Queue.Kafka] = + deriveDecoder[Output.Bad.Queue.Kafka] + implicit val badOutputConfigDecoder: Decoder[Output.Bad] = Decoder.instance { cur => val typeCur = cur.downField("type") @@ -280,12 +324,14 @@ object Config { cur.as[Output.Bad.Queue.Kinesis] case Right("pubsub") => cur.as[Output.Bad.Queue.Pubsub] + case Right("kafka") => + cur.as[Output.Bad.Queue.Kafka] case Right("file") => Right(Output.Bad.File) case Right(other) => Left( DecodingFailure( - s"Bad output type '$other' is not supported yet. Supported types: 'kinesis', 'pubsub', 'file'", + s"Bad output type '$other' is not supported yet. Supported types: 'kinesis', 'pubsub', 'kafka', 'file'", typeCur.history ) ) @@ -299,6 +345,9 @@ object Config { implicit val outputGCSConfigDecoder: Decoder[Output.GCS] = deriveDecoder[Output.GCS] + implicit val outputAzureBlobStorageConfigDecoder: Decoder[Output.AzureBlobStorage] = + deriveDecoder[Output.AzureBlobStorage] + implicit val queueConfigDecoder: Decoder[QueueConfig] = Decoder.instance { cur => val typeCur = cur.downField("type") @@ -309,8 +358,15 @@ object Config { cur.as[QueueConfig.SQS] case Right("pubsub") => cur.as[QueueConfig.Pubsub] + case Right("kafka") => + cur.as[QueueConfig.Kafka] case Right(other) => - Left(DecodingFailure(s"Queue type $other is not supported yet. Supported types: 'SNS', 'SQS' and 'pubsub'", typeCur.history)) + Left( + DecodingFailure( + s"Queue type $other is not supported yet. Supported types: 'SNS', 'SQS', 'pubsub' and 'kafka'", + typeCur.history + ) + ) case Left(DecodingFailure(_, List(CursorOp.DownField("type")))) => Left(DecodingFailure("Cannot find 'type' string in transformer configuration", typeCur.history)) case Left(other) => @@ -327,6 +383,9 @@ object Config { implicit val pubsubConfigDecoder: Decoder[QueueConfig.Pubsub] = deriveDecoder[QueueConfig.Pubsub] + implicit val kafkaConfigDecoder: Decoder[QueueConfig.Kafka] = + deriveDecoder[QueueConfig.Kafka] + implicit val configDecoder: Decoder[Config] = deriveDecoder[Config].ensure(validateConfig) diff --git a/modules/common-transformer-stream/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/common/processing/TestApplication.scala b/modules/common-transformer-stream/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/common/processing/TestApplication.scala index 001b1e170..fe71a28f5 100644 --- a/modules/common-transformer-stream/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/common/processing/TestApplication.scala +++ b/modules/common-transformer-stream/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/common/processing/TestApplication.scala @@ -159,6 +159,7 @@ object TestApplication { val updatedOutput = config.output match { case c: Config.Output.S3 => c.copy(path = URI.create(c.path.toString.replace("s3:/", "file:/"))) case c: Config.Output.GCS => c.copy(path = URI.create(c.path.toString.replace("gs:/", "file:/"))) + case c: Config.Output.AzureBlobStorage => c.copy(path = URI.create(c.path.toString.replace("https:/", "file:/"))) } config.copy(output = updatedOutput) } diff --git a/modules/transformer-kafka/src/main/resources/application.conf b/modules/transformer-kafka/src/main/resources/application.conf new file mode 100644 index 000000000..04d27675e --- /dev/null +++ b/modules/transformer-kafka/src/main/resources/application.conf @@ -0,0 +1,38 @@ +"snowplow": { + + #TODO verify defaults for kafka config!! + + "input": { + "type": "kafka" + "consumerConf": { + "enable.auto.commit": "false" + "auto.offset.reset" : "earliest" + "group.id": "transformer" + } + } + + "output": { + "bad": { + "producerConf": { + "acks": "all" + } + } + } + + "queue": { + "type": "kafka" + "producerConf": { + "acks": "all" + } + } + + "formats": { + "transformationType": "widerow" + "fileFormat": "json" + } + + "featureFlags": { + "enableMaxRecordsPerFile": true + } + +} diff --git a/modules/transformer-kafka/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/kafka/KafkaCheckpointer.scala b/modules/transformer-kafka/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/kafka/KafkaCheckpointer.scala new file mode 100644 index 000000000..6ffccf8d2 --- /dev/null +++ b/modules/transformer-kafka/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/kafka/KafkaCheckpointer.scala @@ -0,0 +1,44 @@ +/* + * Copyright (c) 2012-2023 Snowplow Analytics Ltd. All rights reserved. + * + * This program is licensed to you under the Apache License Version 2.0, + * and you may not use this file except in compliance with the Apache License Version 2.0. + * You may obtain a copy of the Apache License Version 2.0 at + * http://www.apache.org/licenses/LICENSE-2.0. + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the Apache License Version 2.0 is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the Apache License Version 2.0 for the specific language governing permissions and + * limitations there under. + */ +package com.snowplowanalytics.snowplow.rdbloader.transformer.stream.kafka + +import cats.Applicative +import cats.effect._ +import cats.implicits._ +import com.snowplowanalytics.snowplow.rdbloader.azure.KafkaConsumer.KafkaMessage +import com.snowplowanalytics.snowplow.rdbloader.common.cloud.Queue +import com.snowplowanalytics.snowplow.rdbloader.transformer.stream.common.sources.Checkpointer + +case class KafkaCheckpointer[F[_]](byPartition: Map[Int, F[Unit]]) + +object KafkaCheckpointer { + + def checkpointer[F[_]: Sync](message: Queue.Consumer.Message[F]): KafkaCheckpointer[F] = + message match { + case m: KafkaMessage[F] => KafkaCheckpointer[F](Map(m.record.record.partition -> m.ack)) + case _ => Checkpointer[F, KafkaCheckpointer[F]].empty + } + + implicit def kafkaCheckpointer[F[_]: Applicative]: Checkpointer[F, KafkaCheckpointer[F]] = + new Checkpointer[F, KafkaCheckpointer[F]] { + def checkpoint(c: KafkaCheckpointer[F]): F[Unit] = c.byPartition.values.toList.sequence_ + + def combine(older: KafkaCheckpointer[F], newer: KafkaCheckpointer[F]): KafkaCheckpointer[F] = + KafkaCheckpointer[F](byPartition = older.byPartition ++ newer.byPartition) + + def empty: KafkaCheckpointer[F] = + KafkaCheckpointer(Map.empty) + } +} diff --git a/modules/transformer-kafka/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/kafka/Main.scala b/modules/transformer-kafka/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/kafka/Main.scala new file mode 100644 index 000000000..788df40ba --- /dev/null +++ b/modules/transformer-kafka/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/kafka/Main.scala @@ -0,0 +1,89 @@ +/* + * Copyright (c) 2012-2023 Snowplow Analytics Ltd. All rights reserved. + * + * This program is licensed to you under the Apache License Version 2.0, + * and you may not use this file except in compliance with the Apache License Version 2.0. + * You may obtain a copy of the Apache License Version 2.0 at + * http://www.apache.org/licenses/LICENSE-2.0. + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the Apache License Version 2.0 is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the Apache License Version 2.0 for the specific language governing permissions and + * limitations there under. + */ +package com.snowplowanalytics.snowplow.rdbloader.transformer.stream.kafka + +import cats.effect._ +import com.snowplowanalytics.snowplow.rdbloader.aws.AzureBlobStorage +import com.snowplowanalytics.snowplow.rdbloader.azure._ +import com.snowplowanalytics.snowplow.rdbloader.common.cloud.{BlobStorage, Queue} +import com.snowplowanalytics.snowplow.rdbloader.transformer.stream.common.{Config, Run} +import com.snowplowanalytics.snowplow.rdbloader.transformer.stream.kafka.generated.BuildInfo + +import com.snowplowanalytics.snowplow.scalatracker.emitters.http4s.ceTracking + +object Main extends IOApp { + + def run(args: List[String]): IO[ExitCode] = + Run.run[IO, KafkaCheckpointer[IO]]( + args, + BuildInfo.name, + BuildInfo.version, + BuildInfo.description, + runtime.compute, + (config, _) => mkSource(config), + c => mkBlobStorage(c), + c => mkBadQueue(c), + mkShreddingCompleteQueue, + KafkaCheckpointer.checkpointer + ) + + private def mkSource[F[_]: Async]( + streamInput: Config.StreamInput + ): Resource[F, Queue.Consumer[F]] = + streamInput match { + case conf: Config.StreamInput.Kafka => + KafkaConsumer.consumer[F]( + conf.bootstrapServers, + conf.topicName, + conf.consumerConf + ) + case _ => + Resource.eval(Async[F].raiseError(new IllegalArgumentException(s"Input is not Kafka"))) + } + + private def mkBlobStorage[F[_]: Async](output: Config.Output): Resource[F, BlobStorage[F]] = + output match { + case _: Config.Output.AzureBlobStorage => + AzureBlobStorage.create[F]() + case _ => + Resource.eval(Async[F].raiseError(new IllegalArgumentException(s"Output is not Azure Blob Storage"))) + } + + private def mkBadQueue[F[_]: Async]( + output: Config.Output.Bad.Queue + ): Resource[F, Queue.ChunkProducer[F]] = + output match { + case kafka: Config.Output.Bad.Queue.Kafka => + KafkaProducer.chunkProducer[F]( + kafka.bootstrapServers, + kafka.topicName, + kafka.producerConf + ) + case _ => + Resource.eval(Async[F].raiseError(new IllegalArgumentException(s"Output queue is not Kafka"))) + } + + private def mkShreddingCompleteQueue[F[_]: Async](queueConfig: Config.QueueConfig): Resource[F, Queue.Producer[F]] = + queueConfig match { + case kafka: Config.QueueConfig.Kafka => + KafkaProducer.producer[F]( + kafka.bootstrapServers, + kafka.topicName, + kafka.producerConf + ) + case _ => + Resource.eval(Async[F].raiseError(new IllegalArgumentException(s"Message queue is not Kafka"))) + } +} diff --git a/modules/transformer-kafka/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/kafka/ConfigSpec.scala b/modules/transformer-kafka/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/kafka/ConfigSpec.scala new file mode 100644 index 000000000..7b1ad7d37 --- /dev/null +++ b/modules/transformer-kafka/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/kafka/ConfigSpec.scala @@ -0,0 +1,140 @@ +/* + * Copyright (c) 2012-2023 Snowplow Analytics Ltd. All rights reserved. + * + * This program is licensed to you under the Apache License Version 2.0, + * and you may not use this file except in compliance with the Apache License Version 2.0. + * You may obtain a copy of the Apache License Version 2.0 at http://www.apache.org/licenses/LICENSE-2.0. + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the Apache License Version 2.0 is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the Apache License Version 2.0 for the specific language governing permissions and limitations there under. + */ +package com.snowplowanalytics.snowplow.rdbloader.transformer.stream.kafka + +import cats.effect.IO +import cats.effect.unsafe.implicits.global +import com.snowplowanalytics.snowplow.badrows.Processor +import com.snowplowanalytics.snowplow.rdbloader.common.config.TransformerConfig +import com.snowplowanalytics.snowplow.rdbloader.common.config.TransformerConfig.Validations +import com.snowplowanalytics.snowplow.rdbloader.common.telemetry.Telemetry +import com.snowplowanalytics.snowplow.rdbloader.generated.BuildInfo +import com.snowplowanalytics.snowplow.rdbloader.transformer.stream.common.Config +import com.snowplowanalytics.snowplow.rdbloader.transformer.stream.common.ConfigUtils._ +import org.specs2.mutable.Specification + +import java.net.URI +import java.time.Instant +import scala.concurrent.duration._ + +class ConfigSpec extends Specification { + import ConfigSpec._ + + "config fromString" should { + "be able to parse extended transformer-kafka config" in { + val result = + getConfigFromResource("/transformer/azure/transformer.kafka.config.reference.hocon", c => Config.parse[IO](c).value.unsafeRunSync()) + val expected = Config( + exampleStreamInput, + exampleWindowPeriod, + exampleOutput, + exampleQueueConfig, + TransformerConfig.Formats.WideRow.JSON, + exampleMonitoringStream, + exampleTelemetry, + exampleDefaultFeatureFlags, + exampleValidations + ) + result must beRight(expected) + } + + "be able to parse minimal transformer-kafka config" in { + val result = getConfigFromResource("/transformer/azure/transformer.kafka.config.minimal.hocon", testParseStreamConfig) + val expected = Config( + exampleStreamInput, + exampleWindowPeriod, + exampleDefaultOutput, + exampleQueueConfig, + TransformerConfig.Formats.WideRow.JSON, + exampleDefaultMonitoringStream, + defaultTelemetry, + exampleDefaultFeatureFlags, + emptyValidations + ) + result must beRight(expected) + } + } + +} + +object ConfigSpec { + val exampleStreamInput = Config.StreamInput.Kafka( + "enriched", + "localhost:9092", + Map("enable.auto.commit" -> "false", "auto.offset.reset" -> "earliest", "group.id" -> "transformer") + ) + val exampleWindowPeriod = 10.minutes + val exampleOutput = Config.Output.AzureBlobStorage( + URI.create("https://accountName.blob.core.windows.net/transformed/"), + TransformerConfig.Compression.Gzip, + 4096, + 10000, + Config.Output.Bad.Queue.Kafka( + "bad", + "localhost:9092", + Map("acks" -> "all") + ) + ) + val exampleDefaultOutput = exampleOutput.copy(bad = Config.Output.Bad.File) + + val exampleQueueConfig = Config.QueueConfig.Kafka( + "loaderTopic", + "localhost:9092", + Map("acks" -> "all") + ) + val exampleFormats = TransformerConfig.Formats.WideRow.JSON + val exampleMonitoringStream = Config.Monitoring( + Some(TransformerConfig.Sentry(URI.create("http://sentry.acme.com"))), + Config.MetricsReporters( + Some(Config.MetricsReporters.StatsD("localhost", 8125, Map("app" -> "transformer"), 1.minute, None)), + Some(Config.MetricsReporters.Stdout(1.minutes, None)), + true + ) + ) + val exampleDefaultMonitoringStream = Config.Monitoring( + None, + Config.MetricsReporters(None, Some(Config.MetricsReporters.Stdout(1.minutes, None)), true) + ) + val exampleTelemetry = + Telemetry.Config( + false, + 15.minutes, + "POST", + "collector-g.snowplowanalytics.com", + 443, + true, + Some("my_pipeline"), + Some("hfy67e5ydhtrd"), + Some("665bhft5u6udjf"), + Some("transformer-kafka-ce"), + Some("1.0.0") + ) + val defaultTelemetry = + Telemetry.Config( + false, + 15.minutes, + "POST", + "collector-g.snowplowanalytics.com", + 443, + true, + None, + None, + None, + None, + None + ) + val exampleDefaultFeatureFlags = TransformerConfig.FeatureFlags(false, None, true, false) + val exampleValidations = Validations(Some(Instant.parse("2021-11-18T11:00:00.00Z"))) + val emptyValidations = Validations(None) + val TestProcessor = Processor(BuildInfo.name, BuildInfo.version) +} diff --git a/project/BuildSettings.scala b/project/BuildSettings.scala index f651ee73f..42bb616fc 100644 --- a/project/BuildSettings.scala +++ b/project/BuildSettings.scala @@ -288,4 +288,14 @@ object BuildSettings { ) ++ buildSettings ++ assemblySettings ++ dynVerSettings ++ addExampleConfToTestCp } + lazy val transformerKafkaBuildSettings = + Seq( + name := "snowplow-transformer-kafka", + Docker / packageName := "transformer-kafka", + buildInfoPackage := "com.snowplowanalytics.snowplow.rdbloader.transformer.stream.kafka.generated", + buildInfoKeys := List(name, version, description) + ) ++ buildSettings ++ assemblySettings ++ dynVerSettings ++ addExampleConfToTestCp + + + } diff --git a/project/Dependencies.scala b/project/Dependencies.scala index 771f44cdb..e5bbf55af 100644 --- a/project/Dependencies.scala +++ b/project/Dependencies.scala @@ -142,6 +142,7 @@ object Dependencies { val hadoopAws = ("org.apache.hadoop" % "hadoop-aws" % V.hadoopClient % Runtime) .exclude("com.amazonaws", "aws-java-sdk-bundle") // aws-java-sdk-core is already present in assembled jar val hadoopGcp = "com.google.cloud.bigdataoss" % "gcs-connector" % V.hadoopGcpClient % Runtime + val hadoopAzure = "org.apache.hadoop" % "hadoop-azure" % "3.3.5" % Runtime val kinesisClient = ("software.amazon.kinesis" % "amazon-kinesis-client" % V.kinesisClient) .exclude("software.amazon.glue", "schema-registry-common") .exclude("software.amazon.glue", "schema-registry-serde") @@ -328,6 +329,10 @@ object Dependencies { hadoopGcp ) + val transformerKafkaDependencies = Seq( + hadoopAzure + ) + val commonStreamTransformerExclusions = Seq( ExclusionRule(organization = "ch.qos.logback"), From 7cadd05d84b0b798a1f4a8cb33827bc9abf5c570 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Piotr=20Poniedzia=C5=82ek?= Date: Tue, 30 May 2023 09:07:34 +0200 Subject: [PATCH 03/14] transfomer-kafka: blob storage improvements --- .../rdbloader/azure/AzureBlobStorage.scala | 140 +++++++++--------- .../transformer/stream/common/Config.scala | 4 +- .../common/processing/TestApplication.scala | 2 +- .../rdbloader/common/cloud/BlobStorage.scala | 6 +- .../transformer/stream/kafka/Main.scala | 60 ++------ .../transformer/stream/kafka/Queues.scala | 63 ++++++++ .../stream/kafka/AzuriteBasedDevApp.scala | 115 ++++++++++++++ 7 files changed, 267 insertions(+), 123 deletions(-) create mode 100644 modules/transformer-kafka/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/kafka/Queues.scala create mode 100644 modules/transformer-kafka/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/kafka/AzuriteBasedDevApp.scala diff --git a/modules/azure/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/azure/AzureBlobStorage.scala b/modules/azure/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/azure/AzureBlobStorage.scala index 4cf250b05..881eb98c9 100644 --- a/modules/azure/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/azure/AzureBlobStorage.scala +++ b/modules/azure/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/azure/AzureBlobStorage.scala @@ -12,95 +12,103 @@ * See the Apache License Version 2.0 for the specific language governing permissions and * limitations there under. */ -package com.snowplowanalytics.snowplow.rdbloader.aws +package com.snowplowanalytics.snowplow.rdbloader.azure import blobstore.azure.{AzureBlob, AzureStore} +import blobstore.url.exception.{AuthorityParseError, MultipleUrlValidationException, Throwables} +import blobstore.url.{Authority, Path, Url} +import cats.data.Validated.{Invalid, Valid} +import cats.data.ValidatedNec import cats.effect._ import cats.implicits._ -import fs2.{Pipe, Stream} -import blobstore.url.{Authority, Path, Url} -import blobstore.url.exception.{MultipleUrlValidationException, Throwables} import com.azure.identity.DefaultAzureCredentialBuilder -import com.azure.storage.blob.BlobServiceClientBuilder +import com.azure.storage.blob.{BlobServiceAsyncClient, BlobServiceClientBuilder} import com.snowplowanalytics.snowplow.rdbloader.common.cloud.BlobStorage import com.snowplowanalytics.snowplow.rdbloader.common.cloud.BlobStorage.{Folder, Key} +import fs2.{Pipe, Stream} -class AzureBlobStorage[F[_]: Async] private (store: AzureStore[F]) extends BlobStorage[F] { +import java.net.URI - override def list(folder: Folder, recursive: Boolean): Stream[F, BlobStorage.BlobObject] = { - val (authority, path) = BlobStorage.splitPath(folder) - Authority - .parse(authority) - .fold( - errors => Stream.raiseError[F](new MultipleUrlValidationException(errors)), - authority => - // TODO - // fs2-blobstore uses 'authority' as a container name https://github.com/fs2-blobstore/fs2-blobstore/blob/a95a8a43ed6d4b7cfac73d85aa52356028256110/azure/src/main/scala/blobstore/azure/AzureStore.scala#L294 - // is that correct...? It seems 'authority' is already provided when we build client with 'endpoint' method and value like "{accountName}.blob.core.windows.net" - store - .list(Url("https", authority, Path(path)), recursive) - .map { url: Url[AzureBlob] => - val bucketName = url.authority.show - val keyPath = url.path.relative.show - val key = BlobStorage.Key.coerce(s"https://$bucketName/$keyPath") - BlobStorage.BlobObject(key, url.path.representation.size.getOrElse(0L)) - } - ) - } +class AzureBlobStorage[F[_]: Async] private (store: AzureStore[F], endpoint: String) extends BlobStorage[F] { - override def put(key: Key, overwrite: Boolean): Pipe[F, Byte, Unit] = { - val (authority, path) = BlobStorage.splitKey(key) - Authority - .parse(authority) - .fold( - errors => _ => Stream.raiseError[F](new MultipleUrlValidationException(errors)), - authority => store.put(Url("https", authority, Path(path)), overwrite) - ) - } + override def list(folder: Folder, recursive: Boolean): Stream[F, BlobStorage.BlobObject] = + createStorageUrlFrom(folder) match { + case Valid(url) => + store + .list(url, recursive) + .map(createBlobObject) + case Invalid(errors) => + Stream.raiseError[F](new MultipleUrlValidationException(errors)) + } + + override def put(key: Key, overwrite: Boolean): Pipe[F, Byte, Unit] = + createStorageUrlFrom(key) match { + case Valid(url) => + store.put(url, overwrite) + case Invalid(errors) => + _ => Stream.raiseError[F](new MultipleUrlValidationException(errors)) + } + + override def get(key: Key): F[Either[Throwable, String]] = + createStorageUrlFrom(key) match { + case Valid(url) => + store + .get(url, 1024) + .compile + .to(Array) + .map(array => new String(array)) + .attempt + case Invalid(errors) => + Async[F].delay(new MultipleUrlValidationException(errors).asLeft[String]) + } + + override def keyExists(key: Key): F[Boolean] = + createStorageUrlFrom(key) match { + case Valid(url) => + store.list(url).compile.toList.map(_.nonEmpty) + case Invalid(errors) => + Async[F].raiseError(new MultipleUrlValidationException(errors)) + } + + // input path format like 'endpoint/container/blobPath', where 'endpoint' is 'scheme://host' + private def createStorageUrlFrom(input: String): ValidatedNec[AuthorityParseError, Url[String]] = { + val scheme = input.split("://").head + val `endpoint/` = if (endpoint.endsWith("/")) endpoint else s"$endpoint/" + val `container/path` = input.stripPrefix(`endpoint/`) + val `[container, path]` = `container/path`.split("/") + val container = `[container, path]`.head + val path = `container/path`.stripPrefix(container) - override def get(key: Key): F[Either[Throwable, String]] = { - val (authority, path) = BlobStorage.splitKey(key) Authority - .parse(authority) - .fold( - errors => Async[F].delay(new MultipleUrlValidationException(errors).asLeft[String]), - authority => - store - .get(Url("https", authority, Path(path)), 1024) - .compile - .to(Array) - .map(array => new String(array)) - .attempt - ) + .parse(container) + .map(authority => Url(scheme, authority, Path(path))) } - override def keyExists(key: Key): F[Boolean] = { - val (authority, path) = BlobStorage.splitKey(key) - Authority - .parse(authority) - .fold( - errors => Async[F].raiseError(new MultipleUrlValidationException(errors)), - authority => store.list(Url("https", authority, Path(path))).compile.toList.map(_.nonEmpty) - ) + private def createBlobObject(url: Url[AzureBlob]) = { + val key = BlobStorage.Key.coerce(s"$endpoint/${url.representation.container}/${url.path.relative}") + BlobStorage.BlobObject(key, url.path.representation.size.getOrElse(0L)) } } object AzureBlobStorage { - def create[F[_]: Async](): Resource[F, BlobStorage[F]] = - createStore().map(new AzureBlobStorage(_)) + def createDefault[F[_]: Async](path: URI): Resource[F, BlobStorage[F]] = { + val builder = new BlobServiceClientBuilder().credential(new DefaultAzureCredentialBuilder().build) + create(path, builder) + } + + def create[F[_]: Async](path: URI, builder: BlobServiceClientBuilder): Resource[F, BlobStorage[F]] = { + val endpoint = extractEndpoint(path) + val client = builder.endpoint(endpoint).buildAsyncClient() + createStore(client).map(new AzureBlobStorage(_, endpoint)) + } - private def createStore[F[_]: Async](): Resource[F, AzureStore[F]] = { - val credentials = new DefaultAzureCredentialBuilder().build - val client = new BlobServiceClientBuilder() - // TODO Do we need to pass 'endpoint' here? - // .endpoint("https://{accountName}.blob.core.windows.net") - .credential(credentials) - .buildAsyncClient() + private def extractEndpoint(path: URI): String = + path.toString.stripSuffix("/").split("/").dropRight(1).mkString("/") + private def createStore[F[_]: Async](client: BlobServiceAsyncClient): Resource[F, AzureStore[F]] = AzureStore .builder[F](client) .build .fold(errors => Resource.raiseError(errors.reduce(Throwables.collapsingSemigroup)), Resource.pure) - } } diff --git a/modules/common-transformer-stream/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/common/Config.scala b/modules/common-transformer-stream/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/common/Config.scala index e73192c0f..bb5695b78 100644 --- a/modules/common-transformer-stream/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/common/Config.scala +++ b/modules/common-transformer-stream/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/common/Config.scala @@ -291,12 +291,12 @@ object Config { cur.as[Output.S3] case Right("gs") => cur.as[Output.GCS] - case Right("https") => + case Right("http") | Right("https") => cur.as[Output.AzureBlobStorage] case Right(other) => Left( DecodingFailure( - s"Output type $other is not supported yet. Supported types: 's3', 's3a', 's3n', 'gs', 'https'", + s"Output type $other is not supported yet. Supported types: 's3', 's3a', 's3n', 'gs', 'http', 'https'", pathCur.history ) ) diff --git a/modules/common-transformer-stream/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/common/processing/TestApplication.scala b/modules/common-transformer-stream/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/common/processing/TestApplication.scala index fe71a28f5..31965f51f 100644 --- a/modules/common-transformer-stream/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/common/processing/TestApplication.scala +++ b/modules/common-transformer-stream/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/common/processing/TestApplication.scala @@ -159,7 +159,7 @@ object TestApplication { val updatedOutput = config.output match { case c: Config.Output.S3 => c.copy(path = URI.create(c.path.toString.replace("s3:/", "file:/"))) case c: Config.Output.GCS => c.copy(path = URI.create(c.path.toString.replace("gs:/", "file:/"))) - case c: Config.Output.AzureBlobStorage => c.copy(path = URI.create(c.path.toString.replace("https:/", "file:/"))) + case c: Config.Output.AzureBlobStorage => c.copy(path = URI.create(c.path.toString.replace("http:/", "file:/"))) } config.copy(output = updatedOutput) } diff --git a/modules/common/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/common/cloud/BlobStorage.scala b/modules/common/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/common/cloud/BlobStorage.scala index 1d85a736e..07466721b 100644 --- a/modules/common/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/common/cloud/BlobStorage.scala +++ b/modules/common/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/common/cloud/BlobStorage.scala @@ -88,7 +88,7 @@ object BlobStorage { object Folder extends tag.Tagger[BlobStorageFolderTag] { def parse(s: String): Either[String, Folder] = s match { - case _ if !correctlyPrefixed(s) => s"Bucket name $s doesn't start with s3:// s3a:// s3n:// gs:// or https:// prefix".asLeft + case _ if !correctlyPrefixed(s) => s"Bucket name $s doesn't start with s3:// s3a:// s3n:// gs:// http:// or https:// prefix".asLeft case _ if s.length > 1024 => "Key length cannot be more than 1024 symbols".asLeft case _ => coerce(s).asRight } @@ -117,7 +117,7 @@ object BlobStorage { * Extract `xx://path/run=YYYY-MM-dd-HH-mm-ss/atomic-events` part from Set of prefixes that can be * used in config.yml In the end it won't affect how blob storage is accessed */ - val supportedPrefixes = Set("s3", "s3n", "s3a", "gs", "https") + val supportedPrefixes = Set("s3", "s3n", "s3a", "gs", "http", "https") private def correctlyPrefixed(s: String): Boolean = supportedPrefixes.foldLeft(false) { (result, prefix) => @@ -146,7 +146,7 @@ object BlobStorage { fixPrefix(s).asInstanceOf[Key] def parse(s: String): Either[String, Key] = s match { - case _ if !correctlyPrefixed(s) => s"Bucket name $s doesn't start with s3:// s3a:// s3n:// gs:// https:// prefix".asLeft + case _ if !correctlyPrefixed(s) => s"Bucket name $s doesn't start with s3:// s3a:// s3n:// gs:// http:// or https:// prefix".asLeft case _ if s.length > 1024 => "Key length cannot be more than 1024 symbols".asLeft case _ if s.endsWith("/") => "Blob storage key cannot have trailing slash".asLeft case _ => coerce(s).asRight diff --git a/modules/transformer-kafka/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/kafka/Main.scala b/modules/transformer-kafka/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/kafka/Main.scala index 788df40ba..c4456a883 100644 --- a/modules/transformer-kafka/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/kafka/Main.scala +++ b/modules/transformer-kafka/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/kafka/Main.scala @@ -15,12 +15,10 @@ package com.snowplowanalytics.snowplow.rdbloader.transformer.stream.kafka import cats.effect._ -import com.snowplowanalytics.snowplow.rdbloader.aws.AzureBlobStorage -import com.snowplowanalytics.snowplow.rdbloader.azure._ -import com.snowplowanalytics.snowplow.rdbloader.common.cloud.{BlobStorage, Queue} +import com.snowplowanalytics.snowplow.rdbloader.azure.AzureBlobStorage +import com.snowplowanalytics.snowplow.rdbloader.common.cloud.BlobStorage import com.snowplowanalytics.snowplow.rdbloader.transformer.stream.common.{Config, Run} import com.snowplowanalytics.snowplow.rdbloader.transformer.stream.kafka.generated.BuildInfo - import com.snowplowanalytics.snowplow.scalatracker.emitters.http4s.ceTracking object Main extends IOApp { @@ -32,58 +30,18 @@ object Main extends IOApp { BuildInfo.version, BuildInfo.description, runtime.compute, - (config, _) => mkSource(config), - c => mkBlobStorage(c), - c => mkBadQueue(c), - mkShreddingCompleteQueue, + (config, _) => Queues.createInputQueue(config), + c => createBlobStorage(c), + c => Queues.createBadOutputQueue(c), + Queues.createShreddingCompleteQueue, KafkaCheckpointer.checkpointer ) - private def mkSource[F[_]: Async]( - streamInput: Config.StreamInput - ): Resource[F, Queue.Consumer[F]] = - streamInput match { - case conf: Config.StreamInput.Kafka => - KafkaConsumer.consumer[F]( - conf.bootstrapServers, - conf.topicName, - conf.consumerConf - ) - case _ => - Resource.eval(Async[F].raiseError(new IllegalArgumentException(s"Input is not Kafka"))) - } - - private def mkBlobStorage[F[_]: Async](output: Config.Output): Resource[F, BlobStorage[F]] = + private def createBlobStorage[F[_]: Async](output: Config.Output): Resource[F, BlobStorage[F]] = output match { - case _: Config.Output.AzureBlobStorage => - AzureBlobStorage.create[F]() + case c: Config.Output.AzureBlobStorage => + AzureBlobStorage.createDefault[F](c.path) case _ => Resource.eval(Async[F].raiseError(new IllegalArgumentException(s"Output is not Azure Blob Storage"))) } - - private def mkBadQueue[F[_]: Async]( - output: Config.Output.Bad.Queue - ): Resource[F, Queue.ChunkProducer[F]] = - output match { - case kafka: Config.Output.Bad.Queue.Kafka => - KafkaProducer.chunkProducer[F]( - kafka.bootstrapServers, - kafka.topicName, - kafka.producerConf - ) - case _ => - Resource.eval(Async[F].raiseError(new IllegalArgumentException(s"Output queue is not Kafka"))) - } - - private def mkShreddingCompleteQueue[F[_]: Async](queueConfig: Config.QueueConfig): Resource[F, Queue.Producer[F]] = - queueConfig match { - case kafka: Config.QueueConfig.Kafka => - KafkaProducer.producer[F]( - kafka.bootstrapServers, - kafka.topicName, - kafka.producerConf - ) - case _ => - Resource.eval(Async[F].raiseError(new IllegalArgumentException(s"Message queue is not Kafka"))) - } } diff --git a/modules/transformer-kafka/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/kafka/Queues.scala b/modules/transformer-kafka/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/kafka/Queues.scala new file mode 100644 index 000000000..42f96e495 --- /dev/null +++ b/modules/transformer-kafka/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/kafka/Queues.scala @@ -0,0 +1,63 @@ +/* + * Copyright (c) 2012-2023 Snowplow Analytics Ltd. All rights reserved. + * + * This program is licensed to you under the Apache License Version 2.0, + * and you may not use this file except in compliance with the Apache License Version 2.0. + * You may obtain a copy of the Apache License Version 2.0 at + * http://www.apache.org/licenses/LICENSE-2.0. + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the Apache License Version 2.0 is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the Apache License Version 2.0 for the specific language governing permissions and + * limitations there under. + */ +package com.snowplowanalytics.snowplow.rdbloader.transformer.stream.kafka + +import cats.effect._ +import com.snowplowanalytics.snowplow.rdbloader.azure._ +import com.snowplowanalytics.snowplow.rdbloader.common.cloud.Queue +import com.snowplowanalytics.snowplow.rdbloader.transformer.stream.common.Config + +private[kafka] object Queues { + + def createInputQueue[F[_]: Async]( + streamInput: Config.StreamInput + ): Resource[F, Queue.Consumer[F]] = + streamInput match { + case conf: Config.StreamInput.Kafka => + KafkaConsumer.consumer[F]( + conf.bootstrapServers, + conf.topicName, + conf.consumerConf + ) + case _ => + Resource.eval(Async[F].raiseError(new IllegalArgumentException(s"Input is not Kafka"))) + } + + def createBadOutputQueue[F[_]: Async]( + output: Config.Output.Bad.Queue + ): Resource[F, Queue.ChunkProducer[F]] = + output match { + case kafka: Config.Output.Bad.Queue.Kafka => + KafkaProducer.chunkProducer[F]( + kafka.bootstrapServers, + kafka.topicName, + kafka.producerConf + ) + case _ => + Resource.eval(Async[F].raiseError(new IllegalArgumentException(s"Output queue is not Kafka"))) + } + + def createShreddingCompleteQueue[F[_]: Async](queueConfig: Config.QueueConfig): Resource[F, Queue.Producer[F]] = + queueConfig match { + case kafka: Config.QueueConfig.Kafka => + KafkaProducer.producer[F]( + kafka.bootstrapServers, + kafka.topicName, + kafka.producerConf + ) + case _ => + Resource.eval(Async[F].raiseError(new IllegalArgumentException(s"Message queue is not Kafka"))) + } +} diff --git a/modules/transformer-kafka/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/kafka/AzuriteBasedDevApp.scala b/modules/transformer-kafka/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/kafka/AzuriteBasedDevApp.scala new file mode 100644 index 000000000..eaa4cffd1 --- /dev/null +++ b/modules/transformer-kafka/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/kafka/AzuriteBasedDevApp.scala @@ -0,0 +1,115 @@ +/* + * Copyright (c) 2012-2023 Snowplow Analytics Ltd. All rights reserved. + * + * This program is licensed to you under the Apache License Version 2.0, + * and you may not use this file except in compliance with the Apache License Version 2.0. + * You may obtain a copy of the Apache License Version 2.0 at http://www.apache.org/licenses/LICENSE-2.0. + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the Apache License Version 2.0 is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the Apache License Version 2.0 for the specific language governing permissions and limitations there under. + */ +package com.snowplowanalytics.snowplow.rdbloader.transformer.stream.kafka + +import cats.effect._ +import com.azure.storage.blob.BlobServiceClientBuilder +import com.azure.storage.common.StorageSharedKeyCredential +import com.snowplowanalytics.snowplow.rdbloader.azure.AzureBlobStorage +import com.snowplowanalytics.snowplow.rdbloader.common.cloud.BlobStorage +import com.snowplowanalytics.snowplow.rdbloader.transformer.stream.common.{Config, Run} +import com.snowplowanalytics.snowplow.rdbloader.transformer.stream.kafka.generated.BuildInfo +import com.snowplowanalytics.snowplow.scalatracker.emitters.http4s.ceTracking + +import java.util.Base64 + +/** + * Test transformer application that can be run locally with local Azure-like resources. + * + * To run following resources are required: + * + * - Kafka cluster (localhost:9092) with two topics: 'enriched' (input) and 'shreddingComplete' + * (for shredding complete message to notify loader) + * - Azurite Blob Storage (http://127.0.0.1:10000/devstoreaccount1) with `transformed` blob + * container created + * + * In the future it could be converted to automatic integration tests using testcontainers. + */ +object AzuriteBasedDevApp extends IOApp { + + val appConfig = + """ + |{ + | "input": { + | "topicName": "enriched" + | "bootstrapServers": "localhost:9092" + | } + | "output": { + | "path": "http://127.0.0.1:10000/devstoreaccount1/transformed" + | } + | "windowing": "1 minute" + | + | "queue": { + | "topicName": "shreddingComplete" + | "bootstrapServers": "localhost:9092" + | } + |} + |""".stripMargin + + val resolverConfig = + """ + |{ + | "schema": "iglu:com.snowplowanalytics.iglu/resolver-config/jsonschema/1-0-0", + | "data": { + | "cacheSize": 500, + | "cacheTtl": 30, + | "repositories": [ + | { + | "name": "Iglu Central", + | "priority": 0, + | "vendorPrefixes": [ ], + | "connection": { + | "http": { + | "uri": "http://iglucentral.com" + | } + | } + | } + | ] + | } + |} + |""".stripMargin + + val credentials = new StorageSharedKeyCredential( + "devstoreaccount1", + "Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==" + ) + + def run(args: List[String]): IO[ExitCode] = { + val fixedArgs = List("--config", encode(appConfig), "--iglu-config", encode(resolverConfig)) + Run.run[IO, KafkaCheckpointer[IO]]( + fixedArgs, + BuildInfo.name, + BuildInfo.version, + BuildInfo.description, + runtime.compute, + (config, _) => Queues.createInputQueue(config), + c => createBlobStorageWithAzuriteKeys(c), + c => Queues.createBadOutputQueue(c), + Queues.createShreddingCompleteQueue, + KafkaCheckpointer.checkpointer + ) + } + + private def createBlobStorageWithAzuriteKeys[F[_]: Async](output: Config.Output): Resource[F, BlobStorage[F]] = + output match { + case c: Config.Output.AzureBlobStorage => + val clientBuilder = new BlobServiceClientBuilder().credential(credentials) + AzureBlobStorage.create(c.path, clientBuilder) + case _ => + Resource.eval(Async[F].raiseError(new IllegalArgumentException(s"Output is not Azure Blob Storage"))) + } + + private def encode(value: String) = + new String(Base64.getUrlEncoder.encode(value.getBytes("UTF-8"))) + +} From 8ef1f2d22eaf0cd4f297d87dc5a714b48d1a16b3 Mon Sep 17 00:00:00 2001 From: spenes Date: Thu, 15 Jun 2023 13:02:48 +0300 Subject: [PATCH 04/14] transformer-kafka: add auth for writing parquet to Azure Data Lake --- .../rdbloader/azure/AzureBlobStorage.scala | 57 ++++++++++++------- .../rdbloader/azure/AzureTokenProvider.scala | 48 ++++++++++++++++ .../transformer/stream/common/Resources.scala | 10 +++- .../transformer/stream/common/Run.scala | 7 ++- .../stream/common/parquet/ParquetOps.scala | 29 ++++++++++ .../stream/common/parquet/ParquetSink.scala | 11 ++-- .../common/processing/TestApplication.scala | 4 +- .../transformer/stream/kafka/Main.scala | 21 ++++++- project/Dependencies.scala | 7 ++- 9 files changed, 159 insertions(+), 35 deletions(-) create mode 100644 modules/azure/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/azure/AzureTokenProvider.scala create mode 100644 modules/common-transformer-stream/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/common/parquet/ParquetOps.scala diff --git a/modules/azure/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/azure/AzureBlobStorage.scala b/modules/azure/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/azure/AzureBlobStorage.scala index 881eb98c9..10bb60a5f 100644 --- a/modules/azure/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/azure/AzureBlobStorage.scala +++ b/modules/azure/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/azure/AzureBlobStorage.scala @@ -22,14 +22,13 @@ import cats.data.ValidatedNec import cats.effect._ import cats.implicits._ import com.azure.identity.DefaultAzureCredentialBuilder -import com.azure.storage.blob.{BlobServiceAsyncClient, BlobServiceClientBuilder} +import com.azure.storage.blob.{BlobServiceAsyncClient, BlobServiceClientBuilder, BlobUrlParts} import com.snowplowanalytics.snowplow.rdbloader.common.cloud.BlobStorage import com.snowplowanalytics.snowplow.rdbloader.common.cloud.BlobStorage.{Folder, Key} import fs2.{Pipe, Stream} - import java.net.URI -class AzureBlobStorage[F[_]: Async] private (store: AzureStore[F], endpoint: String) extends BlobStorage[F] { +class AzureBlobStorage[F[_]: Async] private (store: AzureStore[F], path: AzureBlobStorage.PathParts) extends BlobStorage[F] { override def list(folder: Folder, recursive: Boolean): Stream[F, BlobStorage.BlobObject] = createStorageUrlFrom(folder) match { @@ -71,21 +70,13 @@ class AzureBlobStorage[F[_]: Async] private (store: AzureStore[F], endpoint: Str } // input path format like 'endpoint/container/blobPath', where 'endpoint' is 'scheme://host' - private def createStorageUrlFrom(input: String): ValidatedNec[AuthorityParseError, Url[String]] = { - val scheme = input.split("://").head - val `endpoint/` = if (endpoint.endsWith("/")) endpoint else s"$endpoint/" - val `container/path` = input.stripPrefix(`endpoint/`) - val `[container, path]` = `container/path`.split("/") - val container = `[container, path]`.head - val path = `container/path`.stripPrefix(container) - + private def createStorageUrlFrom(input: String): ValidatedNec[AuthorityParseError, Url[String]] = Authority - .parse(container) - .map(authority => Url(scheme, authority, Path(path))) - } + .parse(path.containerName) + .map(authority => Url(path.scheme, authority, Path(path.extractRelative(input)))) private def createBlobObject(url: Url[AzureBlob]) = { - val key = BlobStorage.Key.coerce(s"$endpoint/${url.representation.container}/${url.path.relative}") + val key = BlobStorage.Key.coerce(s"${path.fullPath}/${url.path.relative}") BlobStorage.BlobObject(key, url.path.representation.size.getOrElse(0L)) } } @@ -98,17 +89,41 @@ object AzureBlobStorage { } def create[F[_]: Async](path: URI, builder: BlobServiceClientBuilder): Resource[F, BlobStorage[F]] = { - val endpoint = extractEndpoint(path) - val client = builder.endpoint(endpoint).buildAsyncClient() - createStore(client).map(new AzureBlobStorage(_, endpoint)) + val pathParts = parsePath(path.toString) + val client = builder.endpoint(pathParts.root).buildAsyncClient() + createStore(client).map(new AzureBlobStorage(_, pathParts)) } - private def extractEndpoint(path: URI): String = - path.toString.stripSuffix("/").split("/").dropRight(1).mkString("/") - private def createStore[F[_]: Async](client: BlobServiceAsyncClient): Resource[F, AzureStore[F]] = AzureStore .builder[F](client) .build .fold(errors => Resource.raiseError(errors.reduce(Throwables.collapsingSemigroup)), Resource.pure) + + final case class PathParts( + fullPath: String, + containerName: String, + storageAccountName: String, + scheme: String, + endpointSuffix: String, + relative: String + ) { + def extractRelative(p: String): String = + p.stripPrefix(fullPath) + + def root: String = + s"$scheme://$storageAccountName.blob.$endpointSuffix" + } + + def parsePath(path: String): PathParts = { + val parts = BlobUrlParts.parse(path) + PathParts( + fullPath = path, + containerName = parts.getBlobContainerName, + storageAccountName = parts.getAccountName, + scheme = parts.getScheme, + endpointSuffix = parts.getHost.stripPrefix(s"${parts.getAccountName}.blob."), + relative = parts.getBlobName + ) + } } diff --git a/modules/azure/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/azure/AzureTokenProvider.scala b/modules/azure/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/azure/AzureTokenProvider.scala new file mode 100644 index 000000000..a3ab57eef --- /dev/null +++ b/modules/azure/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/azure/AzureTokenProvider.scala @@ -0,0 +1,48 @@ +/* + * Copyright (c) 2012-2023 Snowplow Analytics Ltd. All rights reserved. + * + * This program is licensed to you under the Apache License Version 2.0, + * and you may not use this file except in compliance with the Apache License Version 2.0. + * You may obtain a copy of the Apache License Version 2.0 at + * http://www.apache.org/licenses/LICENSE-2.0. + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the Apache License Version 2.0 is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the Apache License Version 2.0 for the specific language governing permissions and + * limitations there under. + */ +package com.snowplowanalytics.snowplow.rdbloader.azure + +import java.util.Date + +import com.azure.core.credential.TokenRequestContext +import com.azure.identity.DefaultAzureCredentialBuilder + +import org.apache.hadoop.fs.azurebfs.extensions.CustomTokenProviderAdaptee +import org.apache.hadoop.conf.Configuration + +/** + * Creates Azure tokens for using with Hadoop file system. It isn't directly used in the project. + * Instead, class name is given as Hadoop configuration in the Main of Transformer Kafka. Then, it + * is used by Hadoop Azure File System to generate tokens. + */ +class AzureTokenProvider extends CustomTokenProviderAdaptee { + + private var expiryTime: Date = _ + private var accountName: String = _ + + override def initialize(configuration: Configuration, accountName: String): Unit = + this.accountName = accountName + + override def getAccessToken: String = { + val creds = new DefaultAzureCredentialBuilder().build() + val request = new TokenRequestContext() + request.addScopes("https://" + accountName) + val token = creds.getToken(request).block() + this.expiryTime = new Date(token.getExpiresAt.toInstant.toEpochMilli) + token.getToken + } + + override def getExpiryTime: Date = expiryTime +} diff --git a/modules/common-transformer-stream/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/common/Resources.scala b/modules/common-transformer-stream/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/common/Resources.scala index f2ae83739..7f87d7c5e 100644 --- a/modules/common-transformer-stream/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/common/Resources.scala +++ b/modules/common-transformer-stream/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/common/Resources.scala @@ -38,6 +38,7 @@ import com.snowplowanalytics.snowplow.rdbloader.common.cloud.{BlobStorage, Queue import com.snowplowanalytics.snowplow.rdbloader.common.telemetry.Telemetry import com.snowplowanalytics.snowplow.rdbloader.common.transformation.EventUtils.EventParser import com.snowplowanalytics.snowplow.rdbloader.common.transformation.{EventUtils, PropertiesCache, PropertiesKey} +import com.snowplowanalytics.snowplow.rdbloader.transformer.stream.common.parquet.ParquetOps import com.snowplowanalytics.snowplow.rdbloader.transformer.stream.common.Config.Output.Bad import com.snowplowanalytics.snowplow.rdbloader.transformer.stream.common.metrics.Metrics import com.snowplowanalytics.snowplow.rdbloader.transformer.stream.common.sinks.BadSink @@ -60,7 +61,8 @@ case class Resources[F[_], C]( checkpointer: Queue.Consumer.Message[F] => C, blobStorage: BlobStorage[F], badSink: BadSink[F], - registryLookup: RegistryLookup[F] + registryLookup: RegistryLookup[F], + parquetOps: ParquetOps ) object Resources { @@ -77,7 +79,8 @@ object Resources { mkSink: Config.Output => Resource[F, BlobStorage[F]], mkBadQueue: Config.Output.Bad.Queue => Resource[F, Queue.ChunkProducer[F]], mkQueue: Config.QueueConfig => Resource[F, Queue.Producer[F]], - checkpointer: Queue.Consumer.Message[F] => C + checkpointer: Queue.Consumer.Message[F] => C, + parquetOps: ParquetOps ): Resource[F, Resources[F, C]] = for { producer <- mkQueue(config.queue) @@ -116,7 +119,8 @@ object Resources { checkpointer, blobStorage, badSink, - registryLookup + registryLookup, + parquetOps ) private def mkBadSink[F[_]: Applicative]( diff --git a/modules/common-transformer-stream/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/common/Run.scala b/modules/common-transformer-stream/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/common/Run.scala index e6aee355d..e835274b2 100644 --- a/modules/common-transformer-stream/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/common/Run.scala +++ b/modules/common-transformer-stream/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/common/Run.scala @@ -7,6 +7,7 @@ import cats.effect._ import scala.concurrent.ExecutionContext import com.snowplowanalytics.snowplow.badrows.Processor +import com.snowplowanalytics.snowplow.rdbloader.transformer.stream.common.parquet.ParquetOps import com.snowplowanalytics.snowplow.rdbloader.transformer.stream.common.Config.{Monitoring, StreamInput} import com.snowplowanalytics.snowplow.rdbloader.transformer.stream.common.sources.Checkpointer import com.snowplowanalytics.snowplow.rdbloader.common.cloud.{BlobStorage, Queue} @@ -28,7 +29,8 @@ object Run { mkSink: Config.Output => Resource[F, BlobStorage[F]], mkBadQueue: Config.Output.Bad.Queue => Resource[F, Queue.ChunkProducer[F]], mkQueue: Config.QueueConfig => Resource[F, Queue.Producer[F]], - checkpointer: Queue.Consumer.Message[F] => C + checkpointer: Queue.Consumer.Message[F] => C, + parquetOps: ParquetOps = ParquetOps.noop ): F[ExitCode] = for { parsed <- CliConfig.loadConfigFrom[F](buildName, buildDescription)(args: Seq[String]).value @@ -45,7 +47,8 @@ object Run { mkSink, mkBadQueue, mkQueue, - checkpointer + checkpointer, + parquetOps ) .use { resources => val processor = Processor(buildName, buildVersion) diff --git a/modules/common-transformer-stream/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/common/parquet/ParquetOps.scala b/modules/common-transformer-stream/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/common/parquet/ParquetOps.scala new file mode 100644 index 000000000..0c2451ea7 --- /dev/null +++ b/modules/common-transformer-stream/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/common/parquet/ParquetOps.scala @@ -0,0 +1,29 @@ +/* + * Copyright (c) 2012-2023 Snowplow Analytics Ltd. All rights reserved. + * + * This program is licensed to you under the Apache License Version 2.0, + * and you may not use this file except in compliance with the Apache License Version 2.0. + * You may obtain a copy of the Apache License Version 2.0 at + * http://www.apache.org/licenses/LICENSE-2.0. + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the Apache License Version 2.0 is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the Apache License Version 2.0 for the specific language governing permissions and + * limitations there under. + */ +package com.snowplowanalytics.snowplow.rdbloader.transformer.stream.common.parquet + +import org.apache.hadoop.conf.Configuration + +trait ParquetOps { + def transformPath(p: String): String + def hadoopConf: Configuration +} + +object ParquetOps { + def noop: ParquetOps = new ParquetOps { + override def transformPath(p: String): String = p + override def hadoopConf: Configuration = new Configuration() + } +} diff --git a/modules/common-transformer-stream/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/common/parquet/ParquetSink.scala b/modules/common-transformer-stream/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/common/parquet/ParquetSink.scala index 6a0c28a82..bf3612ed8 100644 --- a/modules/common-transformer-stream/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/common/parquet/ParquetSink.scala +++ b/modules/common-transformer-stream/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/common/parquet/ParquetSink.scala @@ -34,9 +34,9 @@ import com.snowplowanalytics.snowplow.rdbloader.transformer.stream.common.parque import com.snowplowanalytics.snowplow.rdbloader.transformer.stream.common.sinks.{SinkPath, TransformedDataOps, Window} import com.snowplowanalytics.snowplow.rdbloader.transformer.stream.common.Resources import fs2.{Pipe, Stream} +import org.apache.hadoop.conf.Configuration import org.apache.parquet.hadoop.metadata.CompressionCodecName import org.apache.parquet.schema.MessageType - import java.net.URI object ParquetSink { @@ -52,14 +52,14 @@ object ParquetSink { ): Pipe[F, Transformed.Data, Unit] = { transformedData => // As uri can use 's3a' schema, using methods from 'java.nio.file.Path' would require additional dependency responsible for adding appropriate 'java.nio.file.spi.FileSystemProvider', see e.g. https://github.com/carlspring/s3fs-nio/ // Simple strings concat works for both cases: uri configured with and without trailing '/', bypassing usage of 'java.nio.file.Path' - val targetPath = s"${uri.toString}/${window.getDir}/${path.value}" + val targetPath = s"${resources.parquetOps.transformPath(uri.toString)}/${window.getDir}/${path.value}" val schemaCreation = createSchemaFromTypes(resources, types).value Stream.eval(schemaCreation).flatMap { case Left(error) => Stream.raiseError[F](new RuntimeException(s"Error while building parquet schema. ${error.show}")) case Right(schema) => - val parquetPipe = writeAsParquet(compression, targetPath, maxRecordsPerFile, schema) + val parquetPipe = writeAsParquet(compression, targetPath, maxRecordsPerFile, schema, resources.parquetOps.hadoopConf) transformedData .mapFilter(_.fieldValues) @@ -81,7 +81,8 @@ object ParquetSink { compression: Compression, path: String, maxRecordsPerFile: Long, - schema: MessageType + schema: MessageType, + hadoopConf: Configuration ) = { implicit val targetSchema = schema @@ -94,7 +95,7 @@ object ParquetSink { .of[List[FieldWithValue]] .preWriteTransformation(buildParquetRecord) .maxCount(maxRecordsPerFile) - .options(ParquetWriter.Options(compressionCodecName = compressionCodecName)) + .options(ParquetWriter.Options(compressionCodecName = compressionCodecName, hadoopConf = hadoopConf)) .write(Path(path)) } diff --git a/modules/common-transformer-stream/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/common/processing/TestApplication.scala b/modules/common-transformer-stream/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/common/processing/TestApplication.scala index 31965f51f..af8b8b3f5 100644 --- a/modules/common-transformer-stream/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/common/processing/TestApplication.scala +++ b/modules/common-transformer-stream/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/common/processing/TestApplication.scala @@ -31,6 +31,7 @@ import fs2.{Pipe, Stream} import org.typelevel.log4cats.slf4j.Slf4jLogger import cats.effect.unsafe.implicits.global import cats.Applicative +import com.snowplowanalytics.snowplow.rdbloader.transformer.stream.common.parquet.ParquetOps import com.snowplowanalytics.snowplow.rdbloader.transformer.stream.common.sources.{Checkpointer, ParsedC} import com.snowplowanalytics.snowplow.rdbloader.transformer.stream.common.{CliConfig, Config, Processing, Resources} import fs2.io.file.Files @@ -72,7 +73,8 @@ object TestApplication { mkSink, _ => mkBadQueue[IO](queueBadSink), _ => queueFromRef[IO](completionsRef), - _ => () + _ => (), + ParquetOps.noop ) .use { resources => logger[IO].info(s"Starting RDB Shredder with ${appConfig} config") *> diff --git a/modules/transformer-kafka/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/kafka/Main.scala b/modules/transformer-kafka/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/kafka/Main.scala index c4456a883..148b59ceb 100644 --- a/modules/transformer-kafka/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/kafka/Main.scala +++ b/modules/transformer-kafka/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/kafka/Main.scala @@ -17,9 +17,11 @@ package com.snowplowanalytics.snowplow.rdbloader.transformer.stream.kafka import cats.effect._ import com.snowplowanalytics.snowplow.rdbloader.azure.AzureBlobStorage import com.snowplowanalytics.snowplow.rdbloader.common.cloud.BlobStorage +import com.snowplowanalytics.snowplow.rdbloader.transformer.stream.common.parquet.ParquetOps import com.snowplowanalytics.snowplow.rdbloader.transformer.stream.common.{Config, Run} import com.snowplowanalytics.snowplow.rdbloader.transformer.stream.kafka.generated.BuildInfo import com.snowplowanalytics.snowplow.scalatracker.emitters.http4s.ceTracking +import org.apache.hadoop.conf.Configuration object Main extends IOApp { @@ -34,7 +36,8 @@ object Main extends IOApp { c => createBlobStorage(c), c => Queues.createBadOutputQueue(c), Queues.createShreddingCompleteQueue, - KafkaCheckpointer.checkpointer + KafkaCheckpointer.checkpointer, + parquetOps ) private def createBlobStorage[F[_]: Async](output: Config.Output): Resource[F, BlobStorage[F]] = @@ -44,4 +47,20 @@ object Main extends IOApp { case _ => Resource.eval(Async[F].raiseError(new IllegalArgumentException(s"Output is not Azure Blob Storage"))) } + + private def parquetOps: ParquetOps = new ParquetOps { + + override def transformPath(p: String): String = { + val parts = AzureBlobStorage.parsePath(p) + s"abfss://${parts.containerName}@${parts.storageAccountName}.dfs.${parts.endpointSuffix}" + } + + override def hadoopConf: Configuration = { + val hadoopConf = new Configuration() + hadoopConf.set("fs.azure.account.auth.type", "Custom") + hadoopConf.set("fs.azure.account.oauth.provider.type", "com.snowplowanalytics.snowplow.rdbloader.azure.AzureTokenProvider") + hadoopConf + } + + } } diff --git a/project/Dependencies.scala b/project/Dependencies.scala index e5bbf55af..fb804e345 100644 --- a/project/Dependencies.scala +++ b/project/Dependencies.scala @@ -137,12 +137,13 @@ object Dependencies { val jacksonDatabind = "com.fasterxml.jackson.core" % "jackson-databind" % V.jacksonDatabind val jacksonCbor = "com.fasterxml.jackson.dataformat" % "jackson-dataformat-cbor" % V.jacksonModule val parquet4s = "com.github.mjakubowski84" %% "parquet4s-fs2" % V.parquet4s + val hadoopCommon = "org.apache.hadoop" % "hadoop-common" % V.hadoopClient val hadoop = "org.apache.hadoop" % "hadoop-client" % V.hadoopClient val parquetHadoop = "org.apache.parquet" % "parquet-hadoop" % V.parquetHadoop val hadoopAws = ("org.apache.hadoop" % "hadoop-aws" % V.hadoopClient % Runtime) .exclude("com.amazonaws", "aws-java-sdk-bundle") // aws-java-sdk-core is already present in assembled jar val hadoopGcp = "com.google.cloud.bigdataoss" % "gcs-connector" % V.hadoopGcpClient % Runtime - val hadoopAzure = "org.apache.hadoop" % "hadoop-azure" % "3.3.5" % Runtime + val hadoopAzure = "org.apache.hadoop" % "hadoop-azure" % V.hadoopClient val kinesisClient = ("software.amazon.kinesis" % "amazon-kinesis-client" % V.kinesisClient) .exclude("software.amazon.glue", "schema-registry-common") .exclude("software.amazon.glue", "schema-registry-serde") @@ -214,7 +215,9 @@ object Dependencies { val azureDependencies = Seq( fs2BlobstoreAzure, azureIdentity, - fs2Kafka + fs2Kafka, + hadoopCommon, + hadoopAzure ) val commonDependencies = Seq( From b889692ddb9a8e361e8c6f5f28947df132b15e68 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Piotr=20Poniedzia=C5=82ek?= Date: Wed, 31 May 2023 15:13:44 +0200 Subject: [PATCH 05/14] loader: add azure --- build.sbt | 6 +- .../azure/databricks.config.minimal.hocon | 15 ++ .../azure/databricks.config.reference.hocon | 229 +++++++++++++++++ .../azure/snowflake.config.minimal.hocon | 21 ++ .../azure/snowflake.config.reference.hocon | 235 ++++++++++++++++++ .../common/telemetry/Telemetry.scala | 1 + .../snowplow/rdbloader/config/Config.scala | 27 +- .../rdbloader/config/StorageTarget.scala | 7 +- .../snowplow/rdbloader/dsl/Environment.scala | 14 ++ 9 files changed, 551 insertions(+), 4 deletions(-) create mode 100644 config/loader/azure/databricks.config.minimal.hocon create mode 100644 config/loader/azure/databricks.config.reference.hocon create mode 100644 config/loader/azure/snowflake.config.minimal.hocon create mode 100644 config/loader/azure/snowflake.config.reference.hocon diff --git a/build.sbt b/build.sbt index c2b4c3435..ac8d74722 100755 --- a/build.sbt +++ b/build.sbt @@ -83,7 +83,11 @@ lazy val loader = project .settings(BuildSettings.loaderBuildSettings) .settings(addCompilerPlugin(Dependencies.betterMonadicFor)) .settings(libraryDependencies ++= Dependencies.loaderDependencies) - .dependsOn(aws % "compile->compile;test->test;runtime->runtime", gcp % "compile->compile;test->test") + .dependsOn( + aws % "compile->compile;test->test;runtime->runtime", + gcp % "compile->compile;test->test", + azure % "compile->compile;test->test" + ) lazy val redshiftLoader = project .in(file("modules/redshift-loader")) diff --git a/config/loader/azure/databricks.config.minimal.hocon b/config/loader/azure/databricks.config.minimal.hocon new file mode 100644 index 000000000..dcb78cd0f --- /dev/null +++ b/config/loader/azure/databricks.config.minimal.hocon @@ -0,0 +1,15 @@ +{ + "blobStorageEndpoint": "https://accountName.blob.core.windows.net" + "messageQueue": { + "type": "kafka" + "bootstrapServers": "localhost:9092" + "topicName": "loaderTopic" + }, + "storage" : { + "host": "abc.cloud.databricks.com" + "password": "Supersecret1" + "schema": "atomic", + "port": 443, + "httpPath": "/databricks/http/path", + } +} \ No newline at end of file diff --git a/config/loader/azure/databricks.config.reference.hocon b/config/loader/azure/databricks.config.reference.hocon new file mode 100644 index 000000000..ba533f231 --- /dev/null +++ b/config/loader/azure/databricks.config.reference.hocon @@ -0,0 +1,229 @@ +{ + + # Azure Blob Storage endpoint, should contain container with transformer's output + "blobStorageEndpoint": "https://accountName.blob.core.windows.net" + + # Kafka topic used by Transformer and Loader to communicate + "messageQueue": { + "type": "kafka" + "bootstrapServers": "localhost:9092" + "topicName": "loaderTopic" + }, + + # Warehouse connection details + "storage" : { + # Hostname of Databricks cluster + "host": "abc.cloud.databricks.com", + # DB password + # TODO handle secret store params instead of plaintext + "password": "secret" + # Optional. Override the Databricks default catalog, e.g. with a Unity catalog name. + "catalog": "hive_metastore", + # DB schema + "schema": "atomic", + # Database port + "port": 443, + # Http Path of Databricks cluster + "httpPath": "/databricks/http/path", + # User agent name for Databricks connection. Optional, default value "snowplow-rdbloader-oss" + "userAgent": "snowplow-rdbloader-oss" + + # Optimize period per table, that will be used as predicate for the OPTIMIZE command. + "eventsOptimizePeriod": "2 days" + }, + + "schedules": { + # Periodic schedules to stop loading, e.g. for Databricks maintenance window + # Any amount of schedules is supported, but recommended to not overlap them + # The schedule works with machine's local timezone (and UTC is recommended) + "noOperation": [ + { + # Human-readable name of the no-op window + "name": "Maintenance window", + # Cron expression with second granularity + "when": "0 0 12 * * ?", + # For how long the loader should be paused + "duration": "1 hour" + } + ], + # Loader runs periodic OPTIMIZE statements to prevent growing number of files behind delta tables. + "optimizeEvents": "0 0 0 ? * *", + "optimizeManifest": "0 0 5 ? * *" + } + + # Observability and reporting options + "monitoring": { + # Snowplow tracking (optional) + "snowplow": { + "appId": "databricks-loader", + "collector": "snplow.acme.com", + }, + + # An endpoint for alerts and infromational events + # Everything sent to snowplow collector (as properly formed self-describing events) + # will also be sent to the webhook as POST payloads with self-describing JSONs + "webhook": { + # An actual HTTP endpoint + "endpoint": "https://webhook.acme.com", + # Set of arbitrary key-value pairs attached to the payload + "tags": { + "pipeline": "production" + } + }, + + # Optional, for tracking runtime exceptions + "sentry": { + "dsn": "http://sentry.acme.com" + }, + + # Optional, configure how metrics are reported + "metrics": { + # Optional, send metrics to StatsD server + "statsd": { + "hostname": "localhost", + "port": 8125, + # Any key-value pairs to be tagged on every StatsD metric + "tags": { + "app": "rdb-loader" + } + # Optional, override the default metric prefix + # "prefix": "snowplow.rdbloader." + }, + + # Optional, print metrics on stdout (with slf4j) + "stdout": { + # Optional, override the default metric prefix + # "prefix": "snowplow.rdbloader." + } + + # Optional, period for metrics emitted periodically + # Default value 5 minutes + # There is only one periodic metric at the moment. + # This metric is minimum_age_of_loaded_data. + # It specifies how old is the latest event in the warehouse. + "period": "5 minutes" + }, + + # Optional, configuration for periodic unloaded/corrupted folders checks + "folders": { + # Path where Loader could store auxiliary logs + # Loader should be able to write here, Databricks should be able to load from here + "staging": "https://accountName.blob.core.windows.net/staging/", + # How often to check + "period": "1 hour" + # Specifies since when folder monitoring will check + "since": "14 days" + # Specifies until when folder monitoring will check + "until": "7 days" + # Path to transformer archive (must be same as Transformer's `output.path`) + "transformerOutput": "https://accountName.blob.core.windows.net/transformed/" + # How many times the check can fail before generating an alarm instead of warning + "failBeforeAlarm": 3 + }, + + # Periodic DB health-check, raising a warning if DB hasn't responded to `SELECT 1` + "healthCheck": { + # How often query a DB + "frequency": "20 minutes", + # How long to wait for a response + "timeout": "15 seconds" + } + }, + + # Immediate retries configuration + # Unlike retryQueue these retries happen immediately, without proceeding to another message + "retries": { + # Starting backoff period + "backoff": "30 seconds" + # A strategy to use when deciding on next backoff + "strategy": "EXPONENTIAL" + # How many attempts to make before sending the message into retry queue + # If missing - the loader will be retrying until cumulative bound + "attempts": 3, + # When backoff reaches this delay the Loader will stop retrying + # Missing cumulativeBound with missing attempts will force to retry inifintely + "cumulativeBound": "1 hour" + }, + + # Check the target destination to make sure it is ready. + # Retry the checking until target got ready and block the application in the meantime + "readyCheck": { + # Starting backoff period + "backoff": "15 seconds" + # A strategy to use when deciding on next backoff + "strategy": "CONSTANT" + # When backoff reaches this delay the Loader will stop retrying + "cumulativeBound": "10 minutes" + }, + + # Retries configuration for initilization block + # It will retry on all exceptions from there + "initRetries": { + # Starting backoff period + "backoff": "30 seconds" + # A strategy to use when deciding on next backoff + "strategy": "EXPONENTIAL" + # How many attempts to make before sending the message into retry queue + # If missing - the loader will be retrying until cumulative bound + "attempts": 3, + # When backoff reaches this delay the Loader will stop retrying + # Missing cumulativeBound with missing attempts will force to retry inifintely + "cumulativeBound": "1 hour" + }, + + # Additional backlog of recently failed folders that could be automatically retried + # Retry Queue saves a failed folder and then re-reads the info from shredding_complete S3 file + "retryQueue": { + # How often batch of failed folders should be pulled into a discovery queue + "period": "30 minutes", + # How many failures should be kept in memory + # After the limit is reached new failures are dropped + "size": 64, + # How many attempt to make for each folder + # After the limit is reached new failures are dropped + "maxAttempts": 3, + # Artificial pause after each failed folder being added to the queue + "interval": "5 seconds" + }, + + "timeouts": { + # How long loading (actual COPY statements) can take before considering Databricks unhealthy + # Without any progress (i.e. different subfolder) within this period, loader + # will abort the transaction + "loading": "45 minutes", + + # How long non-loading steps (such as ALTER TABLE or metadata queries) can take + # before considering Databricks unhealthy + "nonLoading": "10 minutes" + } + + # Optional. Configure telemetry + # All the fields are optional + "telemetry": { + # Set to true to disable telemetry + "disable": false + # Interval for the heartbeat event + "interval": 15 minutes + # HTTP method used to send the heartbeat event + "method": "POST" + # URI of the collector receiving the heartbeat event + "collectorUri": "collector-g.snowplowanalytics.com" + # Port of the collector receiving the heartbeat event + "collectorPort": 443 + # Whether to use https or not + "secure": true + # Identifier intended to tie events together across modules, + # infrastructure and apps when used consistently + "userProvidedId": "my_pipeline" + # ID automatically generated upon running a modules deployment script + # Intended to identify each independent module, and the infrastructure it controls + "autoGeneratedId": "hfy67e5ydhtrd" + # Unique identifier for the VM instance + # Unique for each instance of the app running within a module + "instanceId": "665bhft5u6udjf" + # Name of the terraform module that deployed the app + "moduleName": "rdb-loader-ce" + # Version of the terraform module that deployed the app + "moduleVersion": "1.0.0" + } +} diff --git a/config/loader/azure/snowflake.config.minimal.hocon b/config/loader/azure/snowflake.config.minimal.hocon new file mode 100644 index 000000000..966c9b875 --- /dev/null +++ b/config/loader/azure/snowflake.config.minimal.hocon @@ -0,0 +1,21 @@ +{ + "blobStorageEndpoint": "https://accountName.blob.core.windows.net" + "messageQueue": { + "type": "kafka" + "bootstrapServers": "localhost:9092" + "topicName": "loaderTopic" + }, + "storage" : { + "type": "snowflake", + + "snowflakeRegion": "us-west-2", + "username": "admin", + "password": "Supersecret1", + "account": "acme", + "warehouse": "wh", + "schema": "atomic", + "database": "snowplow", + + "transformedStage": "snowplow_stage" + } +} diff --git a/config/loader/azure/snowflake.config.reference.hocon b/config/loader/azure/snowflake.config.reference.hocon new file mode 100644 index 000000000..02f8608f3 --- /dev/null +++ b/config/loader/azure/snowflake.config.reference.hocon @@ -0,0 +1,235 @@ +{ + + # Azure Blob Storage endpoint, should contain container with transformer's output + "blobStorageEndpoint": "https://accountName.blob.core.windows.net" + + # Kafka topic used by Transformer and Loader to communicate + "messageQueue": { + "type": "kafka" + "bootstrapServers": "localhost:9092" + "topicName": "loaderTopic" + }, + + # Warehouse connection details + "storage" : { + # The destination DB + "type": "snowflake", + + # A region where Snowflake DB is placed + "snowflakeRegion": "us-west-2", + # DB user with permissions to load data + "username": "admin", + # DB password + # TODO handle secret store params instead of plaintext + "password": "secret" + # Snowflake account + "account": "acme", + # A warehouse to use for loading + "warehouse": "wh", + # DB schema + "schema": "atomic", + # DB name + "database": "snowplow", + # A stage where the data is stored. + "transformedStage": "snowplow_stage" + # A stage where the data for monitoring.folders is stored. + # Must be provided iff monitoring.folders is configured. + "folderMonitoringStage": "snowplow_folders_stage" + # An optional host name that will take a priority over automatically derived + "jdbcHost": "acme.eu-central-1.snowflake.com" + # Either "ResumeWarehouse" (the default) or "Select1". The command the loader runs to prepare the JDBC connection. + "readyCheck": "ResumeWarehouse" + }, + + "schedules": { + # Periodic schedules to stop loading + # Any amount of schedules is supported, but recommended to not overlap them + # The schedule works with machine's local timezone (and UTC is recommended) + "noOperation": [ + { + # Human-readable name of the no-op window + "name": "Maintenance window", + # Cron expression with second granularity + "when": "0 0 12 * * ?", + # For how long the loader should be paused + "duration": "1 hour" + } + ] + } + + # Observability and reporting options + "monitoring": { + # Snowplow tracking (optional) + "snowplow": { + "appId": "snowflake-loader", + "collector": "snplow.acme.com", + }, + + # An endpoint for alerts and infromational events + # Everything sent to snowplow collector (as properly formed self-describing events) + # will also be sent to the webhook as POST payloads with self-describing JSONs + "webhook": { + # An actual HTTP endpoint + "endpoint": "https://webhook.acme.com", + # Set of arbitrary key-value pairs attached to the payload + "tags": { + "pipeline": "production" + } + }, + + # Optional, for tracking runtime exceptions + "sentry": { + "dsn": "http://sentry.acme.com" + }, + + # Optional, configure how metrics are reported + "metrics": { + # Optional, send metrics to StatsD server + "statsd": { + "hostname": "localhost", + "port": 8125, + # Any key-value pairs to be tagged on every StatsD metric + "tags": { + "app": "rdb-loader" + } + # Optional, override the default metric prefix + # "prefix": "snowplow.rdbloader." + }, + + # Optional, print metrics on stdout (with slf4j) + "stdout": { + # Optional, override the default metric prefix + # "prefix": "snowplow.rdbloader." + } + + # Optional, period for metrics emitted periodically + # Default value 5 minutes + # There is only one periodic metric at the moment. + # This metric is minimum_age_of_loaded_data. + # It specifies how old is the latest event in the warehouse. + "period": "5 minutes" + }, + + # Optional, configuration for periodic unloaded/corrupted folders checks + "folders": { + # Path where Loader could store auxiliary logs + # Loader should be able to write here, Snowflake should be able to load from here + "staging": "https://accountName.blob.core.windows.net/staging/", + # How often to check + "period": "1 hour" + # Specifies since when folder monitoring will check + "since": "14 days" + # Specifies until when folder monitoring will check + "until": "7 days" + # Path to transformer archive (must be same as Transformer's `output.path`) + "transformerOutput": "https://accountName.blob.core.windows.net/transformed/" + # How many times the check can fail before generating an alarm instead of warning + "failBeforeAlarm": 3 + }, + + # Periodic DB health-check, raising a warning if DB hasn't responded to `SELECT 1` + "healthCheck": { + # How often query a DB + "frequency": "20 minutes", + # How long to wait for a response + "timeout": "15 seconds" + } + }, + + # Immediate retries configuration + # Unlike retryQueue these retries happen immediately, without proceeding to another message + "retries": { + # Starting backoff period + "backoff": "30 seconds" + # A strategy to use when deciding on next backoff + "strategy": "EXPONENTIAL" + # How many attempts to make before sending the message into retry queue + # If missing - the loader will be retrying until cumulative bound + "attempts": 3, + # When backoff reaches this delay the Loader will stop retrying + # Missing cumulativeBound with missing attempts will force to retry inifintely + "cumulativeBound": "1 hour" + }, + + # Check the target destination to make sure it is ready. + # Retry the checking until target got ready and block the application in the meantime + "readyCheck": { + # Starting backoff period + "backoff": "15 seconds" + # A strategy to use when deciding on next backoff + "strategy": "CONSTANT" + # When backoff reaches this delay the Loader will stop retrying + "cumulativeBound": "10 minutes" + }, + + # Retries configuration for initilization block + # It will retry on all exceptions from there + "initRetries": { + # Starting backoff period + "backoff": "30 seconds" + # A strategy to use when deciding on next backoff + "strategy": "EXPONENTIAL" + # How many attempts to make before sending the message into retry queue + # If missing - the loader will be retrying until cumulative bound + "attempts": 3, + # When backoff reaches this delay the Loader will stop retrying + # Missing cumulativeBound with missing attempts will force to retry inifintely + "cumulativeBound": "1 hour" + }, + + # Additional backlog of recently failed folders that could be automatically retried + # Retry Queue saves a failed folder and then re-reads the info from shredding_complete S3 file + "retryQueue": { + # How often batch of failed folders should be pulled into a discovery queue + "period": "30 minutes", + # How many failures should be kept in memory + # After the limit is reached new failures are dropped + "size": 64, + # How many attempt to make for each folder + # After the limit is reached new failures are dropped + "maxAttempts": 3, + # Artificial pause after each failed folder being added to the queue + "interval": "5 seconds" + }, + + "timeouts": { + # How long loading (actual COPY statements) can take before considering Snowflake unhealthy + # Without any progress (i.e. different subfolder) within this period, loader + # will abort the transaction + "loading": "45 minutes", + + # How long non-loading steps (such as ALTER TABLE or metadata queries) can take + # before considering Snowflake unhealthy + "nonLoading": "10 minutes" + } + + # Optional. Configure telemetry + # All the fields are optional + "telemetry": { + # Set to true to disable telemetry + "disable": false + # Interval for the heartbeat event + "interval": 15 minutes + # HTTP method used to send the heartbeat event + "method": "POST" + # URI of the collector receiving the heartbeat event + "collectorUri": "collector-g.snowplowanalytics.com" + # Port of the collector receiving the heartbeat event + "collectorPort": 443 + # Whether to use https or not + "secure": true + # Identifier intended to tie events together across modules, + # infrastructure and apps when used consistently + "userProvidedId": "my_pipeline" + # ID automatically generated upon running a modules deployment script + # Intended to identify each independent module, and the infrastructure it controls + "autoGeneratedId": "hfy67e5ydhtrd" + # Unique identifier for the VM instance + # Unique for each instance of the app running within a module + "instanceId": "665bhft5u6udjf" + # Name of the terraform module that deployed the app + "moduleName": "rdb-loader-ce" + # Version of the terraform module that deployed the app + "moduleVersion": "1.0.0" + } +} diff --git a/modules/common/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/common/telemetry/Telemetry.scala b/modules/common/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/common/telemetry/Telemetry.scala index c7746b53e..c8f00f52c 100644 --- a/modules/common/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/common/telemetry/Telemetry.scala +++ b/modules/common/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/common/telemetry/Telemetry.scala @@ -135,6 +135,7 @@ object Telemetry { object Cloud { case object Aws extends Cloud case object Gcp extends Cloud + case object Azure extends Cloud implicit val encoder: Encoder[Cloud] = Encoder.encodeString.contramap[Cloud](_.toString.toUpperCase) } diff --git a/modules/loader/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/config/Config.scala b/modules/loader/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/config/Config.scala index 7478e4e1d..5b8963e4f 100644 --- a/modules/loader/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/config/Config.scala +++ b/modules/loader/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/config/Config.scala @@ -166,6 +166,16 @@ object Config { } } } + + final case class Azure(blobStorageEndpoint: URI, messageQueue: Azure.Kafka) extends Cloud + + object Azure { + final case class Kafka( + topicName: String, + bootstrapServers: String, + consumerConf: Map[String, String] + ) + } } /** @@ -284,8 +294,12 @@ object Config { cur.up.as[Cloud.AWS] case Right("pubsub") => cur.up.as[Cloud.GCP] + case Right("kafka") => + cur.up.as[Cloud.Azure] case Right(other) => - Left(DecodingFailure(s"Message queue type $other is not supported yet. Supported types: 'sqs', 'pubsub'", cur.history)) + Left( + DecodingFailure(s"Message queue type $other is not supported yet. Supported types: 'sqs', 'pubsub', 'kafka'", cur.history) + ) case Left(DecodingFailure(_, List(CursorOp.DownField("type")))) => Left(DecodingFailure("Cannot find 'type' field in the config", cur.history)) case Left(other) => @@ -303,8 +317,14 @@ object Config { implicit val gcpDecoder: Decoder[Cloud.GCP] = deriveDecoder[Cloud.GCP] + implicit val azureDecoder: Decoder[Cloud.Azure] = + deriveDecoder[Cloud.Azure] + implicit val pubsubDecoder: Decoder[Cloud.GCP.Pubsub] = deriveDecoder[Cloud.GCP.Pubsub] + + implicit val kafkaDecoder: Decoder[Cloud.Azure.Kafka] = + deriveDecoder[Cloud.Azure.Kafka] } /** Post-decoding validation, making sure different parts are consistent */ @@ -322,6 +342,11 @@ object Config { case (StorageTarget.LoadAuthMethod.NoCreds, StorageTarget.LoadAuthMethod.NoCreds) => Nil case _ => List("Only 'NoCreds' load auth method is supported with GCP") } + case _: Config.Cloud.Azure => + (config.storage.foldersLoadAuthMethod, config.storage.eventsLoadAuthMethod) match { + case (StorageTarget.LoadAuthMethod.NoCreds, StorageTarget.LoadAuthMethod.NoCreds) => Nil + case _ => List("Only 'NoCreds' load auth method is supported with Azure") + } case _ => Nil } diff --git a/modules/loader/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/config/StorageTarget.scala b/modules/loader/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/config/StorageTarget.scala index 17e39d019..d0122a221 100644 --- a/modules/loader/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/config/StorageTarget.scala +++ b/modules/loader/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/config/StorageTarget.scala @@ -194,6 +194,8 @@ object StorageTarget { val AwsRegionsWithSegment = List("us-east-2", "us-east-1-gov", "ca-central-1", "eu-west-2", "ap-northeast-1", "ap-south-1") val GcpRegions = List("us-central1", "europe-west2", "europe-west4") + + // TODO do we have to change here anything? // val AzureRegions = List("west-us-2", "central-us", "east-us-2", "us-gov-virginia", "canada-central", "west-europe", "switzerland-north", "southeast-asia", "australia-east") // Host corresponds to Snowflake full account name which might include cloud platform and region @@ -208,9 +210,10 @@ object StorageTarget { s"$a.$r.snowflakecomputing.com".asRight else if (AwsRegionsWithSegment.contains(r)) s"$a.$r.aws.snowflakecomputing.com".asRight - else if (GcpRegions.contains(r)) + else if (GcpRegions.contains(r)) { s"$a.$r.gcp.snowflakecomputing.com".asRight - else s"$a.$r.azure.snowflakecomputing.com".asRight + // TODO check azure regions + } else s"$a.$r.azure.snowflakecomputing.com".asRight case (_, _, _) => "Snowflake config requires either jdbcHost or both account and region".asLeft } diff --git a/modules/loader/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/dsl/Environment.scala b/modules/loader/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/dsl/Environment.scala index f14331d88..1817f0306 100644 --- a/modules/loader/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/dsl/Environment.scala +++ b/modules/loader/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/dsl/Environment.scala @@ -25,6 +25,7 @@ import com.snowplowanalytics.snowplow.rdbloader.common.Sentry import com.snowplowanalytics.snowplow.rdbloader.common.telemetry.Telemetry import com.snowplowanalytics.snowplow.rdbloader.common.cloud.{BlobStorage, Queue, SecretStore} import com.snowplowanalytics.snowplow.rdbloader.aws.{EC2ParameterStore, S3, SQS} +import com.snowplowanalytics.snowplow.rdbloader.azure.{AzureBlobStorage, KafkaConsumer} import com.snowplowanalytics.snowplow.rdbloader.gcp.{GCS, Pubsub, SecretManager} import com.snowplowanalytics.snowplow.rdbloader.cloud.{JsonPathDiscovery, LoadAuthService} import com.snowplowanalytics.snowplow.rdbloader.state.{Control, State} @@ -179,12 +180,25 @@ object Environment { ) secretStore <- SecretManager.secretManager[F] } yield CloudServices(blobStorage, queueConsumer, loadAuthService, jsonPathDiscovery, secretStore) + case c: Cloud.Azure => + for { + loadAuthService <- LoadAuthService.noop[F] + jsonPathDiscovery = JsonPathDiscovery.noop[F] + implicit0(blobStorage: BlobStorage[F]) <- AzureBlobStorage.createDefault[F](c.blobStorageEndpoint) + queueConsumer <- KafkaConsumer.consumer[F]( + bootstrapServers = c.messageQueue.bootstrapServers, + topicName = c.messageQueue.topicName, + consumerConf = c.messageQueue.consumerConf + ) + secretStore = SecretStore.noop[F] // TODO implement secret store for Azure + } yield CloudServices(blobStorage, queueConsumer, loadAuthService, jsonPathDiscovery, secretStore) } def getCloudForTelemetry(config: Config[_]): Option[Telemetry.Cloud] = config.cloud match { case _: Cloud.AWS => Telemetry.Cloud.Aws.some case _: Cloud.GCP => Telemetry.Cloud.Gcp.some + case _: Cloud.Azure => Telemetry.Cloud.Azure.some } def getRegionForTelemetry(config: Config[_]): Option[String] = From ee9bac5934c3a24ed9d9cbcc56e7d1465c7a31a2 Mon Sep 17 00:00:00 2001 From: spenes Date: Sun, 18 Jun 2023 23:54:47 +0300 Subject: [PATCH 06/14] Loader: Add temp creds for Azure --- .../azure/databricks.config.reference.hocon | 23 +++ .../azure/snowflake.config.reference.hocon | 20 +++ .../rdbloader/azure/AzureBlobStorage.scala | 35 ++-- .../loader/databricks/Databricks.scala | 24 ++- .../loader/databricks/DatabricksSpec.scala | 33 +++- .../src/main/resources/application.conf | 5 + .../snowplow/rdbloader/Loader.scala | 3 +- .../rdbloader/cloud/LoadAuthService.scala | 168 ------------------ .../cloud/authservice/AWSAuthService.scala | 92 ++++++++++ .../cloud/authservice/AzureAuthService.scala | 92 ++++++++++ .../cloud/authservice/LoadAuthService.scala | 134 ++++++++++++++ .../snowplow/rdbloader/config/Config.scala | 25 ++- .../rdbloader/config/StorageTarget.scala | 39 +++- .../snowplow/rdbloader/db/Statement.scala | 2 +- .../snowplow/rdbloader/dsl/Environment.scala | 11 +- .../rdbloader/dsl/FolderMonitoring.scala | 2 +- .../snowplow/rdbloader/loading/Load.scala | 2 +- .../snowplow/rdbloader/package.scala | 2 +- .../snowplow/rdbloader/ConfigSpec.scala | 52 +++++- .../rdbloader/dsl/FolderMonitoringSpec.scala | 4 +- .../snowplow/rdbloader/loading/LoadSpec.scala | 4 +- .../rdbloader/test/PureLoadAuthService.scala | 2 +- .../snowplow/loader/redshift/Redshift.scala | 6 +- .../loader/redshift/RedshiftSpec.scala | 2 +- .../snowplow/loader/snowflake/Snowflake.scala | 27 ++- .../transformer/stream/kafka/Main.scala | 6 +- 26 files changed, 578 insertions(+), 237 deletions(-) delete mode 100644 modules/loader/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/cloud/LoadAuthService.scala create mode 100644 modules/loader/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/cloud/authservice/AWSAuthService.scala create mode 100644 modules/loader/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/cloud/authservice/AzureAuthService.scala create mode 100644 modules/loader/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/cloud/authservice/LoadAuthService.scala diff --git a/config/loader/azure/databricks.config.reference.hocon b/config/loader/azure/databricks.config.reference.hocon index ba533f231..be6f21a0f 100644 --- a/config/loader/azure/databricks.config.reference.hocon +++ b/config/loader/azure/databricks.config.reference.hocon @@ -30,6 +30,29 @@ # Optimize period per table, that will be used as predicate for the OPTIMIZE command. "eventsOptimizePeriod": "2 days" + + # Optional, default method is 'NoCreds' + # Specifies the auth method to use with 'COPY INTO' statement. + "loadAuthMethod": { + # With 'NoCreds', no credentials will be passed to 'COPY INTO' statement. + # Databricks cluster needs to have permission to access transformer + # output Azure Blob Storage container. More information can be found here: + # https://docs.databricks.com/storage/azure-storage.html + "type": "NoCreds" + } + #"loadAuthMethod": { + # # With 'TempCreds', temporary credentials will be created for every + # # load operation and these temporary credentials will be passed to + # # 'COPY INTO' statement. With this way, Databricks cluster doesn't need + # # permission to access to transformer output Azure Blob Storage container. + # # This access will be provided by temporary credentials. + # "type": "TempCreds" + # + # # If 'TempCreds' load auth method is used, this value will be used as a session duration + # # of temporary credentials used for loading data and folder monitoring. + # # Optional, default value "1 hour" + # "credentialsTtl": "1 hour" + #} }, "schedules": { diff --git a/config/loader/azure/snowflake.config.reference.hocon b/config/loader/azure/snowflake.config.reference.hocon index 02f8608f3..f1f618e52 100644 --- a/config/loader/azure/snowflake.config.reference.hocon +++ b/config/loader/azure/snowflake.config.reference.hocon @@ -37,6 +37,26 @@ "folderMonitoringStage": "snowplow_folders_stage" # An optional host name that will take a priority over automatically derived "jdbcHost": "acme.eu-central-1.snowflake.com" + # Optional, default method is 'NoCreds' + # Specifies the auth method to use with 'COPY INTO' statement. + "loadAuthMethod": { + # With 'NoCreds', no credentials will be passed to 'COPY INTO' statement. + # Instead, 'transformedStage' and 'folderMonitoringStage' specified above will be used. + # More information can be found here: + # https://docs.snowflake.com/en/user-guide/data-load-azure-config + "type": "NoCreds" + } + #"loadAuthMethod": { + # # With 'TempCreds', temporary credentials will be created for every + # # load operation and these temporary credentials will be passed to + # # 'COPY INTO' statement. + # "type": "TempCreds" + # + # # If 'TempCreds' load auth method is used, this value will be used as a session duration + # # of temporary credentials used for loading data and folder monitoring. + # # Optional, default value "1 hour" + # "credentialsTtl": "1 hour" + #} # Either "ResumeWarehouse" (the default) or "Select1". The command the loader runs to prepare the JDBC connection. "readyCheck": "ResumeWarehouse" }, diff --git a/modules/azure/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/azure/AzureBlobStorage.scala b/modules/azure/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/azure/AzureBlobStorage.scala index 10bb60a5f..5d284e503 100644 --- a/modules/azure/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/azure/AzureBlobStorage.scala +++ b/modules/azure/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/azure/AzureBlobStorage.scala @@ -76,7 +76,7 @@ class AzureBlobStorage[F[_]: Async] private (store: AzureStore[F], path: AzureBl .map(authority => Url(path.scheme, authority, Path(path.extractRelative(input)))) private def createBlobObject(url: Url[AzureBlob]) = { - val key = BlobStorage.Key.coerce(s"${path.fullPath}/${url.path.relative}") + val key = path.fullPath.withKey(url.path.relative.show) BlobStorage.BlobObject(key, url.path.representation.size.getOrElse(0L)) } } @@ -89,7 +89,7 @@ object AzureBlobStorage { } def create[F[_]: Async](path: URI, builder: BlobServiceClientBuilder): Resource[F, BlobStorage[F]] = { - val pathParts = parsePath(path.toString) + val pathParts = PathParts.parse(path.toString) val client = builder.endpoint(pathParts.root).buildAsyncClient() createStore(client).map(new AzureBlobStorage(_, pathParts)) } @@ -101,7 +101,7 @@ object AzureBlobStorage { .fold(errors => Resource.raiseError(errors.reduce(Throwables.collapsingSemigroup)), Resource.pure) final case class PathParts( - fullPath: String, + fullPath: Folder, containerName: String, storageAccountName: String, scheme: String, @@ -111,19 +111,24 @@ object AzureBlobStorage { def extractRelative(p: String): String = p.stripPrefix(fullPath) - def root: String = - s"$scheme://$storageAccountName.blob.$endpointSuffix" + def root: Folder = + Folder.coerce(s"$scheme://$storageAccountName.blob.$endpointSuffix") + + def toParquetPath: Folder = + Folder.coerce(s"abfss://$containerName@$storageAccountName.dfs.$endpointSuffix").append(relative) } - def parsePath(path: String): PathParts = { - val parts = BlobUrlParts.parse(path) - PathParts( - fullPath = path, - containerName = parts.getBlobContainerName, - storageAccountName = parts.getAccountName, - scheme = parts.getScheme, - endpointSuffix = parts.getHost.stripPrefix(s"${parts.getAccountName}.blob."), - relative = parts.getBlobName - ) + object PathParts { + def parse(path: String): PathParts = { + val parts = BlobUrlParts.parse(path) + PathParts( + fullPath = Folder.coerce(path), + containerName = parts.getBlobContainerName, + storageAccountName = parts.getAccountName, + scheme = parts.getScheme, + endpointSuffix = parts.getHost.stripPrefix(s"${parts.getAccountName}.blob."), + relative = parts.getBlobName + ) + } } } diff --git a/modules/databricks-loader/src/main/scala/com/snowplowanalytics/snowplow/loader/databricks/Databricks.scala b/modules/databricks-loader/src/main/scala/com/snowplowanalytics/snowplow/loader/databricks/Databricks.scala index d845709e8..d7cc2adfa 100644 --- a/modules/databricks-loader/src/main/scala/com/snowplowanalytics/snowplow/loader/databricks/Databricks.scala +++ b/modules/databricks-loader/src/main/scala/com/snowplowanalytics/snowplow/loader/databricks/Databricks.scala @@ -19,13 +19,15 @@ import doobie.implicits._ import io.circe.syntax._ import com.snowplowanalytics.iglu.core.SchemaKey import com.snowplowanalytics.iglu.schemaddl.migrations.{Migration, SchemaList} +import com.snowplowanalytics.snowplow.rdbloader.azure.AzureBlobStorage import com.snowplowanalytics.snowplow.rdbloader.LoadStatements +import com.snowplowanalytics.snowplow.rdbloader.common.cloud.BlobStorage.Folder import com.snowplowanalytics.snowplow.rdbloader.config.{Config, StorageTarget} import com.snowplowanalytics.snowplow.rdbloader.db.Columns.{ColumnName, ColumnsToCopy, ColumnsToSkip, EventTableColumns} import com.snowplowanalytics.snowplow.rdbloader.db.Migration.{Block, Entity} import com.snowplowanalytics.snowplow.rdbloader.db.{AtomicColumns, Manifest, Statement, Target} import com.snowplowanalytics.snowplow.rdbloader.dsl.DAO -import com.snowplowanalytics.snowplow.rdbloader.cloud.LoadAuthService.LoadAuthMethod +import com.snowplowanalytics.snowplow.rdbloader.cloud.authservice.LoadAuthService.LoadAuthMethod import com.snowplowanalytics.snowplow.rdbloader.discovery.{DataDiscovery, ShreddedType} import com.snowplowanalytics.snowplow.rdbloader.loading.EventsTable @@ -110,16 +112,18 @@ object Databricks { val frManifest = Fragment.const(qualify(Manifest.Name)) sql"SELECT run_id FROM $frTableName MINUS SELECT base FROM $frManifest" case Statement.FoldersCopy(source, loadAuthMethod, _) => + val updatedSource = replaceScheme(source) val frTableName = Fragment.const(qualify(AlertingTempTableName)) - val frPath = Fragment.const0(source) + val frPath = Fragment.const0(updatedSource) val frAuth = loadAuthMethodFragment(loadAuthMethod) sql"""COPY INTO $frTableName FROM (SELECT _C0::VARCHAR(512) RUN_ID FROM '$frPath' $frAuth) FILEFORMAT = CSV""" case Statement.EventsCopy(path, _, toCopy, toSkip, _, loadAuthMethod, _) => + val updatedPath = replaceScheme(path) val frTableName = Fragment.const(qualify(EventsTable.MainName)) - val frPath = Fragment.const0(path.append("output=good")) + val frPath = Fragment.const0(updatedPath.append("output=good")) val nonNulls = toCopy.names.map(_.value) val nulls = toSkip.names.map(c => s"NULL AS ${c.value}") val currentTimestamp = "current_timestamp() AS load_tstamp" @@ -205,6 +209,14 @@ object Databricks { case Some(catalog) => s"${catalog}.${tgt.schema}.$tableName" case None => s"${tgt.schema}.$tableName" } + + private def replaceScheme(path: Folder): Folder = + // If url scheme is https, it means that given path is Azure Blob Storage path. + // We need to convert it to the format suitable to be loaded to Databricks. + if (path.startsWith("https")) + Folder.coerce(AzureBlobStorage.PathParts.parse(path).toParquetPath) + else + path } Right(result) case other => @@ -221,10 +233,14 @@ object Databricks { loadAuthMethod match { case LoadAuthMethod.NoCreds => Fragment.empty - case LoadAuthMethod.TempCreds(awsAccessKey, awsSecretKey, awsSessionToken, _) => + case LoadAuthMethod.TempCreds.AWS(awsAccessKey, awsSecretKey, awsSessionToken, _) => Fragment.const0( s"WITH ( CREDENTIAL (AWS_ACCESS_KEY = '$awsAccessKey', AWS_SECRET_KEY = '$awsSecretKey', AWS_SESSION_TOKEN = '$awsSessionToken') )" ) + case LoadAuthMethod.TempCreds.Azure(sasToken, _) => + Fragment.const0( + s"WITH ( CREDENTIAL (AZURE_SAS_TOKEN = '$sasToken') )" + ) } private def columnsToCopyFromDiscoveredData(discovery: DataDiscovery): ColumnsToCopy = { diff --git a/modules/databricks-loader/src/test/scala/com/snowplowanalytics/snowplow/loader/databricks/DatabricksSpec.scala b/modules/databricks-loader/src/test/scala/com/snowplowanalytics/snowplow/loader/databricks/DatabricksSpec.scala index 7d58e825a..b0e6be174 100644 --- a/modules/databricks-loader/src/test/scala/com/snowplowanalytics/snowplow/loader/databricks/DatabricksSpec.scala +++ b/modules/databricks-loader/src/test/scala/com/snowplowanalytics/snowplow/loader/databricks/DatabricksSpec.scala @@ -22,7 +22,7 @@ import com.snowplowanalytics.snowplow.rdbloader.common.cloud.BlobStorage import com.snowplowanalytics.snowplow.rdbloader.config.{Config, StorageTarget} import com.snowplowanalytics.snowplow.rdbloader.db.Columns.{ColumnName, ColumnsToCopy, ColumnsToSkip} import com.snowplowanalytics.snowplow.rdbloader.db.{Statement, Target} -import com.snowplowanalytics.snowplow.rdbloader.cloud.LoadAuthService.LoadAuthMethod +import com.snowplowanalytics.snowplow.rdbloader.cloud.authservice.LoadAuthService.LoadAuthMethod import com.snowplowanalytics.snowplow.rdbloader.ConfigSpec._ import java.time.Instant @@ -130,7 +130,7 @@ class DatabricksSpec extends Specification { } } - "create sql with credentials for loading" in { + "create sql with credentials for loading from S3" in { val toCopy = ColumnsToCopy( List( ColumnName("app_id"), @@ -144,7 +144,7 @@ class DatabricksSpec extends Specification { ColumnName("contexts_com_acme_yyy_1") ) ) - val loadAuthMethod = LoadAuthMethod.TempCreds("testAccessKey", "testSecretKey", "testSessionToken", Instant.now.plusSeconds(3600)) + val loadAuthMethod = LoadAuthMethod.TempCreds.AWS("testAccessKey", "testSecretKey", "testSessionToken", Instant.now.plusSeconds(3600)) val statement = Statement.EventsCopy(baseFolder, Compression.Gzip, toCopy, toSkip, TypesInfo.WideRow(PARQUET, List.empty), loadAuthMethod, ()) @@ -154,6 +154,33 @@ class DatabricksSpec extends Specification { ) } } + + "create sql with credentials for loading from Azure Blob Storage" in { + val toCopy = ColumnsToCopy( + List( + ColumnName("app_id"), + ColumnName("unstruct_event_com_acme_aaa_1"), + ColumnName("contexts_com_acme_xxx_1") + ) + ) + val toSkip = ColumnsToSkip( + List( + ColumnName("unstruct_event_com_acme_bbb_1"), + ColumnName("contexts_com_acme_yyy_1") + ) + ) + val baseFolder: BlobStorage.Folder = + BlobStorage.Folder.coerce("https://test.blob.core.windows.net/test-container/path1/path2") + val loadAuthMethod = LoadAuthMethod.TempCreds.Azure("testToken", Instant.now.plusSeconds(3600)) + val statement = + Statement.EventsCopy(baseFolder, Compression.Gzip, toCopy, toSkip, TypesInfo.WideRow(PARQUET, List.empty), loadAuthMethod, ()) + + target.toFragment(statement).toString must beLike { case sql => + sql must contain( + s"SELECT app_id,unstruct_event_com_acme_aaa_1,contexts_com_acme_xxx_1,NULL AS unstruct_event_com_acme_bbb_1,NULL AS contexts_com_acme_yyy_1,current_timestamp() AS load_tstamp from 'abfss://test-container@test.dfs.core.windows.net/path1/path2/output=good/' WITH ( CREDENTIAL (AZURE_SAS_TOKEN = '${loadAuthMethod.sasToken}') )" + ) + } + } } } diff --git a/modules/loader/src/main/resources/application.conf b/modules/loader/src/main/resources/application.conf index 94a442ab8..7677f91e1 100644 --- a/modules/loader/src/main/resources/application.conf +++ b/modules/loader/src/main/resources/application.conf @@ -3,6 +3,11 @@ "messageQueue": { "parallelPullCount": 1, "bufferSize": 10 + "consumerConf": { + "enable.auto.commit": "false" + "auto.offset.reset" : "latest" + "group.id": "loader" + } } "monitoring": { "metrics": { diff --git a/modules/loader/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/Loader.scala b/modules/loader/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/Loader.scala index 3f04046ec..c7e53a9bb 100644 --- a/modules/loader/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/Loader.scala +++ b/modules/loader/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/Loader.scala @@ -40,7 +40,8 @@ import com.snowplowanalytics.snowplow.rdbloader.dsl.{ import com.snowplowanalytics.snowplow.rdbloader.dsl.Monitoring import com.snowplowanalytics.snowplow.rdbloader.loading.{EventsTable, Load, Retry, Stage, TargetCheck} import com.snowplowanalytics.snowplow.rdbloader.loading.Retry._ -import com.snowplowanalytics.snowplow.rdbloader.cloud.{JsonPathDiscovery, LoadAuthService} +import com.snowplowanalytics.snowplow.rdbloader.cloud.JsonPathDiscovery +import com.snowplowanalytics.snowplow.rdbloader.cloud.authservice.LoadAuthService import com.snowplowanalytics.snowplow.rdbloader.state.{Control, MakeBusy} object Loader { diff --git a/modules/loader/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/cloud/LoadAuthService.scala b/modules/loader/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/cloud/LoadAuthService.scala deleted file mode 100644 index 347adf3b3..000000000 --- a/modules/loader/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/cloud/LoadAuthService.scala +++ /dev/null @@ -1,168 +0,0 @@ -/* - * Copyright (c) 2014-2022 Snowplow Analytics Ltd. All rights reserved. - * - * This program is licensed to you under the Apache License Version 2.0, - * and you may not use this file except in compliance with the Apache License Version 2.0. - * You may obtain a copy of the Apache License Version 2.0 at http://www.apache.org/licenses/LICENSE-2.0. - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the Apache License Version 2.0 is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the Apache License Version 2.0 for the specific language governing permissions and limitations there under. - */ -package com.snowplowanalytics.snowplow.rdbloader.cloud - -import cats.{Applicative, ~>} -import cats.effect._ -import cats.implicits._ -import com.snowplowanalytics.snowplow.rdbloader.config.StorageTarget -import software.amazon.awssdk.regions.Region -import software.amazon.awssdk.services.sts.StsAsyncClient -import software.amazon.awssdk.services.sts.model.AssumeRoleRequest - -import java.time.Instant - -trait LoadAuthService[F[_]] { self => - def forLoadingEvents: F[LoadAuthService.LoadAuthMethod] - def forFolderMonitoring: F[LoadAuthService.LoadAuthMethod] - - def mapK[G[_]](arrow: F ~> G): LoadAuthService[G] = - new LoadAuthService[G] { - def forLoadingEvents: G[LoadAuthService.LoadAuthMethod] = arrow(self.forLoadingEvents) - def forFolderMonitoring: G[LoadAuthService.LoadAuthMethod] = arrow(self.forFolderMonitoring) - } -} - -object LoadAuthService { - def apply[F[_]](implicit ev: LoadAuthService[F]): LoadAuthService[F] = ev - - /** - * Auth method that is used with COPY INTO statement - */ - sealed trait LoadAuthMethod - - object LoadAuthMethod { - - /** - * Specifies auth method that doesn't use credentials Destination should be already configured - * with some other mean for copying from transformer output bucket - */ - final case object NoCreds extends LoadAuthMethod - - /** - * Specifies auth method that pass temporary credentials to COPY INTO statement - */ - final case class TempCreds( - awsAccessKey: String, - awsSecretKey: String, - awsSessionToken: String, - expires: Instant - ) extends LoadAuthMethod - } - - private trait LoadAuthMethodProvider[F[_]] { - def get: F[LoadAuthService.LoadAuthMethod] - } - - /** - * Get load auth method according to value specified in the config If temporary credentials method - * is specified in the config, it will get temporary credentials with sending request to STS - * service then return credentials. - */ - def aws[F[_]: Async]( - region: String, - eventsLoadAuthMethodConfig: StorageTarget.LoadAuthMethod, - foldersLoadAuthMethodConfig: StorageTarget.LoadAuthMethod - ): Resource[F, LoadAuthService[F]] = - (eventsLoadAuthMethodConfig, foldersLoadAuthMethodConfig) match { - case (StorageTarget.LoadAuthMethod.NoCreds, StorageTarget.LoadAuthMethod.NoCreds) => - noop[F] - case (_, _) => - for { - stsAsyncClient <- Resource.fromAutoCloseable( - Async[F].delay( - StsAsyncClient - .builder() - .region(Region.of(region)) - .build() - ) - ) - eventsAuthProvider <- Resource.eval(awsCreds(stsAsyncClient, eventsLoadAuthMethodConfig)) - foldersAuthProvider <- Resource.eval(awsCreds(stsAsyncClient, foldersLoadAuthMethodConfig)) - } yield new LoadAuthService[F] { - override def forLoadingEvents: F[LoadAuthMethod] = - eventsAuthProvider.get - override def forFolderMonitoring: F[LoadAuthMethod] = - foldersAuthProvider.get - } - } - - private def awsCreds[F[_]: Async]( - client: StsAsyncClient, - loadAuthConfig: StorageTarget.LoadAuthMethod - ): F[LoadAuthMethodProvider[F]] = - loadAuthConfig match { - case StorageTarget.LoadAuthMethod.NoCreds => - Concurrent[F].pure { - new LoadAuthMethodProvider[F] { - def get: F[LoadAuthService.LoadAuthMethod] = Concurrent[F].pure(LoadAuthMethod.NoCreds) - } - } - case tc: StorageTarget.LoadAuthMethod.TempCreds => - awsTempCreds(client, tc) - } - - /** - * Either fetches new temporary credentials from STS, or returns cached temporary credentials if - * they are still valid - * - * The new credentials are valid for *twice* the length of time they requested for. This means - * there is a high chance we can re-use the cached credentials later. - * - * @param client - * Used to fetch new credentials - * @param tempCredsConfig - * Configuration required for the STS request. - */ - private def awsTempCreds[F[_]: Async]( - client: StsAsyncClient, - tempCredsConfig: StorageTarget.LoadAuthMethod.TempCreds - ): F[LoadAuthMethodProvider[F]] = - for { - ref <- Ref.of(Option.empty[LoadAuthMethod.TempCreds]) - } yield new LoadAuthMethodProvider[F] { - override def get: F[LoadAuthMethod] = - for { - opt <- ref.get - now <- Clock[F].realTimeInstant - next <- opt match { - case Some(tc) if tc.expires.isAfter(now.plusMillis(tempCredsConfig.credentialsTtl.toMillis)) => - Concurrent[F].pure(tc) - case _ => - for { - assumeRoleRequest <- Concurrent[F].delay( - AssumeRoleRequest - .builder() - .durationSeconds(tempCredsConfig.credentialsTtl.toSeconds.toInt) - .roleArn(tempCredsConfig.roleArn) - .roleSessionName(tempCredsConfig.roleSessionName) - .build() - ) - response <- Async[F].fromCompletableFuture( - Async[F].delay(client.assumeRole(assumeRoleRequest)) - ) - creds = response.credentials() - } yield LoadAuthMethod.TempCreds(creds.accessKeyId, creds.secretAccessKey, creds.sessionToken, creds.expiration) - } - _ <- ref.set(Some(next)) - } yield next - } - - def noop[F[_]: Applicative]: Resource[F, LoadAuthService[F]] = - Resource.pure[F, LoadAuthService[F]](new LoadAuthService[F] { - override def forLoadingEvents: F[LoadAuthMethod] = - Applicative[F].pure(LoadAuthMethod.NoCreds) - override def forFolderMonitoring: F[LoadAuthMethod] = - Applicative[F].pure(LoadAuthMethod.NoCreds) - }) -} diff --git a/modules/loader/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/cloud/authservice/AWSAuthService.scala b/modules/loader/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/cloud/authservice/AWSAuthService.scala new file mode 100644 index 000000000..66ec8b21a --- /dev/null +++ b/modules/loader/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/cloud/authservice/AWSAuthService.scala @@ -0,0 +1,92 @@ +/* + * Copyright (c) 2014-2023 Snowplow Analytics Ltd. All rights reserved. + * + * This program is licensed to you under the Apache License Version 2.0, + * and you may not use this file except in compliance with the Apache License Version 2.0. + * You may obtain a copy of the Apache License Version 2.0 at http://www.apache.org/licenses/LICENSE-2.0. + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the Apache License Version 2.0 is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the Apache License Version 2.0 for the specific language governing permissions and limitations there under. + */ +package com.snowplowanalytics.snowplow.rdbloader.cloud.authservice + +import cats.effect._ +import cats.implicits._ + +import software.amazon.awssdk.regions.Region +import software.amazon.awssdk.services.sts.StsAsyncClient +import software.amazon.awssdk.services.sts.model.AssumeRoleRequest + +import com.snowplowanalytics.snowplow.rdbloader.config.StorageTarget +import com.snowplowanalytics.snowplow.rdbloader.cloud.authservice.LoadAuthService._ + +object AWSAuthService { + + /** + * Get load auth method according to value specified in the config. If temporary credentials + * method is specified in the config, it will get temporary credentials with sending request to + * STS service then return credentials. + */ + def create[F[_]: Async]( + region: String, + eventsLoadAuthMethodConfig: StorageTarget.LoadAuthMethod, + foldersLoadAuthMethodConfig: StorageTarget.LoadAuthMethod + ): Resource[F, LoadAuthService[F]] = + (eventsLoadAuthMethodConfig, foldersLoadAuthMethodConfig) match { + case (e: StorageTarget.LoadAuthMethod.AWS, f: StorageTarget.LoadAuthMethod.AWS) => + (e, f) match { + case (StorageTarget.LoadAuthMethod.NoCreds, StorageTarget.LoadAuthMethod.NoCreds) => + noop[F] + case (_, _) => + for { + stsAsyncClient <- createClient(region) + provider = authMethodProvider[F](stsAsyncClient)(_) + s <- LoadAuthService.create(provider(e), provider(f)) + } yield s + } + case (_, _) => + Resource.raiseError[F, LoadAuthService[F], Throwable]( + new IllegalStateException("AWS auth service needs AWS temp credentials configuration") + ) + } + + private def createClient[F[_]: Async](region: String): Resource[F, StsAsyncClient] = + Resource.fromAutoCloseable( + Async[F].delay( + StsAsyncClient + .builder() + .region(Region.of(region)) + .build() + ) + ) + + private def authMethodProvider[F[_]: Async]( + client: StsAsyncClient + )( + loadAuthConfig: StorageTarget.LoadAuthMethod.AWS + ): F[LoadAuthMethodProvider[F]] = + loadAuthConfig match { + case StorageTarget.LoadAuthMethod.NoCreds => + LoadAuthMethodProvider.noop + case tc: StorageTarget.LoadAuthMethod.TempCreds.AWSTempCreds => + credsCache( + credentialsTtl = tc.credentialsTtl, + getCreds = for { + assumeRoleRequest <- Concurrent[F].delay( + AssumeRoleRequest + .builder() + .durationSeconds(tc.credentialsTtl.toSeconds.toInt) + .roleArn(tc.roleArn) + .roleSessionName(tc.roleSessionName) + .build() + ) + response <- Async[F].fromCompletableFuture( + Async[F].delay(client.assumeRole(assumeRoleRequest)) + ) + creds = response.credentials() + } yield LoadAuthMethod.TempCreds.AWS(creds.accessKeyId, creds.secretAccessKey, creds.sessionToken, creds.expiration) + ) + } +} diff --git a/modules/loader/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/cloud/authservice/AzureAuthService.scala b/modules/loader/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/cloud/authservice/AzureAuthService.scala new file mode 100644 index 000000000..a1c04c742 --- /dev/null +++ b/modules/loader/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/cloud/authservice/AzureAuthService.scala @@ -0,0 +1,92 @@ +/* + * Copyright (c) 2014-2023 Snowplow Analytics Ltd. All rights reserved. + * + * This program is licensed to you under the Apache License Version 2.0, + * and you may not use this file except in compliance with the Apache License Version 2.0. + * You may obtain a copy of the Apache License Version 2.0 at http://www.apache.org/licenses/LICENSE-2.0. + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the Apache License Version 2.0 is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the Apache License Version 2.0 for the specific language governing permissions and limitations there under. + */ + +package com.snowplowanalytics.snowplow.rdbloader.cloud.authservice + +import java.time.OffsetDateTime + +import cats.effect._ + +import com.azure.identity.DefaultAzureCredentialBuilder +import com.azure.storage.blob.sas.{BlobContainerSasPermission, BlobServiceSasSignatureValues} +import com.azure.storage.blob.{BlobContainerClient, BlobServiceClient, BlobServiceClientBuilder} + +import com.snowplowanalytics.snowplow.rdbloader.config.StorageTarget +import com.snowplowanalytics.snowplow.rdbloader.azure.AzureBlobStorage +import com.snowplowanalytics.snowplow.rdbloader.cloud.authservice.LoadAuthService._ + +object AzureAuthService { + + def create[F[_]: Async]( + blobStorageEndpoint: String, + eventsLoadAuthMethodConfig: StorageTarget.LoadAuthMethod, + foldersLoadAuthMethodConfig: StorageTarget.LoadAuthMethod + ): Resource[F, LoadAuthService[F]] = + (eventsLoadAuthMethodConfig, foldersLoadAuthMethodConfig) match { + case (e: StorageTarget.LoadAuthMethod.Azure, f: StorageTarget.LoadAuthMethod.Azure) => + (e, f) match { + case (StorageTarget.LoadAuthMethod.NoCreds, StorageTarget.LoadAuthMethod.NoCreds) => + noop[F] + case (_, _) => + for { + (blobServiceClient, blobContainerClient) <- createClients(blobStorageEndpoint) + provider = authMethodProvider[F](blobServiceClient, blobContainerClient)(_) + s <- LoadAuthService.create(provider(e), provider(f)) + } yield s + } + case (_, _) => + Resource.raiseError[F, LoadAuthService[F], Throwable]( + new IllegalStateException("Azure auth service needs Azure temp credentials configuration") + ) + } + + private def createClients[F[_]: Async]( + blobStorageEndpoint: String + ): Resource[F, (BlobServiceClient, BlobContainerClient)] = + Resource.eval( + Async[F].delay { + val builder = new BlobServiceClientBuilder() + .credential(new DefaultAzureCredentialBuilder().build) + .endpoint(blobStorageEndpoint) + val pathParts = AzureBlobStorage.PathParts.parse(blobStorageEndpoint) + val blobServiceClient = builder.buildClient() + val blobContainerClient = blobServiceClient.getBlobContainerClient(pathParts.containerName) + (blobServiceClient, blobContainerClient) + } + ) + + private def authMethodProvider[F[_]: Async]( + blobServiceClient: BlobServiceClient, + blobContainerClient: BlobContainerClient + )( + loadAuthConfig: StorageTarget.LoadAuthMethod.Azure + ): F[LoadAuthMethodProvider[F]] = + loadAuthConfig match { + case StorageTarget.LoadAuthMethod.NoCreds => + LoadAuthMethodProvider.noop + case tc: StorageTarget.LoadAuthMethod.TempCreds.AzureTempCreds => + credsCache( + credentialsTtl = tc.credentialsTtl, + getCreds = Async[F].delay { + val keyStart = OffsetDateTime.now() + val keyExpiry = OffsetDateTime.now().plusSeconds(tc.credentialsTtl.toSeconds) + val userDelegationKey = blobServiceClient.getUserDelegationKey(keyStart, keyExpiry) + val blobContainerSas = new BlobContainerSasPermission() + blobContainerSas.setReadPermission(true).setListPermission(true) + val blobServiceSasSignatureValues = new BlobServiceSasSignatureValues(keyExpiry, blobContainerSas) + val sasToken = blobContainerClient.generateUserDelegationSas(blobServiceSasSignatureValues, userDelegationKey) + LoadAuthMethod.TempCreds.Azure(sasToken, keyExpiry.toInstant) + } + ) + } +} diff --git a/modules/loader/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/cloud/authservice/LoadAuthService.scala b/modules/loader/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/cloud/authservice/LoadAuthService.scala new file mode 100644 index 000000000..a782bf98f --- /dev/null +++ b/modules/loader/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/cloud/authservice/LoadAuthService.scala @@ -0,0 +1,134 @@ +/* + * Copyright (c) 2014-2023 Snowplow Analytics Ltd. All rights reserved. + * + * This program is licensed to you under the Apache License Version 2.0, + * and you may not use this file except in compliance with the Apache License Version 2.0. + * You may obtain a copy of the Apache License Version 2.0 at http://www.apache.org/licenses/LICENSE-2.0. + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the Apache License Version 2.0 is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the Apache License Version 2.0 for the specific language governing permissions and limitations there under. + */ +package com.snowplowanalytics.snowplow.rdbloader.cloud.authservice + +import java.time.Instant + +import cats.effect._ +import cats.implicits._ +import cats.{Applicative, ~>} + +import scala.concurrent.duration.FiniteDuration + +trait LoadAuthService[F[_]] { self => + def forLoadingEvents: F[LoadAuthService.LoadAuthMethod] + def forFolderMonitoring: F[LoadAuthService.LoadAuthMethod] + + def mapK[G[_]](arrow: F ~> G): LoadAuthService[G] = + new LoadAuthService[G] { + def forLoadingEvents: G[LoadAuthService.LoadAuthMethod] = arrow(self.forLoadingEvents) + def forFolderMonitoring: G[LoadAuthService.LoadAuthMethod] = arrow(self.forFolderMonitoring) + } +} + +object LoadAuthService { + def apply[F[_]](implicit ev: LoadAuthService[F]): LoadAuthService[F] = ev + + /** + * Auth method that is used with COPY INTO statement + */ + sealed trait LoadAuthMethod + + object LoadAuthMethod { + + /** + * Specifies auth method that doesn't use credentials Destination should be already configured + * with some other mean for copying from transformer output bucket + */ + final case object NoCreds extends LoadAuthMethod + + /** + * Specifies auth method that pass temporary credentials to COPY INTO statement + */ + sealed trait TempCreds extends LoadAuthMethod { + def expires: Instant + } + + object TempCreds { + + final case class AWS( + awsAccessKey: String, + awsSecretKey: String, + awsSessionToken: String, + expires: Instant + ) extends TempCreds + + final case class Azure( + sasToken: String, + expires: Instant + ) extends TempCreds + } + } + + trait LoadAuthMethodProvider[F[_]] { + def get: F[LoadAuthService.LoadAuthMethod] + } + + object LoadAuthMethodProvider { + def noop[F[_]: Concurrent]: F[LoadAuthMethodProvider[F]] = + Concurrent[F].pure { + new LoadAuthMethodProvider[F] { + def get: F[LoadAuthService.LoadAuthMethod] = Concurrent[F].pure(LoadAuthMethod.NoCreds) + } + } + } + + def create[F[_]: Async]( + eventsAuthProvider: F[LoadAuthMethodProvider[F]], + foldersAuthProvider: F[LoadAuthMethodProvider[F]] + ): Resource[F, LoadAuthService[F]] = + Resource.eval( + for { + e <- eventsAuthProvider + f <- foldersAuthProvider + } yield new LoadAuthService[F] { + override def forLoadingEvents: F[LoadAuthMethod] = e.get + override def forFolderMonitoring: F[LoadAuthMethod] = f.get + } + ) + + def noop[F[_]: Applicative]: Resource[F, LoadAuthService[F]] = + Resource.pure[F, LoadAuthService[F]](new LoadAuthService[F] { + override def forLoadingEvents: F[LoadAuthMethod] = + Applicative[F].pure(LoadAuthMethod.NoCreds) + override def forFolderMonitoring: F[LoadAuthMethod] = + Applicative[F].pure(LoadAuthMethod.NoCreds) + }) + + /** + * Either fetches new temporary credentials, or returns cached temporary credentials if they are + * still valid + * + * The new credentials are valid for *twice* the length of time they requested for. This means + * there is a high chance we can re-use the cached credentials later. + */ + def credsCache[F[_]: Async]( + credentialsTtl: FiniteDuration, + getCreds: => F[LoadAuthMethod.TempCreds] + ): F[LoadAuthMethodProvider[F]] = + for { + ref <- Ref.of(Option.empty[LoadAuthMethod.TempCreds]) + } yield new LoadAuthMethodProvider[F] { + override def get: F[LoadAuthMethod] = + for { + opt <- ref.get + now <- Clock[F].realTimeInstant + next <- opt match { + case Some(tc) if tc.expires.isAfter(now.plusMillis(credentialsTtl.toMillis)) => + Concurrent[F].pure(tc) + case _ => getCreds + } + _ <- ref.set(Some(next)) + } yield next + } +} diff --git a/modules/loader/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/config/Config.scala b/modules/loader/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/config/Config.scala index 5b8963e4f..09196cf8f 100644 --- a/modules/loader/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/config/Config.scala +++ b/modules/loader/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/config/Config.scala @@ -330,24 +330,31 @@ object Config { /** Post-decoding validation, making sure different parts are consistent */ def validateConfig(config: Config[StorageTarget]): List[String] = List( - authMethodValidation(config), + authMethodValidation(config.storage.eventsLoadAuthMethod, config.cloud), + authMethodValidation(config.storage.foldersLoadAuthMethod, config.cloud), targetSnowflakeValidation(config), targetRedshiftValidation(config) ).flatten - private def authMethodValidation(config: Config[StorageTarget]): List[String] = - config.cloud match { + private def authMethodValidation(loadAuthMethod: StorageTarget.LoadAuthMethod, cloud: Config.Cloud): List[String] = + cloud match { case _: Config.Cloud.GCP => - (config.storage.foldersLoadAuthMethod, config.storage.eventsLoadAuthMethod) match { - case (StorageTarget.LoadAuthMethod.NoCreds, StorageTarget.LoadAuthMethod.NoCreds) => Nil + loadAuthMethod match { + case StorageTarget.LoadAuthMethod.NoCreds => Nil case _ => List("Only 'NoCreds' load auth method is supported with GCP") } + case _: Config.Cloud.AWS => + loadAuthMethod match { + case StorageTarget.LoadAuthMethod.NoCreds => Nil + case _: StorageTarget.LoadAuthMethod.TempCreds.AWSTempCreds => Nil + case _ => List("Given 'TempCreds' configuration isn't suitable for AWS") + } case _: Config.Cloud.Azure => - (config.storage.foldersLoadAuthMethod, config.storage.eventsLoadAuthMethod) match { - case (StorageTarget.LoadAuthMethod.NoCreds, StorageTarget.LoadAuthMethod.NoCreds) => Nil - case _ => List("Only 'NoCreds' load auth method is supported with Azure") + loadAuthMethod match { + case StorageTarget.LoadAuthMethod.NoCreds => Nil + case _: StorageTarget.LoadAuthMethod.TempCreds.AzureTempCreds => Nil + case _ => List("Given 'TempCreds' configuration isn't suitable for Azure") } - case _ => Nil } def targetSnowflakeValidation(config: Config[StorageTarget]): List[String] = diff --git a/modules/loader/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/config/StorageTarget.scala b/modules/loader/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/config/StorageTarget.scala index d0122a221..887819a90 100644 --- a/modules/loader/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/config/StorageTarget.scala +++ b/modules/loader/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/config/StorageTarget.scala @@ -381,12 +381,26 @@ object StorageTarget { sealed trait LoadAuthMethod extends Product with Serializable object LoadAuthMethod { - final case object NoCreds extends LoadAuthMethod - final case class TempCreds( - roleArn: String, - roleSessionName: String, - credentialsTtl: FiniteDuration - ) extends LoadAuthMethod + sealed trait Azure + sealed trait AWS + + final case object NoCreds extends LoadAuthMethod with Azure with AWS + + sealed trait TempCreds extends LoadAuthMethod + + object TempCreds { + final case class AWSTempCreds( + roleArn: String, + roleSessionName: String, + credentialsTtl: FiniteDuration + ) extends TempCreds + with AWS + + final case class AzureTempCreds( + credentialsTtl: FiniteDuration + ) extends TempCreds + with Azure + } } /** @@ -467,7 +481,18 @@ object StorageTarget { } implicit def tempCredsAuthMethodDecoder: Decoder[LoadAuthMethod.TempCreds] = - deriveDecoder[LoadAuthMethod.TempCreds] + Decoder.instance { cur => + if (cur.downField("roleArn").succeeded) + cur.as[LoadAuthMethod.TempCreds.AWSTempCreds] + else + cur.as[LoadAuthMethod.TempCreds.AzureTempCreds] + } + + implicit def awsTempCredsAuthMethodDecoder: Decoder[LoadAuthMethod.TempCreds.AWSTempCreds] = + deriveDecoder[LoadAuthMethod.TempCreds.AWSTempCreds] + + implicit def azureTempCredsAuthMethodDecoder: Decoder[LoadAuthMethod.TempCreds.AzureTempCreds] = + deriveDecoder[LoadAuthMethod.TempCreds.AzureTempCreds] // Custom decoder for backward compatibility implicit def snowflakeStageDecoder: Decoder[Snowflake.Stage] = diff --git a/modules/loader/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/db/Statement.scala b/modules/loader/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/db/Statement.scala index 896c259f2..eb01cea1b 100644 --- a/modules/loader/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/db/Statement.scala +++ b/modules/loader/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/db/Statement.scala @@ -18,7 +18,7 @@ import com.snowplowanalytics.snowplow.rdbloader.common.LoaderMessage import com.snowplowanalytics.snowplow.rdbloader.common.cloud.BlobStorage import com.snowplowanalytics.snowplow.rdbloader.common.config.TransformerConfig.Compression import com.snowplowanalytics.snowplow.rdbloader.db.Columns.{ColumnsToCopy, ColumnsToSkip} -import com.snowplowanalytics.snowplow.rdbloader.cloud.LoadAuthService.LoadAuthMethod +import com.snowplowanalytics.snowplow.rdbloader.cloud.authservice.LoadAuthService.LoadAuthMethod import com.snowplowanalytics.snowplow.rdbloader.discovery.ShreddedType import com.snowplowanalytics.snowplow.rdbloader.loading.EventsTable diff --git a/modules/loader/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/dsl/Environment.scala b/modules/loader/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/dsl/Environment.scala index 1817f0306..c3b68e344 100644 --- a/modules/loader/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/dsl/Environment.scala +++ b/modules/loader/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/dsl/Environment.scala @@ -27,7 +27,8 @@ import com.snowplowanalytics.snowplow.rdbloader.common.cloud.{BlobStorage, Queue import com.snowplowanalytics.snowplow.rdbloader.aws.{EC2ParameterStore, S3, SQS} import com.snowplowanalytics.snowplow.rdbloader.azure.{AzureBlobStorage, KafkaConsumer} import com.snowplowanalytics.snowplow.rdbloader.gcp.{GCS, Pubsub, SecretManager} -import com.snowplowanalytics.snowplow.rdbloader.cloud.{JsonPathDiscovery, LoadAuthService} +import com.snowplowanalytics.snowplow.rdbloader.cloud.JsonPathDiscovery +import com.snowplowanalytics.snowplow.rdbloader.cloud.authservice.{AWSAuthService, AzureAuthService, LoadAuthService} import com.snowplowanalytics.snowplow.rdbloader.state.{Control, State} import com.snowplowanalytics.snowplow.rdbloader.config.{CliConfig, Config, StorageTarget} import com.snowplowanalytics.snowplow.rdbloader.config.Config.Cloud @@ -158,8 +159,8 @@ object Environment { Some(postProcess) ) loadAuthService <- - LoadAuthService - .aws[F](c.region.name, config.storage.eventsLoadAuthMethod, config.storage.foldersLoadAuthMethod) + AWSAuthService + .create[F](c.region.name, config.storage.eventsLoadAuthMethod, config.storage.foldersLoadAuthMethod) jsonPathDiscovery = JsonPathDiscovery.aws[F](c.region.name) secretStore <- EC2ParameterStore.secretStore[F] } yield CloudServices(blobStorage, queueConsumer, loadAuthService, jsonPathDiscovery, secretStore) @@ -182,7 +183,9 @@ object Environment { } yield CloudServices(blobStorage, queueConsumer, loadAuthService, jsonPathDiscovery, secretStore) case c: Cloud.Azure => for { - loadAuthService <- LoadAuthService.noop[F] + loadAuthService <- + AzureAuthService + .create[F](c.blobStorageEndpoint.toString, config.storage.eventsLoadAuthMethod, config.storage.foldersLoadAuthMethod) jsonPathDiscovery = JsonPathDiscovery.noop[F] implicit0(blobStorage: BlobStorage[F]) <- AzureBlobStorage.createDefault[F](c.blobStorageEndpoint) queueConsumer <- KafkaConsumer.consumer[F]( diff --git a/modules/loader/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/dsl/FolderMonitoring.scala b/modules/loader/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/dsl/FolderMonitoring.scala index 15b7d8aec..2be8c997a 100644 --- a/modules/loader/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/dsl/FolderMonitoring.scala +++ b/modules/loader/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/dsl/FolderMonitoring.scala @@ -19,7 +19,7 @@ import cats.{Applicative, Functor, Monad, MonadThrow} import cats.implicits._ import cats.effect.kernel.{Async, Clock, Ref, Sync, Temporal} import cats.effect.std.Semaphore -import com.snowplowanalytics.snowplow.rdbloader.cloud.LoadAuthService +import com.snowplowanalytics.snowplow.rdbloader.cloud.authservice.LoadAuthService import com.snowplowanalytics.snowplow.rdbloader.common.cloud.BlobStorage import com.snowplowanalytics.snowplow.rdbloader.common.cloud.BlobStorage.Folder import doobie.util.Get diff --git a/modules/loader/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/loading/Load.scala b/modules/loader/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/loading/Load.scala index 30b06706b..d80c2c75d 100644 --- a/modules/loader/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/loading/Load.scala +++ b/modules/loader/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/loading/Load.scala @@ -19,10 +19,10 @@ import cats.effect.Clock import retry.Sleep // This project +import com.snowplowanalytics.snowplow.rdbloader.cloud.authservice.LoadAuthService import com.snowplowanalytics.snowplow.rdbloader.common.LoaderMessage import com.snowplowanalytics.snowplow.rdbloader.common.cloud.BlobStorage import com.snowplowanalytics.snowplow.rdbloader.db.{Control, Manifest, Migration, Target} -import com.snowplowanalytics.snowplow.rdbloader.cloud.LoadAuthService import com.snowplowanalytics.snowplow.rdbloader.discovery.DataDiscovery import com.snowplowanalytics.snowplow.rdbloader.dsl.{DAO, Iglu, Logging, Monitoring, Transaction} import com.snowplowanalytics.snowplow.rdbloader.dsl.metrics.Metrics diff --git a/modules/loader/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/package.scala b/modules/loader/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/package.scala index 87b43e9a2..f987c80d8 100644 --- a/modules/loader/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/package.scala +++ b/modules/loader/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/package.scala @@ -24,7 +24,7 @@ import doobie.implicits.javasql._ import io.circe.parser.parse import com.snowplowanalytics.iglu.core.SchemaKey -import com.snowplowanalytics.snowplow.rdbloader.cloud.LoadAuthService +import com.snowplowanalytics.snowplow.rdbloader.cloud.authservice.LoadAuthService import com.snowplowanalytics.snowplow.rdbloader.common.LoaderMessage.{Count, ManifestType, Timestamps} import com.snowplowanalytics.snowplow.rdbloader.common.cloud.BlobStorage import com.snowplowanalytics.snowplow.rdbloader.common.config.TransformerConfig.Compression diff --git a/modules/loader/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/ConfigSpec.scala b/modules/loader/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/ConfigSpec.scala index 1f32c7325..8e3a5495a 100644 --- a/modules/loader/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/ConfigSpec.scala +++ b/modules/loader/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/ConfigSpec.scala @@ -89,6 +89,56 @@ class ConfigSpec extends Specification { ) ) } + "return suitable results for different pairs of cloud type - load auth method" in { + val dest = exampleSnowflake.copy( + folderMonitoringStage = None, + transformedStage = None + ) + val noCredsDest = dest.copy( + loadAuthMethod = StorageTarget.LoadAuthMethod.NoCreds, + folderMonitoringStage = Some(exampleFolderMonitoringStage), + transformedStage = Some(exampleTransformedStage) + ) + val awsTempCredsDest = dest.copy( + loadAuthMethod = exampleTempCreds + ) + val azureTempCredsDest = dest.copy( + loadAuthMethod = StorageTarget.LoadAuthMethod.TempCreds.AzureTempCreds(1.hour) + ) + val awsConfig = exampleCloud + val gcpConfig = Config.Cloud.GCP( + messageQueue = Config.Cloud.GCP.Pubsub("projects/project-id/subscriptions/subscription-id", None, 1, 1) + ) + val azureConfig = Config.Cloud.Azure( + URI.create("https://test.blob.core.windows.net/test-container/"), + Config.Cloud.Azure.Kafka("test-topic", "127.0.0.1:8080", Map.empty) + ) + val config = exampleConfig + + Config.validateConfig(config.copy(storage = noCredsDest, cloud = awsConfig)) must beEmpty + Config.validateConfig(config.copy(storage = noCredsDest, cloud = gcpConfig)) must beEmpty + Config.validateConfig(config.copy(storage = noCredsDest, cloud = azureConfig)) must beEmpty + Config.validateConfig(config.copy(storage = awsTempCredsDest, cloud = awsConfig)) must beEmpty + Config.validateConfig(config.copy(storage = azureTempCredsDest, cloud = azureConfig)) must beEmpty + Config.validateConfig(config.copy(storage = awsTempCredsDest, cloud = gcpConfig)) must beEqualTo( + List( + "Only 'NoCreds' load auth method is supported with GCP", + "Only 'NoCreds' load auth method is supported with GCP" + ) + ) + Config.validateConfig(config.copy(storage = azureTempCredsDest, cloud = awsConfig)) must beEqualTo( + List( + "Given 'TempCreds' configuration isn't suitable for AWS", + "Given 'TempCreds' configuration isn't suitable for AWS" + ) + ) + Config.validateConfig(config.copy(storage = awsTempCredsDest, cloud = azureConfig)) must beEqualTo( + List( + "Given 'TempCreds' configuration isn't suitable for Azure", + "Given 'TempCreds' configuration isn't suitable for Azure" + ) + ) + } } } @@ -166,7 +216,7 @@ object ConfigSpec { val exampleTimeouts: Config.Timeouts = Config.Timeouts(45.minutes, 10.minutes, 5.minutes, 20.minutes, 30.seconds) val exampleRetries: Config.Retries = Config.Retries(Config.Strategy.Exponential, Some(3), 30.seconds, Some(1.hour)) val exampleReadyCheck: Config.Retries = Config.Retries(Config.Strategy.Constant, None, 15.seconds, Some(10.minutes)) - val exampleTempCreds = StorageTarget.LoadAuthMethod.TempCreds( + val exampleTempCreds = StorageTarget.LoadAuthMethod.TempCreds.AWSTempCreds( "test_role_arn", "test_role_session_name", 1.hour diff --git a/modules/loader/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/dsl/FolderMonitoringSpec.scala b/modules/loader/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/dsl/FolderMonitoringSpec.scala index 51e9853d4..2f647c8f2 100644 --- a/modules/loader/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/dsl/FolderMonitoringSpec.scala +++ b/modules/loader/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/dsl/FolderMonitoringSpec.scala @@ -19,8 +19,8 @@ import cats.effect.unsafe.implicits.global import com.snowplowanalytics.snowplow.rdbloader.common.cloud.BlobStorage import com.snowplowanalytics.snowplow.rdbloader.config.Config import com.snowplowanalytics.snowplow.rdbloader.db.{Statement, Target} -import com.snowplowanalytics.snowplow.rdbloader.cloud.LoadAuthService -import com.snowplowanalytics.snowplow.rdbloader.cloud.LoadAuthService.LoadAuthMethod +import com.snowplowanalytics.snowplow.rdbloader.cloud.authservice.LoadAuthService.LoadAuthMethod +import com.snowplowanalytics.snowplow.rdbloader.cloud.authservice.LoadAuthService import com.snowplowanalytics.snowplow.rdbloader.test.{ Pure, PureAWS, diff --git a/modules/loader/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/loading/LoadSpec.scala b/modules/loader/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/loading/LoadSpec.scala index 6be16e5b6..2306e1f96 100644 --- a/modules/loader/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/loading/LoadSpec.scala +++ b/modules/loader/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/loading/LoadSpec.scala @@ -25,10 +25,10 @@ import com.snowplowanalytics.snowplow.rdbloader.common.config.Semver import com.snowplowanalytics.snowplow.rdbloader.discovery.{DataDiscovery, ShreddedType} import com.snowplowanalytics.snowplow.rdbloader.dsl.{DAO, Iglu, Logging, RetryingTransaction, Transaction} import com.snowplowanalytics.snowplow.rdbloader.db.{Manifest, Statement} -import com.snowplowanalytics.snowplow.rdbloader.cloud.LoadAuthService -import com.snowplowanalytics.snowplow.rdbloader.cloud.LoadAuthService.LoadAuthMethod +import com.snowplowanalytics.snowplow.rdbloader.cloud.authservice.LoadAuthService.LoadAuthMethod import org.specs2.mutable.Specification import com.snowplowanalytics.snowplow.rdbloader.SpecHelpers._ +import com.snowplowanalytics.snowplow.rdbloader.cloud.authservice.LoadAuthService import com.snowplowanalytics.snowplow.rdbloader.common.cloud.BlobStorage import com.snowplowanalytics.snowplow.rdbloader.db.Columns.{ColumnsToCopy, ColumnsToSkip} import com.snowplowanalytics.snowplow.rdbloader.test.TestState.LogEntry diff --git a/modules/loader/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/test/PureLoadAuthService.scala b/modules/loader/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/test/PureLoadAuthService.scala index 54cac1974..63892a8da 100644 --- a/modules/loader/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/test/PureLoadAuthService.scala +++ b/modules/loader/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/test/PureLoadAuthService.scala @@ -12,7 +12,7 @@ */ package com.snowplowanalytics.snowplow.rdbloader.test -import com.snowplowanalytics.snowplow.rdbloader.cloud.LoadAuthService +import com.snowplowanalytics.snowplow.rdbloader.cloud.authservice.LoadAuthService object PureLoadAuthService { def interpreter: LoadAuthService[Pure] = diff --git a/modules/redshift-loader/src/main/scala/com/snowplowanalytics/snowplow/loader/redshift/Redshift.scala b/modules/redshift-loader/src/main/scala/com/snowplowanalytics/snowplow/loader/redshift/Redshift.scala index 1dde1f44d..4369ca9f3 100644 --- a/modules/redshift-loader/src/main/scala/com/snowplowanalytics/snowplow/loader/redshift/Redshift.scala +++ b/modules/redshift-loader/src/main/scala/com/snowplowanalytics/snowplow/loader/redshift/Redshift.scala @@ -37,7 +37,7 @@ import com.snowplowanalytics.snowplow.rdbloader.config.{Config, StorageTarget} import com.snowplowanalytics.snowplow.rdbloader.db.Columns.{ColumnsToCopy, ColumnsToSkip, EventTableColumns} import com.snowplowanalytics.snowplow.rdbloader.db.Migration.{Block, Entity, Item, NoPreStatements, NoStatements} import com.snowplowanalytics.snowplow.rdbloader.db.{AtomicColumns, Manifest, Statement, Target} -import com.snowplowanalytics.snowplow.rdbloader.cloud.LoadAuthService.LoadAuthMethod +import com.snowplowanalytics.snowplow.rdbloader.cloud.authservice.LoadAuthService.LoadAuthMethod import com.snowplowanalytics.snowplow.rdbloader.discovery.{DataDiscovery, ShreddedType} import com.snowplowanalytics.snowplow.rdbloader.dsl.DAO import com.snowplowanalytics.snowplow.rdbloader.loading.EventsTable @@ -305,10 +305,12 @@ object Redshift { case LoadAuthMethod.NoCreds => val roleArn = roleArnOpt.getOrElse(throw new IllegalStateException("roleArn needs to be provided with 'NoCreds' auth method")) Fragment.const0(s"aws_iam_role=$roleArn") - case LoadAuthMethod.TempCreds(awsAccessKey, awsSecretKey, awsSessionToken, _) => + case LoadAuthMethod.TempCreds.AWS(awsAccessKey, awsSecretKey, awsSessionToken, _) => Fragment.const0( s"aws_access_key_id=$awsAccessKey;aws_secret_access_key=$awsSecretKey;token=$awsSessionToken" ) + case _: LoadAuthMethod.TempCreds.Azure => + throw new IllegalStateException("Azure temp credentials can't be used with Redshift") } val ManifestColumns = List( diff --git a/modules/redshift-loader/src/test/scala/com/snowplowanalytics/snowplow/loader/redshift/RedshiftSpec.scala b/modules/redshift-loader/src/test/scala/com/snowplowanalytics/snowplow/loader/redshift/RedshiftSpec.scala index 21f05de86..9e05482a9 100644 --- a/modules/redshift-loader/src/test/scala/com/snowplowanalytics/snowplow/loader/redshift/RedshiftSpec.scala +++ b/modules/redshift-loader/src/test/scala/com/snowplowanalytics/snowplow/loader/redshift/RedshiftSpec.scala @@ -19,7 +19,7 @@ import com.snowplowanalytics.snowplow.rdbloader.db.{Migration, Target} import org.specs2.mutable.Specification import com.snowplowanalytics.snowplow.loader.redshift.db.MigrationSpec import com.snowplowanalytics.snowplow.rdbloader.SpecHelpers.validConfig -import com.snowplowanalytics.snowplow.rdbloader.cloud.LoadAuthService +import com.snowplowanalytics.snowplow.rdbloader.cloud.authservice.LoadAuthService import com.snowplowanalytics.snowplow.rdbloader.common.LoaderMessage.SnowplowEntity.{Context, SelfDescribingEvent} import com.snowplowanalytics.snowplow.rdbloader.common.LoaderMessage.TypesInfo import com.snowplowanalytics.snowplow.rdbloader.common.cloud.BlobStorage.Folder diff --git a/modules/snowflake-loader/src/main/scala/com/snowplowanalytics/snowplow/loader/snowflake/Snowflake.scala b/modules/snowflake-loader/src/main/scala/com/snowplowanalytics/snowplow/loader/snowflake/Snowflake.scala index a07a64f14..cfcb5c333 100644 --- a/modules/snowflake-loader/src/main/scala/com/snowplowanalytics/snowplow/loader/snowflake/Snowflake.scala +++ b/modules/snowflake-loader/src/main/scala/com/snowplowanalytics/snowplow/loader/snowflake/Snowflake.scala @@ -41,7 +41,7 @@ import com.snowplowanalytics.snowplow.rdbloader.config.{Config, StorageTarget} import com.snowplowanalytics.snowplow.rdbloader.db.Columns.{ColumnName, ColumnsToCopy, ColumnsToSkip, EventTableColumns} import com.snowplowanalytics.snowplow.rdbloader.db.Migration.{Block, Entity, Item} import com.snowplowanalytics.snowplow.rdbloader.db.{AtomicColumns, Manifest, Statement, Target} -import com.snowplowanalytics.snowplow.rdbloader.cloud.LoadAuthService.LoadAuthMethod +import com.snowplowanalytics.snowplow.rdbloader.cloud.authservice.LoadAuthService.LoadAuthMethod import com.snowplowanalytics.snowplow.rdbloader.discovery.{DataDiscovery, ShreddedType} import com.snowplowanalytics.snowplow.rdbloader.loading.EventsTable import com.snowplowanalytics.snowplow.rdbloader.dsl.DAO @@ -157,7 +157,8 @@ object Snowflake { val frTableName = Fragment.const(qualify(AlertingTempTableName)) val frManifest = Fragment.const(qualify(Manifest.Name)) sql"SELECT run_id FROM $frTableName MINUS SELECT base FROM $frManifest" - case Statement.FoldersCopy(source, loadAuthMethod, initQueryResult: InitQueryResult) => + case Statement.FoldersCopy(s, loadAuthMethod, initQueryResult: InitQueryResult) => + val updatedSource = replaceScheme(s) val frTableName = Fragment.const(qualify(AlertingTempTableName)) val frPath = loadAuthMethod match { case LoadAuthMethod.NoCreds => @@ -165,25 +166,26 @@ object Snowflake { val stage = tgt.folderMonitoringStage.getOrElse( throw new IllegalStateException("Folder Monitoring is launched without monitoring stage being provided") ) - val afterStage = findPathAfterStage(stage, initQueryResult.folderMonitoringStagePath, source) + val afterStage = findPathAfterStage(stage, initQueryResult.folderMonitoringStagePath, updatedSource) Fragment.const0(s"@${qualify(stage.name)}/$afterStage") case _: LoadAuthMethod.TempCreds => - Fragment.const0(source) + Fragment.const0(updatedSource) } val frCredentials = loadAuthMethodFragment(loadAuthMethod) sql"""|COPY INTO $frTableName - |FROM $frPath $frCredentials + |FROM '$frPath' $frCredentials |FILE_FORMAT = (TYPE = CSV)""".stripMargin case Statement.FoldersCopy(_, _, _) => throw new IllegalStateException("Init query result has wrong format in FoldersCopy") - case Statement.EventsCopy(path, _, columns, _, typesInfo, _, initQueryResult: InitQueryResult) => + case Statement.EventsCopy(p, _, columns, _, typesInfo, _, initQueryResult: InitQueryResult) => + val updatedPath = replaceScheme(p) // This is validated on config decoding stage val stage = tgt.transformedStage.getOrElse( throw new IllegalStateException("Transformed stage is tried to be used without being provided") ) - val afterStage = findPathAfterStage(stage, initQueryResult.transformedStagePath, path) + val afterStage = findPathAfterStage(stage, initQueryResult.transformedStagePath, updatedPath) val frPath = Fragment.const0(s"@${qualify(stage.name)}/$afterStage/output=good/") val frCopy = Fragment.const0(s"${qualify(EventsTable.MainName)}(${columnsForCopy(columns)})") val frSelectColumns = Fragment.const0(columnsForSelect(columns)) @@ -212,8 +214,9 @@ object Snowflake { sql"DROP TABLE IF EXISTS $frTableName" case s: Statement.EventsCopyToTempTable => + val updatedPath = replaceScheme(s.path) val frCopy = Fragment.const0(s"${qualify(s.table)}($TempTableColumn)") - val frPath = Fragment.const0(s.path.append("output=good")) + val frPath = Fragment.const0(updatedPath.append("output=good")) val frCredentials = loadAuthMethodFragment(s.tempCreds) val frOnError = buildErrorFragment(s.typesInfo) val frFileFormat = buildFileFormatFragment(s.typesInfo) @@ -320,6 +323,8 @@ object Snowflake { else columnNames } + + private def replaceScheme(path: Folder): Folder = Folder.coerce(path.replace("https://", "azure://")) } Right(result) @@ -358,10 +363,14 @@ object Snowflake { loadAuthMethod match { case LoadAuthMethod.NoCreds => Fragment.empty - case LoadAuthMethod.TempCreds(awsAccessKey, awsSecretKey, awsSessionToken, _) => + case LoadAuthMethod.TempCreds.AWS(awsAccessKey, awsSecretKey, awsSessionToken, _) => Fragment.const0( s"CREDENTIALS = (AWS_KEY_ID = '${awsAccessKey}' AWS_SECRET_KEY = '${awsSecretKey}' AWS_TOKEN = '${awsSessionToken}')" ) + case LoadAuthMethod.TempCreds.Azure(sasToken, _) => + Fragment.const0( + s"CREDENTIALS = (AZURE_SAS_TOKEN = '$sasToken')" + ) } private def columnsToCopyFromDiscoveredData(discovery: DataDiscovery): ColumnsToCopy = { diff --git a/modules/transformer-kafka/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/kafka/Main.scala b/modules/transformer-kafka/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/kafka/Main.scala index 148b59ceb..175965c9f 100644 --- a/modules/transformer-kafka/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/kafka/Main.scala +++ b/modules/transformer-kafka/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/kafka/Main.scala @@ -50,10 +50,8 @@ object Main extends IOApp { private def parquetOps: ParquetOps = new ParquetOps { - override def transformPath(p: String): String = { - val parts = AzureBlobStorage.parsePath(p) - s"abfss://${parts.containerName}@${parts.storageAccountName}.dfs.${parts.endpointSuffix}" - } + override def transformPath(p: String): String = + AzureBlobStorage.PathParts.parse(p).toParquetPath override def hadoopConf: Configuration = { val hadoopConf = new Configuration() From 8a6a995f7a8d23d5289b7a3582ba90f4b8b3fa7e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Piotr=20Poniedzia=C5=82ek?= Date: Fri, 9 Jun 2023 13:14:27 +0200 Subject: [PATCH 07/14] Add transformer-kafka to CI --- .github/workflows/ci.yml | 5 ++++- build.sbt | 10 ++++++++++ 2 files changed, 14 insertions(+), 1 deletion(-) diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index 248eda4e1..478322283 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -44,6 +44,7 @@ jobs: - databricksLoader - transformerKinesis - transformerPubsub + - transformerKafka steps: - name: Checkout Github uses: actions/checkout@v2 @@ -164,7 +165,8 @@ jobs: 'project databricksLoader; assembly' \ 'project transformerBatch; assembly' \ 'project transformerKinesis; assembly' \ - 'project transformerPubsub; assembly' + 'project transformerPubsub; assembly' \ + 'project transformerKafka; assembly' - name: Get current version id: ver run: echo "::set-output name=project_version::${GITHUB_REF#refs/tags/}" @@ -182,5 +184,6 @@ jobs: modules/transformer-batch/target/scala-2.12/snowplow-transformer-batch-${{ steps.ver.outputs.project_version }}.jar modules/transformer-kinesis/target/scala-2.12/snowplow-transformer-kinesis-${{ steps.ver.outputs.project_version }}.jar modules/transformer-pubsub/target/scala-2.12/snowplow-transformer-pubsub-${{ steps.ver.outputs.project_version }}.jar + modules/transformer-kafka/target/scala-2.12/snowplow-transformer-kafka-${{ steps.ver.outputs.project_version }}.jar env: GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} diff --git a/build.sbt b/build.sbt index ac8d74722..9b1b8bf2d 100755 --- a/build.sbt +++ b/build.sbt @@ -192,3 +192,13 @@ lazy val transformerKafka = project .dependsOn(commonTransformerStream % "compile->compile;test->test;runtime->runtime", azure % "compile->compile;test->test;runtime->runtime") .enablePlugins(JavaAppPackaging, SnowplowDockerPlugin, BuildInfoPlugin) + +lazy val transformerKafkaDistroless = project + .in(file("modules/distroless/transformer-kafka")) + .settings(sourceDirectory := (transformerKafka / sourceDirectory).value) + .settings(BuildSettings.transformerKafkaBuildSettings) + .settings(addCompilerPlugin(Dependencies.betterMonadicFor)) + .settings(libraryDependencies ++= Dependencies.transformerKafkaDependencies) + .settings(excludeDependencies ++= Dependencies.commonStreamTransformerExclusions) + .dependsOn(commonTransformerStream % "compile->compile;test->test;runtime->runtime", azure % "compile->compile;test->test") + .enablePlugins(JavaAppPackaging, SnowplowDistrolessDockerPlugin, BuildInfoPlugin) \ No newline at end of file From 92807bb7d4a834c256f210a7b049cb2922ccb2e6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Piotr=20Poniedzia=C5=82ek?= Date: Wed, 28 Jun 2023 19:43:55 +0200 Subject: [PATCH 08/14] Fix azure token provider scope --- .../snowplow/rdbloader/azure/AzureTokenProvider.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/modules/azure/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/azure/AzureTokenProvider.scala b/modules/azure/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/azure/AzureTokenProvider.scala index a3ab57eef..f6c5db15a 100644 --- a/modules/azure/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/azure/AzureTokenProvider.scala +++ b/modules/azure/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/azure/AzureTokenProvider.scala @@ -37,8 +37,7 @@ class AzureTokenProvider extends CustomTokenProviderAdaptee { override def getAccessToken: String = { val creds = new DefaultAzureCredentialBuilder().build() - val request = new TokenRequestContext() - request.addScopes("https://" + accountName) + val request = new TokenRequestContext().addScopes(s"https://$accountName/.default") val token = creds.getToken(request).block() this.expiryTime = new Date(token.getExpiresAt.toInstant.toEpochMilli) token.getToken From 7b9cb4475b2b71b68a3a947a03d7e36010bbdc3b Mon Sep 17 00:00:00 2001 From: spenes Date: Thu, 22 Jun 2023 16:12:01 +0300 Subject: [PATCH 09/14] Loader: add postProcess to Kafka consumer --- .../snowplow/rdbloader/azure/KafkaConsumer.scala | 13 +++++++++---- .../snowplow/rdbloader/dsl/Environment.scala | 4 +++- 2 files changed, 12 insertions(+), 5 deletions(-) diff --git a/modules/azure/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/azure/KafkaConsumer.scala b/modules/azure/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/azure/KafkaConsumer.scala index 44741bc7a..f87e5d4bb 100644 --- a/modules/azure/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/azure/KafkaConsumer.scala +++ b/modules/azure/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/azure/KafkaConsumer.scala @@ -32,7 +32,8 @@ object KafkaConsumer { def consumer[F[_]: Async]( bootstrapServers: String, topicName: String, - consumerConf: Map[String, String] + consumerConf: Map[String, String], + postProcess: Option[Queue.Consumer.PostProcess[F]] = None ): Resource[F, Queue.Consumer[F]] = { val consumerSettings = ConsumerSettings[F, String, Array[Byte]] @@ -49,9 +50,13 @@ object KafkaConsumer { .evalMap { consumer => consumer.subscribeTo(topicName) *> Async[F].pure { new Queue.Consumer[F] { - override def read: fs2.Stream[F, Consumer.Message[F]] = - consumer.records - .map(KafkaMessage(_)) + override def read: fs2.Stream[F, Consumer.Message[F]] = { + val stream = consumer.records.map(KafkaMessage(_)) + postProcess match { + case None => stream + case Some(p) => stream.flatMap(p.process(_)) + } + } } } } diff --git a/modules/loader/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/dsl/Environment.scala b/modules/loader/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/dsl/Environment.scala index c3b68e344..d83268902 100644 --- a/modules/loader/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/dsl/Environment.scala +++ b/modules/loader/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/dsl/Environment.scala @@ -188,10 +188,12 @@ object Environment { .create[F](c.blobStorageEndpoint.toString, config.storage.eventsLoadAuthMethod, config.storage.foldersLoadAuthMethod) jsonPathDiscovery = JsonPathDiscovery.noop[F] implicit0(blobStorage: BlobStorage[F]) <- AzureBlobStorage.createDefault[F](c.blobStorageEndpoint) + postProcess = Queue.Consumer.postProcess[F] queueConsumer <- KafkaConsumer.consumer[F]( bootstrapServers = c.messageQueue.bootstrapServers, topicName = c.messageQueue.topicName, - consumerConf = c.messageQueue.consumerConf + consumerConf = c.messageQueue.consumerConf, + postProcess = Some(postProcess) ) secretStore = SecretStore.noop[F] // TODO implement secret store for Azure } yield CloudServices(blobStorage, queueConsumer, loadAuthService, jsonPathDiscovery, secretStore) From c336d8c5bb4bc27cd0774006e82514e2b6190f5c Mon Sep 17 00:00:00 2001 From: spenes Date: Thu, 22 Jun 2023 17:46:23 +0300 Subject: [PATCH 10/14] Loader: integrate Azure Key Vault --- .../azure/databricks.config.reference.hocon | 10 ++-- .../azure/snowflake.config.reference.hocon | 8 +++- .../rdbloader/azure/AzureKeyVault.scala | 48 +++++++++++++++++++ .../snowplow/rdbloader/config/Config.scala | 17 ++++++- .../snowplow/rdbloader/dsl/Environment.scala | 4 +- .../snowplow/rdbloader/ConfigSpec.scala | 3 +- project/Dependencies.scala | 6 ++- 7 files changed, 86 insertions(+), 10 deletions(-) create mode 100644 modules/azure/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/azure/AzureKeyVault.scala diff --git a/config/loader/azure/databricks.config.reference.hocon b/config/loader/azure/databricks.config.reference.hocon index be6f21a0f..82904d959 100644 --- a/config/loader/azure/databricks.config.reference.hocon +++ b/config/loader/azure/databricks.config.reference.hocon @@ -14,9 +14,13 @@ "storage" : { # Hostname of Databricks cluster "host": "abc.cloud.databricks.com", - # DB password - # TODO handle secret store params instead of plaintext - "password": "secret" + # DB password + "password": { + # A password can be placed in Azure Key Vault or be a plain text + "secretStore": { + "parameterName": "snowplow.databricks.password" + } + }, # Optional. Override the Databricks default catalog, e.g. with a Unity catalog name. "catalog": "hive_metastore", # DB schema diff --git a/config/loader/azure/snowflake.config.reference.hocon b/config/loader/azure/snowflake.config.reference.hocon index f1f618e52..63696a436 100644 --- a/config/loader/azure/snowflake.config.reference.hocon +++ b/config/loader/azure/snowflake.config.reference.hocon @@ -20,8 +20,12 @@ # DB user with permissions to load data "username": "admin", # DB password - # TODO handle secret store params instead of plaintext - "password": "secret" + "password": { + # A password can be placed in Azure Key Vault or be a plain text + "secretStore": { + "parameterName": "snowplow.snowflake.password" + } + }, # Snowflake account "account": "acme", # A warehouse to use for loading diff --git a/modules/azure/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/azure/AzureKeyVault.scala b/modules/azure/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/azure/AzureKeyVault.scala new file mode 100644 index 000000000..1ca9e6254 --- /dev/null +++ b/modules/azure/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/azure/AzureKeyVault.scala @@ -0,0 +1,48 @@ +/* + * Copyright (c) 2012-2023 Snowplow Analytics Ltd. All rights reserved. + * + * This program is licensed to you under the Apache License Version 2.0, + * and you may not use this file except in compliance with the Apache License Version 2.0. + * You may obtain a copy of the Apache License Version 2.0 at + * http://www.apache.org/licenses/LICENSE-2.0. + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the Apache License Version 2.0 is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the Apache License Version 2.0 for the specific language governing permissions and + * limitations there under. + */ +package com.snowplowanalytics.snowplow.rdbloader.azure + +import cats.effect.{Resource, Sync} + +import com.azure.identity.DefaultAzureCredentialBuilder +import com.azure.security.keyvault.secrets.SecretClientBuilder + +import com.snowplowanalytics.snowplow.rdbloader.common.cloud.SecretStore + +object AzureKeyVault { + + def create[F[_]: Sync](keyVaultName: Option[String]): Resource[F, SecretStore[F]] = + keyVaultName match { + case None => + Resource.pure(new SecretStore[F] { + override def getValue(key: String): F[String] = + Sync[F].raiseError(new IllegalStateException("Azure vault name isn't given")) + }) + case Some(vaultName) => + for { + client <- Resource.eval(Sync[F].delay { + new SecretClientBuilder() + .vaultUrl("https://" + vaultName + ".vault.azure.net") + .credential(new DefaultAzureCredentialBuilder().build()) + .buildClient() + }) + secretStore <- Resource.pure(new SecretStore[F] { + override def getValue(key: String): F[String] = + Sync[F].delay(client.getSecret(key).getValue) + }) + } yield secretStore + } + +} diff --git a/modules/loader/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/config/Config.scala b/modules/loader/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/config/Config.scala index 09196cf8f..e7365ad34 100644 --- a/modules/loader/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/config/Config.scala +++ b/modules/loader/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/config/Config.scala @@ -167,7 +167,11 @@ object Config { } } - final case class Azure(blobStorageEndpoint: URI, messageQueue: Azure.Kafka) extends Cloud + final case class Azure( + blobStorageEndpoint: URI, + messageQueue: Azure.Kafka, + azureVaultName: Option[String] + ) extends Cloud object Azure { final case class Kafka( @@ -332,10 +336,21 @@ object Config { List( authMethodValidation(config.storage.eventsLoadAuthMethod, config.cloud), authMethodValidation(config.storage.foldersLoadAuthMethod, config.cloud), + azureVaultCheck(config), targetSnowflakeValidation(config), targetRedshiftValidation(config) ).flatten + private def azureVaultCheck(config: Config[StorageTarget]): List[String] = + config.cloud match { + case c: Config.Cloud.Azure if c.azureVaultName.isEmpty => + (config.storage.password, config.storage.sshTunnel.flatMap(_.bastion.key)) match { + case (_: StorageTarget.PasswordConfig.EncryptedKey, _) | (_, Some(_)) => List("Azure vault name is needed") + case _ => Nil + } + case _ => Nil + } + private def authMethodValidation(loadAuthMethod: StorageTarget.LoadAuthMethod, cloud: Config.Cloud): List[String] = cloud match { case _: Config.Cloud.GCP => diff --git a/modules/loader/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/dsl/Environment.scala b/modules/loader/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/dsl/Environment.scala index d83268902..3be0773dd 100644 --- a/modules/loader/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/dsl/Environment.scala +++ b/modules/loader/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/dsl/Environment.scala @@ -25,7 +25,7 @@ import com.snowplowanalytics.snowplow.rdbloader.common.Sentry import com.snowplowanalytics.snowplow.rdbloader.common.telemetry.Telemetry import com.snowplowanalytics.snowplow.rdbloader.common.cloud.{BlobStorage, Queue, SecretStore} import com.snowplowanalytics.snowplow.rdbloader.aws.{EC2ParameterStore, S3, SQS} -import com.snowplowanalytics.snowplow.rdbloader.azure.{AzureBlobStorage, KafkaConsumer} +import com.snowplowanalytics.snowplow.rdbloader.azure.{AzureBlobStorage, AzureKeyVault, KafkaConsumer} import com.snowplowanalytics.snowplow.rdbloader.gcp.{GCS, Pubsub, SecretManager} import com.snowplowanalytics.snowplow.rdbloader.cloud.JsonPathDiscovery import com.snowplowanalytics.snowplow.rdbloader.cloud.authservice.{AWSAuthService, AzureAuthService, LoadAuthService} @@ -195,7 +195,7 @@ object Environment { consumerConf = c.messageQueue.consumerConf, postProcess = Some(postProcess) ) - secretStore = SecretStore.noop[F] // TODO implement secret store for Azure + secretStore <- AzureKeyVault.create(c.azureVaultName) } yield CloudServices(blobStorage, queueConsumer, loadAuthService, jsonPathDiscovery, secretStore) } diff --git a/modules/loader/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/ConfigSpec.scala b/modules/loader/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/ConfigSpec.scala index 8e3a5495a..6154ae336 100644 --- a/modules/loader/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/ConfigSpec.scala +++ b/modules/loader/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/ConfigSpec.scala @@ -111,7 +111,8 @@ class ConfigSpec extends Specification { ) val azureConfig = Config.Cloud.Azure( URI.create("https://test.blob.core.windows.net/test-container/"), - Config.Cloud.Azure.Kafka("test-topic", "127.0.0.1:8080", Map.empty) + Config.Cloud.Azure.Kafka("test-topic", "127.0.0.1:8080", Map.empty), + None ) val config = exampleConfig diff --git a/project/Dependencies.scala b/project/Dependencies.scala index fb804e345..dc2ad5f0d 100644 --- a/project/Dependencies.scala +++ b/project/Dependencies.scala @@ -35,6 +35,8 @@ object Dependencies { val fs2Kafka = "3.0.0" val secretManager = "2.7.0" val gcpStorage = "2.16.0" + val azureIdentity = "1.9.0" + val azureKeyVault = "4.6.2" val doobie = "1.0.0-RC2" val monocle = "2.0.3" val catsRetry = "3.1.0" @@ -107,7 +109,8 @@ object Dependencies { val fs2BlobstoreS3 = "com.github.fs2-blobstore" %% "s3" % V.fs2Blobstore val fs2BlobstoreGCS = "com.github.fs2-blobstore" %% "gcs" % V.fs2Blobstore val fs2BlobstoreAzure = "com.github.fs2-blobstore" %% "azure" % V.fs2Blobstore - val azureIdentity = "com.azure" % "azure-identity" % "1.9.0" + val azureIdentity = "com.azure" % "azure-identity" % V.azureIdentity + val azureKeyVault = "com.azure" % "azure-security-keyvault-secrets" % V.azureKeyVault val fs2Cron = "eu.timepit" %% "fs2-cron-cron4s" % V.fs2Cron val fs2PubSub = "com.permutive" %% "fs2-google-pubsub-grpc" % V.fs2PubSub val secretManager = "com.google.cloud" % "google-cloud-secretmanager" % V.secretManager @@ -215,6 +218,7 @@ object Dependencies { val azureDependencies = Seq( fs2BlobstoreAzure, azureIdentity, + azureKeyVault, fs2Kafka, hadoopCommon, hadoopAzure From 329bf31251fa5b7f52cf6ba26cb81d0b869b4969 Mon Sep 17 00:00:00 2001 From: spenes Date: Fri, 23 Jun 2023 23:11:12 +0300 Subject: [PATCH 11/14] Path related fixes on Azure Blob Storage --- .../rdbloader/azure/AzureBlobStorage.scala | 16 +- .../azure/AzureBlobStorageSpec.scala | 148 ++++++++++++++++++ 2 files changed, 155 insertions(+), 9 deletions(-) create mode 100644 modules/azure/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/azure/AzureBlobStorageSpec.scala diff --git a/modules/azure/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/azure/AzureBlobStorage.scala b/modules/azure/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/azure/AzureBlobStorage.scala index 5d284e503..b0bf711a6 100644 --- a/modules/azure/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/azure/AzureBlobStorage.scala +++ b/modules/azure/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/azure/AzureBlobStorage.scala @@ -70,25 +70,25 @@ class AzureBlobStorage[F[_]: Async] private (store: AzureStore[F], path: AzureBl } // input path format like 'endpoint/container/blobPath', where 'endpoint' is 'scheme://host' - private def createStorageUrlFrom(input: String): ValidatedNec[AuthorityParseError, Url[String]] = + protected[azure] def createStorageUrlFrom(input: String): ValidatedNec[AuthorityParseError, Url[String]] = Authority .parse(path.containerName) .map(authority => Url(path.scheme, authority, Path(path.extractRelative(input)))) - private def createBlobObject(url: Url[AzureBlob]) = { - val key = path.fullPath.withKey(url.path.relative.show) + protected[azure] def createBlobObject(url: Url[AzureBlob]): BlobStorage.BlobObject = { + val key = path.root.append(path.containerName).withKey(url.path.relative.show) BlobStorage.BlobObject(key, url.path.representation.size.getOrElse(0L)) } } object AzureBlobStorage { - def createDefault[F[_]: Async](path: URI): Resource[F, BlobStorage[F]] = { + def createDefault[F[_]: Async](path: URI): Resource[F, AzureBlobStorage[F]] = { val builder = new BlobServiceClientBuilder().credential(new DefaultAzureCredentialBuilder().build) create(path, builder) } - def create[F[_]: Async](path: URI, builder: BlobServiceClientBuilder): Resource[F, BlobStorage[F]] = { + def create[F[_]: Async](path: URI, builder: BlobServiceClientBuilder): Resource[F, AzureBlobStorage[F]] = { val pathParts = PathParts.parse(path.toString) val client = builder.endpoint(pathParts.root).buildAsyncClient() createStore(client).map(new AzureBlobStorage(_, pathParts)) @@ -101,7 +101,6 @@ object AzureBlobStorage { .fold(errors => Resource.raiseError(errors.reduce(Throwables.collapsingSemigroup)), Resource.pure) final case class PathParts( - fullPath: Folder, containerName: String, storageAccountName: String, scheme: String, @@ -109,7 +108,7 @@ object AzureBlobStorage { relative: String ) { def extractRelative(p: String): String = - p.stripPrefix(fullPath) + p.stripPrefix(root.append(containerName)) def root: Folder = Folder.coerce(s"$scheme://$storageAccountName.blob.$endpointSuffix") @@ -122,12 +121,11 @@ object AzureBlobStorage { def parse(path: String): PathParts = { val parts = BlobUrlParts.parse(path) PathParts( - fullPath = Folder.coerce(path), containerName = parts.getBlobContainerName, storageAccountName = parts.getAccountName, scheme = parts.getScheme, endpointSuffix = parts.getHost.stripPrefix(s"${parts.getAccountName}.blob."), - relative = parts.getBlobName + relative = Option(parts.getBlobName).getOrElse("") ) } } diff --git a/modules/azure/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/azure/AzureBlobStorageSpec.scala b/modules/azure/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/azure/AzureBlobStorageSpec.scala new file mode 100644 index 000000000..bc1253363 --- /dev/null +++ b/modules/azure/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/azure/AzureBlobStorageSpec.scala @@ -0,0 +1,148 @@ +/* + * Copyright (c) 2020-2023 Snowplow Analytics Ltd. All rights reserved. + * + * This program is licensed to you under the Apache License Version 2.0, * and you may not use this file except in compliance with the Apache License Version 2.0. + * You may obtain a copy of the Apache License Version 2.0 at http://www.apache.org/licenses/LICENSE-2.0. + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the Apache License Version 2.0 is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the Apache License Version 2.0 for the specific language governing permissions and limitations there under. + */ + +package com.snowplowanalytics.snowplow.rdbloader.azure + +import java.net.URI + +import cats.data.Validated.Valid +import cats.effect.IO +import cats.effect.unsafe.implicits.global + +import blobstore.azure.AzureBlob +import blobstore.url.{Authority, Path, Url} + +import com.snowplowanalytics.snowplow.rdbloader.azure.AzureBlobStorage.PathParts +import com.snowplowanalytics.snowplow.rdbloader.common.cloud.BlobStorage +import com.snowplowanalytics.snowplow.rdbloader.common.cloud.BlobStorage.Key + +import org.specs2.mutable.Specification + +class AzureBlobStorageSpec extends Specification { + import AzureBlobStorageSpec._ + + "PathParts" should { + "parse root path" in { + PathParts.parse(testContainerPath) must beEqualTo( + PathParts( + containerName = "test-container", + storageAccountName = "accountName", + scheme = "https", + endpointSuffix = "core.windows.net", + relative = "" + ) + ) + } + + "parse non-root path" in { + PathParts.parse(s"$testContainerPath/path1/path2/") must beEqualTo( + PathParts( + containerName = "test-container", + storageAccountName = "accountName", + scheme = "https", + endpointSuffix = "core.windows.net", + relative = "path1/path2/" + ) + ) + } + + "extract relative path" in { + PathParts.parse(testContainerPath).extractRelative(s"$testContainerPath/path1/path2") must beEqualTo( + "path1/path2" + ) + } + + "extract root" in { + PathParts.parse(s"$testContainerPath/path1/path2").root must beEqualTo( + "https://accountName.blob.core.windows.net/" + ) + } + + "convert to parquet path correctly" in { + PathParts.parse(s"$testContainerPath/path1/path2").toParquetPath must beEqualTo( + "abfss://test-container@accountName.dfs.core.windows.net/path1/path2/" + ) + } + } + + "createStorageUrlFrom" should { + "return expected URL" in { + AzureBlobStorage.createDefault[IO](URI.create(s"$testContainerPath/path1/path2")) + .use { blobStorage => + IO.delay { + blobStorage.createStorageUrlFrom(s"$testContainerPath/path1/path2/path3/path4") + } + } + .unsafeRunSync() must beEqualTo( + Valid(Url("https", Authority.unsafe("test-container"), Path.plain("path1/path2/path3/path4"))) + ) + + AzureBlobStorage.createDefault[IO](URI.create(s"$testContainerPath")) + .use { blobStorage => + IO.delay { + blobStorage.createStorageUrlFrom(s"$testContainerPath/path1/path2/path3/path4") + } + } + .unsafeRunSync() must beEqualTo( + Valid(Url("https", Authority.unsafe("test-container"), Path.plain("path1/path2/path3/path4"))) + ) + } + } + + "createBlobObject" should { + "create blob object from given url correctly" in { + AzureBlobStorage.createDefault[IO](URI.create(s"$testContainerPath/path1/path2")) + .use { blobStorage => + IO.delay { + blobStorage.createBlobObject( + Url( + "https", + Authority.unsafe("test-container"), + Path.of( + "path1/path2/path3/path4", + AzureBlob("test-container", "test-blob", None, Map.empty) + ) + ) + ) + } + } + .unsafeRunSync() must beEqualTo( + BlobStorage.BlobObject(Key.coerce(s"$testContainerPath/path1/path2/path3/path4"), 0L) + ) + + AzureBlobStorage.createDefault[IO](URI.create(s"$testContainerPath")) + .use { blobStorage => + IO.delay { + blobStorage.createBlobObject( + Url( + "https", + Authority.unsafe("test-container"), + Path.of( + "path1/path2/path3/path4", + AzureBlob("test-container", "test-blob", None, Map.empty) + ) + ) + ) + } + } + .unsafeRunSync() must beEqualTo( + BlobStorage.BlobObject(Key.coerce(s"$testContainerPath/path1/path2/path3/path4"), 0L) + ) + } + } + +} + +object AzureBlobStorageSpec { + + val testContainerPath = "https://accountName.blob.core.windows.net/test-container" +} From 0ddbe808b488ee9f62f5f756a21190c827d03393 Mon Sep 17 00:00:00 2001 From: spenes Date: Tue, 4 Jul 2023 01:19:16 +0300 Subject: [PATCH 12/14] Loader: add tests for Azure configs --- .../azure/databricks.config.minimal.hocon | 2 +- .../azure/databricks.config.reference.hocon | 6 +- .../azure/snowflake.config.minimal.hocon | 2 +- .../azure/snowflake.config.reference.hocon | 6 +- .../azure/AzureBlobStorageSpec.scala | 20 +++-- .../loader/databricks/ConfigSpec.scala | 69 +++++++++++++++ .../loader/snowflake/ConfigSpec.scala | 83 +++++++++++++++++++ 7 files changed, 176 insertions(+), 12 deletions(-) diff --git a/config/loader/azure/databricks.config.minimal.hocon b/config/loader/azure/databricks.config.minimal.hocon index dcb78cd0f..a1893a850 100644 --- a/config/loader/azure/databricks.config.minimal.hocon +++ b/config/loader/azure/databricks.config.minimal.hocon @@ -1,5 +1,5 @@ { - "blobStorageEndpoint": "https://accountName.blob.core.windows.net" + "blobStorageEndpoint": "https://accountName.blob.core.windows.net/container-name" "messageQueue": { "type": "kafka" "bootstrapServers": "localhost:9092" diff --git a/config/loader/azure/databricks.config.reference.hocon b/config/loader/azure/databricks.config.reference.hocon index 82904d959..2a9ab7194 100644 --- a/config/loader/azure/databricks.config.reference.hocon +++ b/config/loader/azure/databricks.config.reference.hocon @@ -1,7 +1,11 @@ { # Azure Blob Storage endpoint, should contain container with transformer's output - "blobStorageEndpoint": "https://accountName.blob.core.windows.net" + "blobStorageEndpoint": "https://accountName.blob.core.windows.net/container-name" + + # Name of the Azure Key Vault where application secrets are stored. + # Required if secret store is used in storage.password field. + "azureVaultName": "azure-vault" # Kafka topic used by Transformer and Loader to communicate "messageQueue": { diff --git a/config/loader/azure/snowflake.config.minimal.hocon b/config/loader/azure/snowflake.config.minimal.hocon index 966c9b875..2adb6b0c3 100644 --- a/config/loader/azure/snowflake.config.minimal.hocon +++ b/config/loader/azure/snowflake.config.minimal.hocon @@ -1,5 +1,5 @@ { - "blobStorageEndpoint": "https://accountName.blob.core.windows.net" + "blobStorageEndpoint": "https://accountName.blob.core.windows.net/container-name" "messageQueue": { "type": "kafka" "bootstrapServers": "localhost:9092" diff --git a/config/loader/azure/snowflake.config.reference.hocon b/config/loader/azure/snowflake.config.reference.hocon index 63696a436..c2082a53a 100644 --- a/config/loader/azure/snowflake.config.reference.hocon +++ b/config/loader/azure/snowflake.config.reference.hocon @@ -1,7 +1,11 @@ { # Azure Blob Storage endpoint, should contain container with transformer's output - "blobStorageEndpoint": "https://accountName.blob.core.windows.net" + "blobStorageEndpoint": "https://accountName.blob.core.windows.net/container-name" + + # Name of the Azure Key Vault where application secrets are stored. + # Required if secret store is used in storage.password field. + "azureVaultName": "azure-vault" # Kafka topic used by Transformer and Loader to communicate "messageQueue": { diff --git a/modules/azure/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/azure/AzureBlobStorageSpec.scala b/modules/azure/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/azure/AzureBlobStorageSpec.scala index bc1253363..c94e21df0 100644 --- a/modules/azure/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/azure/AzureBlobStorageSpec.scala +++ b/modules/azure/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/azure/AzureBlobStorageSpec.scala @@ -76,7 +76,8 @@ class AzureBlobStorageSpec extends Specification { "createStorageUrlFrom" should { "return expected URL" in { - AzureBlobStorage.createDefault[IO](URI.create(s"$testContainerPath/path1/path2")) + AzureBlobStorage + .createDefault[IO](URI.create(s"$testContainerPath/path1/path2")) .use { blobStorage => IO.delay { blobStorage.createStorageUrlFrom(s"$testContainerPath/path1/path2/path3/path4") @@ -86,7 +87,8 @@ class AzureBlobStorageSpec extends Specification { Valid(Url("https", Authority.unsafe("test-container"), Path.plain("path1/path2/path3/path4"))) ) - AzureBlobStorage.createDefault[IO](URI.create(s"$testContainerPath")) + AzureBlobStorage + .createDefault[IO](URI.create(s"$testContainerPath")) .use { blobStorage => IO.delay { blobStorage.createStorageUrlFrom(s"$testContainerPath/path1/path2/path3/path4") @@ -100,7 +102,8 @@ class AzureBlobStorageSpec extends Specification { "createBlobObject" should { "create blob object from given url correctly" in { - AzureBlobStorage.createDefault[IO](URI.create(s"$testContainerPath/path1/path2")) + AzureBlobStorage + .createDefault[IO](URI.create(s"$testContainerPath/path1/path2")) .use { blobStorage => IO.delay { blobStorage.createBlobObject( @@ -116,10 +119,11 @@ class AzureBlobStorageSpec extends Specification { } } .unsafeRunSync() must beEqualTo( - BlobStorage.BlobObject(Key.coerce(s"$testContainerPath/path1/path2/path3/path4"), 0L) - ) + BlobStorage.BlobObject(Key.coerce(s"$testContainerPath/path1/path2/path3/path4"), 0L) + ) - AzureBlobStorage.createDefault[IO](URI.create(s"$testContainerPath")) + AzureBlobStorage + .createDefault[IO](URI.create(s"$testContainerPath")) .use { blobStorage => IO.delay { blobStorage.createBlobObject( @@ -135,8 +139,8 @@ class AzureBlobStorageSpec extends Specification { } } .unsafeRunSync() must beEqualTo( - BlobStorage.BlobObject(Key.coerce(s"$testContainerPath/path1/path2/path3/path4"), 0L) - ) + BlobStorage.BlobObject(Key.coerce(s"$testContainerPath/path1/path2/path3/path4"), 0L) + ) } } diff --git a/modules/databricks-loader/src/test/scala/com/snowplowanalytics/snowplow/loader/databricks/ConfigSpec.scala b/modules/databricks-loader/src/test/scala/com/snowplowanalytics/snowplow/loader/databricks/ConfigSpec.scala index 0848cf95e..0cc40da3c 100644 --- a/modules/databricks-loader/src/test/scala/com/snowplowanalytics/snowplow/loader/databricks/ConfigSpec.scala +++ b/modules/databricks-loader/src/test/scala/com/snowplowanalytics/snowplow/loader/databricks/ConfigSpec.scala @@ -14,6 +14,8 @@ package com.snowplowanalytics.snowplow.loader.databricks import scala.concurrent.duration._ +import java.net.URI + import cats.effect.IO import com.snowplowanalytics.snowplow.rdbloader.common.config.Region @@ -77,6 +79,34 @@ class ConfigSpec extends Specification { result must beRight(expected) } + "be able to parse extended Azure Databricks Loader config" in { + val result = getConfigFromResource("/loader/azure/databricks.config.reference.hocon", Config.parseAppConfig[IO]) + val monitoring = exampleMonitoring.copy( + snowplow = exampleMonitoring.snowplow.map(_.copy(appId = "databricks-loader")), + folders = exampleMonitoring.folders.map( + _.copy( + staging = BlobStorage.Folder.coerce("https://accountName.blob.core.windows.net/staging/"), + transformerOutput = BlobStorage.Folder.coerce("https://accountName.blob.core.windows.net/transformed/") + ) + ) + ) + val expected = Config( + ConfigSpec.exampleStorage, + ConfigSpec.exampleAzureCloud, + None, + monitoring, + exampleRetryQueue, + exampleSchedules, + exampleTimeouts, + exampleRetries, + exampleReadyCheck, + exampleInitRetries, + exampleFeatureFlags, + exampleTelemetry + ) + result must beRight(expected) + } + "be able to parse minimal AWS Snowflake Loader config" in { val result = getConfigFromResource("/loader/aws/databricks.config.minimal.hocon", testParseConfig) val storage = ConfigSpec.exampleStorage.copy( @@ -129,6 +159,32 @@ class ConfigSpec extends Specification { ) result must beRight(expected) } + + "be able to parse minimal Azure Snowflake Loader config" in { + val result = getConfigFromResource("/loader/azure/databricks.config.minimal.hocon", testParseConfig) + val storage = ConfigSpec.exampleStorage.copy( + catalog = None, + password = StorageTarget.PasswordConfig.PlainText("Supersecret1") + ) + val retries = exampleRetries.copy(cumulativeBound = Some(20.minutes)) + val readyCheck = exampleReadyCheck.copy(strategy = Config.Strategy.Constant, backoff = 15.seconds) + val initRetries = exampleInitRetries.copy(attempts = None, cumulativeBound = Some(10.minutes)) + val expected = Config( + storage, + ConfigSpec.exampleAzureCloud.copy(azureVaultName = None), + None, + defaultMonitoring, + None, + defaultSchedules, + exampleTimeouts, + retries, + readyCheck, + initRetries, + exampleFeatureFlags, + defaultTelemetry + ) + result must beRight(expected) + } } } @@ -156,4 +212,17 @@ object ConfigSpec { bufferSize = 10 ) ) + val exampleAzureCloud = Config.Cloud.Azure( + blobStorageEndpoint = URI.create("https://accountName.blob.core.windows.net/container-name"), + Config.Cloud.Azure.Kafka( + topicName = "loaderTopic", + bootstrapServers = "localhost:9092", + consumerConf = List( + "enable.auto.commit" -> "false", + "auto.offset.reset" -> "latest", + "group.id" -> "loader" + ).toMap + ), + azureVaultName = Some("azure-vault") + ) } diff --git a/modules/snowflake-loader/src/test/scala/com/snowplowanalytics/snowplow/loader/snowflake/ConfigSpec.scala b/modules/snowflake-loader/src/test/scala/com/snowplowanalytics/snowplow/loader/snowflake/ConfigSpec.scala index 77eadde8d..e479266b9 100644 --- a/modules/snowflake-loader/src/test/scala/com/snowplowanalytics/snowplow/loader/snowflake/ConfigSpec.scala +++ b/modules/snowflake-loader/src/test/scala/com/snowplowanalytics/snowplow/loader/snowflake/ConfigSpec.scala @@ -14,6 +14,8 @@ package com.snowplowanalytics.snowplow.loader.snowflake import scala.concurrent.duration._ +import java.net.URI + import cats.effect.IO import cats.syntax.all._ import com.snowplowanalytics.snowplow.rdbloader.common.RegionSpec @@ -92,6 +94,52 @@ class ConfigSpec extends Specification { result must beRight(expected) } + "be able to parse extended Azure Snowflake Loader config" in { + val storage = exampleSnowflake + .copy(password = StorageTarget.PasswordConfig.EncryptedKey(StorageTarget.EncryptedConfig("snowplow.snowflake.password"))) + .copy(jdbcHost = Some("acme.eu-central-1.snowflake.com")) + .copy(folderMonitoringStage = Some(StorageTarget.Snowflake.Stage("snowplow_folders_stage", None))) + .copy(transformedStage = Some(StorageTarget.Snowflake.Stage("snowplow_stage", None))) + val result = getConfigFromResource("/loader/azure/snowflake.config.reference.hocon", Config.parseAppConfig[IO]) + val azureCloud = Config.Cloud.Azure( + blobStorageEndpoint = URI.create("https://accountName.blob.core.windows.net/container-name"), + Config.Cloud.Azure.Kafka( + topicName = "loaderTopic", + bootstrapServers = "localhost:9092", + consumerConf = List( + "enable.auto.commit" -> "false", + "auto.offset.reset" -> "latest", + "group.id" -> "loader" + ).toMap + ), + azureVaultName = Some("azure-vault") + ) + val monitoring = exampleMonitoring.copy( + snowplow = exampleMonitoring.snowplow.map(_.copy(appId = "snowflake-loader")), + folders = exampleMonitoring.folders.map( + _.copy( + staging = BlobStorage.Folder.coerce("https://accountName.blob.core.windows.net/staging/"), + transformerOutput = BlobStorage.Folder.coerce("https://accountName.blob.core.windows.net/transformed/") + ) + ) + ) + val expected = Config( + storage, + azureCloud, + None, + monitoring, + exampleRetryQueue, + exampleSchedules, + exampleTimeouts, + exampleRetries, + exampleReadyCheck, + exampleInitRetries, + exampleFeatureFlags, + exampleTelemetry + ) + result must beRight(expected) + } + "be able to parse minimal AWS Snowflake Loader config" in { val result = getConfigFromResource("/loader/aws/snowflake.config.minimal.hocon", testParseConfig) val expected = Config( @@ -144,6 +192,41 @@ class ConfigSpec extends Specification { result must beRight(expected) } + "be able to parse minimal Azure Snowflake Loader config" in { + val result = getConfigFromResource("/loader/azure/snowflake.config.minimal.hocon", testParseConfig) + val azureCloud = Config.Cloud.Azure( + blobStorageEndpoint = URI.create("https://accountName.blob.core.windows.net/container-name"), + Config.Cloud.Azure.Kafka( + topicName = "loaderTopic", + bootstrapServers = "localhost:9092", + consumerConf = List( + "enable.auto.commit" -> "false", + "auto.offset.reset" -> "latest", + "group.id" -> "loader" + ).toMap + ), + azureVaultName = None + ) + val expected = Config( + exampleSnowflake + .copy( + transformedStage = Some(StorageTarget.Snowflake.Stage("snowplow_stage", None)) + ), + azureCloud, + None, + defaultMonitoring, + None, + defaultSchedules, + exampleTimeouts, + exampleRetries.copy(cumulativeBound = Some(20.minutes)), + exampleReadyCheck.copy(strategy = Config.Strategy.Constant, backoff = 15.seconds), + exampleInitRetries.copy(attempts = None, cumulativeBound = Some(10.minutes)), + exampleFeatureFlags, + defaultTelemetry + ) + result must beRight(expected) + } + "be able to infer host" in { val exampleSnowflake = StorageTarget.Snowflake( snowflakeRegion = Some("us-west-2"), From e6ced35e3fdba031b06c094ec28a22fe07c6fefc Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Piotr=20Poniedzia=C5=82ek?= Date: Mon, 3 Jul 2023 10:39:57 +0200 Subject: [PATCH 13/14] transformer-kafka: add semi-automatic test scenerios using cloud resources --- .../snowplow/rdbloader/aws/S3.scala | 14 +- .../rdbloader/azure/AzureBlobStorage.scala | 12 +- .../stream/common/ParquetUtils.scala | 34 +- .../common/processing/TestApplication.scala | 3 + .../WiderowParquetProcessingSpec.scala | 31 +- .../rdbloader/common/cloud/BlobStorage.scala | 5 + .../snowplow/rdbloader/gcp/GCS.scala | 14 +- .../snowplow/rdbloader/test/PureAWS.scala | 3 + .../kafka/experimental/AppConfiguration.scala | 25 ++ .../kafka/experimental/AppDependencies.scala | 17 + .../AzureTransformerSpecification.scala | 95 ++++++ .../kafka/experimental/InputBatch.scala | 48 +++ .../experimental/OutputDataRowReader.scala | 40 +++ .../TransformerSpecification.scala | 279 ++++++++++++++++ .../scenarios/BadDetailsScenario.scala | 28 ++ .../kafka/experimental/scenarios/json.scala | 244 ++++++++++++++ .../experimental/scenarios/parquet.scala | 309 ++++++++++++++++++ project/Dependencies.scala | 7 +- 18 files changed, 1159 insertions(+), 49 deletions(-) create mode 100644 modules/transformer-kafka/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/kafka/experimental/AppConfiguration.scala create mode 100644 modules/transformer-kafka/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/kafka/experimental/AppDependencies.scala create mode 100644 modules/transformer-kafka/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/kafka/experimental/AzureTransformerSpecification.scala create mode 100644 modules/transformer-kafka/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/kafka/experimental/InputBatch.scala create mode 100644 modules/transformer-kafka/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/kafka/experimental/OutputDataRowReader.scala create mode 100644 modules/transformer-kafka/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/kafka/experimental/TransformerSpecification.scala create mode 100644 modules/transformer-kafka/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/kafka/experimental/scenarios/BadDetailsScenario.scala create mode 100644 modules/transformer-kafka/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/kafka/experimental/scenarios/json.scala create mode 100644 modules/transformer-kafka/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/kafka/experimental/scenarios/parquet.scala diff --git a/modules/aws/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/aws/S3.scala b/modules/aws/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/aws/S3.scala index b57a0c8e1..99fe6e2ac 100644 --- a/modules/aws/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/aws/S3.scala +++ b/modules/aws/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/aws/S3.scala @@ -50,22 +50,24 @@ object S3 { BlobStorage.BlobObject(key, url.path.representation.size.getOrElse(0L)) } - def get(path: Key): F[Either[Throwable, String]] = { + override def getBytes(path: Key): Stream[F, Byte] = { val (bucketName, keyPath) = BlobStorage.splitKey(path) Authority .parse(bucketName) .fold( - errors => Async[F].delay(new MultipleUrlValidationException(errors).asLeft[String]), + errors => Stream.raiseError[F](new MultipleUrlValidationException(errors)), authority => client .get(Url("s3", authority, Path(keyPath)), 1024) - .compile - .to(Array) - .map(array => new String(array, UTF_8)) - .attempt ) } + def get(path: Key): F[Either[Throwable, String]] = + getBytes(path).compile + .to(Array) + .map(array => new String(array, UTF_8)) + .attempt + def list(folder: BlobStorage.Folder, recursive: Boolean): Stream[F, BlobStorage.BlobObject] = { val (bucketName, folderPath) = BlobStorage.splitPath(folder) Authority diff --git a/modules/azure/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/azure/AzureBlobStorage.scala b/modules/azure/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/azure/AzureBlobStorage.scala index b0bf711a6..0f1bd8f27 100644 --- a/modules/azure/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/azure/AzureBlobStorage.scala +++ b/modules/azure/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/azure/AzureBlobStorage.scala @@ -49,16 +49,18 @@ class AzureBlobStorage[F[_]: Async] private (store: AzureStore[F], path: AzureBl } override def get(key: Key): F[Either[Throwable, String]] = + getBytes(key).compile + .to(Array) + .map(array => new String(array)) + .attempt + + override def getBytes(key: Key): Stream[F, Byte] = createStorageUrlFrom(key) match { case Valid(url) => store .get(url, 1024) - .compile - .to(Array) - .map(array => new String(array)) - .attempt case Invalid(errors) => - Async[F].delay(new MultipleUrlValidationException(errors).asLeft[String]) + Stream.raiseError[F](new MultipleUrlValidationException(errors)) } override def keyExists(key: Key): F[Boolean] = diff --git a/modules/common-transformer-stream/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/common/ParquetUtils.scala b/modules/common-transformer-stream/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/common/ParquetUtils.scala index 92bc04db8..ba42f343b 100644 --- a/modules/common-transformer-stream/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/common/ParquetUtils.scala +++ b/modules/common-transformer-stream/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/common/ParquetUtils.scala @@ -14,8 +14,11 @@ */ package com.snowplowanalytics.snowplow.rdbloader.transformer.stream.common +import cats.effect.IO import com.github.mjakubowski84.parquet4s._ +import com.github.mjakubowski84.parquet4s.parquet.fromParquet import io.circe.Json +import com.github.mjakubowski84.parquet4s.{Path => ParquetPath, RowParquetRecord} import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{Path => HadoopPath} import org.apache.parquet.column.ColumnDescriptor @@ -35,15 +38,15 @@ import java.time.temporal.ChronoUnit import java.time.{Instant, ZoneOffset} import java.util.TimeZone import scala.jdk.CollectionConverters._ - import fs2.io.file.Path +import scala.annotation.nowarn + object ParquetUtils { val config = ValueCodecConfiguration(TimeZone.getTimeZone(ZoneOffset.UTC)) def readParquetColumns(path: Path): Map[File, List[ColumnDescriptor]] = { - val conf = new Configuration(); val parquetFileFilter = new FileFilter { override def accept(pathname: File): Boolean = pathname.toString.endsWith(".parquet") } @@ -51,14 +54,21 @@ object ParquetUtils { new File(path.toString) .listFiles(parquetFileFilter) .map { parquetFile => - @annotation.nowarn("cat=deprecation") - val parquetMetadata = ParquetFileReader.readFooter(conf, new HadoopPath(parquetFile.toString), ParquetMetadataConverter.NO_FILTER) - val columns = parquetMetadata.getFileMetaData.getSchema.getColumns.asScala.toList - (parquetFile, columns) + (parquetFile, readFileColumns(parquetFile)) } .toMap } + @nowarn("cat=deprecation") + def readFileColumns(parquetFile: File): List[ColumnDescriptor] = + ParquetFileReader + .readFooter(new Configuration(), new HadoopPath(parquetFile.toString), ParquetMetadataConverter.NO_FILTER) + .getFileMetaData + .getSchema + .getColumns + .asScala + .toList + def extractColumnsFromSchemaString(schema: String) = MessageTypeParser .parseMessageType(schema) @@ -66,6 +76,18 @@ object ParquetUtils { .asScala .toList + def readParquetRowsAsJsonFrom(path: Path, columns: List[ColumnDescriptor]): IO[List[Json]] = + fromParquet[IO] + .as[RowParquetRecord] + .read(ParquetPath(path.toNioPath.toUri.toString)) + .map { record => + convertParquetRecordToJson(record, List.empty, columns) + } + .compile + .toList + .map(_.sortBy(_.asObject.flatMap(_("event_id")).flatMap(_.asString))) + .map(_.map(_.deepDropNullValues)) + def convertParquetRecordToJson( record: RowParquetRecord, parentPath: List[String], diff --git a/modules/common-transformer-stream/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/common/processing/TestApplication.scala b/modules/common-transformer-stream/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/common/processing/TestApplication.scala index af8b8b3f5..898bdfd9f 100644 --- a/modules/common-transformer-stream/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/common/processing/TestApplication.scala +++ b/modules/common-transformer-stream/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/common/processing/TestApplication.scala @@ -142,6 +142,9 @@ object TestApplication { override def get(path: Key): F[Either[Throwable, String]] = Concurrent[F].raiseError(new Exception("readKey isn't implemented for blob storage file type")) + override def getBytes(path: Key): Stream[F, Byte] = + Stream.empty + override def keyExists(key: Key): F[Boolean] = Concurrent[F].raiseError(new Exception(s"keyExists isn't implemented for blob storage file type")) } diff --git a/modules/common-transformer-stream/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/common/processing/WiderowParquetProcessingSpec.scala b/modules/common-transformer-stream/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/common/processing/WiderowParquetProcessingSpec.scala index 6b4564354..708fdb2ac 100644 --- a/modules/common-transformer-stream/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/common/processing/WiderowParquetProcessingSpec.scala +++ b/modules/common-transformer-stream/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/common/processing/WiderowParquetProcessingSpec.scala @@ -16,18 +16,13 @@ package com.snowplowanalytics.snowplow.rdbloader.transformer.stream.common.proce import cats.effect.IO import cats.effect.unsafe.implicits.global - -import fs2.io.file.Path - -import com.github.mjakubowski84.parquet4s.{Path => ParquetPath, RowParquetRecord} -import com.github.mjakubowski84.parquet4s.parquet.fromParquet import com.snowplowanalytics.snowplow.analytics.scalasdk.Event import com.snowplowanalytics.snowplow.analytics.scalasdk.SnowplowEvent.Contexts -import com.snowplowanalytics.snowplow.rdbloader.transformer.stream.common.AppId -import com.snowplowanalytics.snowplow.rdbloader.transformer.stream.common.ParquetUtils -import com.snowplowanalytics.snowplow.rdbloader.transformer.stream.common.ParquetUtils.readParquetColumns +import com.snowplowanalytics.snowplow.rdbloader.transformer.stream.common.{AppId, ParquetUtils} +import com.snowplowanalytics.snowplow.rdbloader.transformer.stream.common.ParquetUtils.{readParquetColumns, readParquetRowsAsJsonFrom} import com.snowplowanalytics.snowplow.rdbloader.transformer.stream.common.processing.BaseProcessingSpec.TransformerConfig import com.snowplowanalytics.snowplow.rdbloader.transformer.stream.common.processing.WiderowParquetProcessingSpec.{appConfig, igluConfig} +import fs2.io.file.Path import io.circe.syntax.EncoderOps import io.circe.{Json, JsonObject} import org.apache.parquet.column.ColumnDescriptor @@ -61,7 +56,7 @@ class WiderowParquetProcessingSpec extends BaseProcessingSpec { expectedParquetColumns <- readParquetColumnsFromResource( "/processing-spec/4/output/good/parquet/schema" ) // the same schema as in resource file used in WideRowParquetSpec for batch transformer - actualParquetRows <- readParquetRowsFrom(goodPath, expectedParquetColumns) + actualParquetRows <- readParquetRowsAsJsonFrom(goodPath, expectedParquetColumns) actualParquetColumns = readParquetColumns(goodPath) actualBadRows <- readStringRowsFrom(badPath) @@ -97,7 +92,7 @@ class WiderowParquetProcessingSpec extends BaseProcessingSpec { for { output <- process(inputStream, config) expectedParquetColumns <- readParquetColumnsFromResource("/processing-spec/5/output/good/parquet/schema") - actualParquetRows <- readParquetRowsFrom(goodPath, expectedParquetColumns) + actualParquetRows <- readParquetRowsAsJsonFrom(goodPath, expectedParquetColumns) actualParquetColumns = readParquetColumns(goodPath) expectedCompletionMessage <- readMessageFromResource("/processing-spec/5/output/good/parquet/completion.json", outputDirectory) expectedParquetRows <- readGoodParquetEventsFromResource( @@ -130,7 +125,7 @@ class WiderowParquetProcessingSpec extends BaseProcessingSpec { for { output <- process(inputStream, config) expectedParquetColumns <- readParquetColumnsFromResource("/processing-spec/6/output/good/parquet/schema") - actualParquetRows <- readParquetRowsFrom(goodPath, expectedParquetColumns) + actualParquetRows <- readParquetRowsAsJsonFrom(goodPath, expectedParquetColumns) actualParquetColumns = readParquetColumns(goodPath) expectedCompletionMessage <- readMessageFromResource("/processing-spec/6/output/good/parquet/completion.json", outputDirectory) expectedParquetRows <- @@ -166,7 +161,7 @@ class WiderowParquetProcessingSpec extends BaseProcessingSpec { expectedParquetColumns <- readParquetColumnsFromResource( "/processing-spec/7/output/good/parquet/schema" ) // the same schema as in resource file used in WideRowParquetSpec for batch transformer - actualParquetRows <- readParquetRowsFrom(goodPath, expectedParquetColumns) + actualParquetRows <- readParquetRowsAsJsonFrom(goodPath, expectedParquetColumns) actualParquetColumns = readParquetColumns(goodPath) expectedCompletionMessage <- readMessageFromResource("/processing-spec/7/output/good/parquet/completion.json", outputDirectory) } yield { @@ -205,18 +200,6 @@ class WiderowParquetProcessingSpec extends BaseProcessingSpec { .map(transformEventForParquetTest(columnToAdjust.getOrElse("none"))) } - private def readParquetRowsFrom(path: Path, columns: List[ColumnDescriptor]) = - fromParquet[IO] - .as[RowParquetRecord] - .read(ParquetPath(path.toNioPath.toUri.toString)) - .map { record => - ParquetUtils.convertParquetRecordToJson(record, List.empty, columns) - } - .compile - .toList - .map(_.sortBy(_.asObject.flatMap(_("event_id")).flatMap(_.asString))) - .map(_.map(_.deepDropNullValues)) - private def readParquetColumnsFromResource(path: String): IO[List[ColumnDescriptor]] = readLinesFromResource(path) .map(_.mkString) diff --git a/modules/common/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/common/cloud/BlobStorage.scala b/modules/common/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/common/cloud/BlobStorage.scala index 07466721b..3021e3908 100644 --- a/modules/common/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/common/cloud/BlobStorage.scala +++ b/modules/common/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/common/cloud/BlobStorage.scala @@ -35,6 +35,8 @@ trait BlobStorage[F[_]] { def get(path: BlobStorage.Key): F[Either[Throwable, String]] + def getBytes(path: BlobStorage.Key): Stream[F, Byte] + /** Check if blob storage key exist */ def keyExists(key: BlobStorage.Key): F[Boolean] } @@ -217,6 +219,9 @@ object BlobStorage { override def get(path: Key): F[Either[Throwable, String]] = MonadThrow[F].raiseError(new IllegalArgumentException("noop blobstorage interpreter")) + override def getBytes(path: Key): Stream[F, Byte] = + Stream.empty + override def keyExists(key: Key): F[Boolean] = MonadThrow[F].raiseError(new IllegalArgumentException("noop blobstorage interpreter")) } diff --git a/modules/gcp/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/gcp/GCS.scala b/modules/gcp/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/gcp/GCS.scala index 8ee49a9de..c2e814874 100644 --- a/modules/gcp/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/gcp/GCS.scala +++ b/modules/gcp/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/gcp/GCS.scala @@ -65,22 +65,24 @@ object GCS { ) } - override def get(key: Key): F[Either[Throwable, String]] = { + override def getBytes(key: Key): Stream[F, Byte] = { val (bucket, path) = BlobStorage.splitKey(key) Authority .parse(bucket) .fold( - errors => Async[F].delay(new MultipleUrlValidationException(errors).asLeft[String]), + errors => Stream.raiseError[F](new MultipleUrlValidationException(errors)), authority => client .get(Url("gs", authority, Path(path)), 1024) - .compile - .to(Array) - .map(array => new String(array)) - .attempt ) } + override def get(key: Key): F[Either[Throwable, String]] = + getBytes(key).compile + .to(Array) + .map(array => new String(array)) + .attempt + override def keyExists(key: Key): F[Boolean] = { val (bucket, path) = BlobStorage.splitKey(key) Authority diff --git a/modules/loader/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/test/PureAWS.scala b/modules/loader/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/test/PureAWS.scala index 5eec737db..9ec2a8400 100644 --- a/modules/loader/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/test/PureAWS.scala +++ b/modules/loader/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/test/PureAWS.scala @@ -21,6 +21,9 @@ object PureAWS { def get(path: Key): Pure[Either[Throwable, String]] = Pure.pure(Left(new NotImplementedError("Not used in tests"))) + def getBytes(path: Key): Stream[Pure, Byte] = + Stream.empty + def keyExists(key: Key): Pure[Boolean] = Pure.pure(results.keyExists(key)) } diff --git a/modules/transformer-kafka/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/kafka/experimental/AppConfiguration.scala b/modules/transformer-kafka/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/kafka/experimental/AppConfiguration.scala new file mode 100644 index 000000000..6e05b6f8e --- /dev/null +++ b/modules/transformer-kafka/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/kafka/experimental/AppConfiguration.scala @@ -0,0 +1,25 @@ +package com.snowplowanalytics.snowplow.rdbloader.transformer.stream.kafka.experimental +import com.snowplowanalytics.snowplow.rdbloader.common.LoaderMessage.TypesInfo.WideRow +import com.snowplowanalytics.snowplow.rdbloader.common.config.TransformerConfig.Compression + +final case class AppConfiguration( + compression: Compression, + fileFormat: WideRow.WideRowFormat, + windowFrequencyMinutes: Long +) + +object AppConfiguration { + + /** + * Regarding `windowFrequencyMinutes = 1` - officially the default 'windowing' setting for + * streaming transformer is '10 minutes'. As we don't want to make the tests take too much time, + * we use 1 minute here. It means that for all test scenarios using this default confguration, + * transformer instance under the test needs to be configured with `1 minute` windowing setting. + * + * Compression and file format defaults match the ones from the official reference file. + * + * See reference here -> + * https://github.com/snowplow/snowplow-rdb-loader/blob/master/modules/common-transformer-stream/src/main/resources/application.conf + */ + val default = AppConfiguration(Compression.Gzip, WideRow.WideRowFormat.JSON, windowFrequencyMinutes = 1) +} diff --git a/modules/transformer-kafka/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/kafka/experimental/AppDependencies.scala b/modules/transformer-kafka/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/kafka/experimental/AppDependencies.scala new file mode 100644 index 000000000..b4ddedc15 --- /dev/null +++ b/modules/transformer-kafka/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/kafka/experimental/AppDependencies.scala @@ -0,0 +1,17 @@ +package com.snowplowanalytics.snowplow.rdbloader.transformer.stream.kafka.experimental +import cats.effect.IO +import cats.effect.kernel.Resource +import com.snowplowanalytics.snowplow.rdbloader.common.cloud.{BlobStorage, Queue} + +final case class AppDependencies( + blobClient: BlobStorage[IO], + queueConsumer: Queue.Consumer[IO], + producer: Queue.Producer[IO] +) + +object AppDependencies { + + trait Provider { + def createDependencies(): Resource[IO, AppDependencies] + } +} diff --git a/modules/transformer-kafka/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/kafka/experimental/AzureTransformerSpecification.scala b/modules/transformer-kafka/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/kafka/experimental/AzureTransformerSpecification.scala new file mode 100644 index 000000000..5f32e772a --- /dev/null +++ b/modules/transformer-kafka/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/kafka/experimental/AzureTransformerSpecification.scala @@ -0,0 +1,95 @@ +package com.snowplowanalytics.snowplow.rdbloader.transformer.stream.kafka.experimental +import cats.effect.IO +import cats.effect.kernel.Resource +import com.snowplowanalytics.snowplow.rdbloader.azure.{AzureBlobStorage, KafkaConsumer, KafkaProducer} +import com.snowplowanalytics.snowplow.rdbloader.common.cloud.{BlobStorage, Queue} +import com.snowplowanalytics.snowplow.rdbloader.transformer.stream.kafka.experimental.AzureTransformerSpecification._ + +import java.net.URI +import java.util.UUID + +// format: off +/** + * Trait providing Azure specific dependencies for the generic `TransformerSpecification`. + * Before running tests with Azure support, it's necessary to: + * + * 1) Launch transformer application in the Azure cloud. + * 2) Set up input (enriched data) and output (for `shredding_compelete.json` message to notify loader) EventHubs. + * 3) Set up Azure Blob Storage account and container for transformed data. + * 4) Set following environment variables with values specific for your cloud setup: + * + * - TEST_TRANSFORMER_BLOB_STORAGE_URL (format like 'https://${accountName}.blob.core.windows.net/${containerName}) + * - TEST_TRANSFORMER_EVENTHUBS_URL (format like '...-namespace.servicebus.windows.net:9093") + * - TEST_TRANSFORMER_INPUT_HUB_NAME + * - TEST_TRANSFORMER_OUTPUT_HUB_NAME + * - TEST_TRANSFORMER_INPUT_HUB_KEY + * - TEST_TRANSFORMER_OUTPUT_HUB_KEY + * - AZURE_TENANT_ID - may be used by the azure-sdk library to communicate with the cloud if you wish to authenticate with Azure CLI. + * It can be extracted e.g. with `az account show | jq -r '.homeTenantId'` executed in command line. + */ +// format: on +trait AzureTransformerSpecification extends TransformerSpecification with AppDependencies.Provider { + skipAllIf(anyEnvironementVariableMissing()) + + override def createDependencies(): Resource[IO, AppDependencies] = + for { + blobClient <- createBlobClient() + consumer <- createConsumer() + producer <- createProducer() + } yield AppDependencies(blobClient, consumer, producer) + + private def createBlobClient(): Resource[IO, BlobStorage[IO]] = + AzureBlobStorage + .createDefault[IO](URI.create(System.getenv(blobStorageUrlEnv))) + + private def createConsumer(): Resource[IO, Queue.Consumer[IO]] = + KafkaConsumer + .consumer[IO]( + bootstrapServers = System.getenv(eventHubsUrlEnv), + topicName = System.getenv(outputHubNameEnv), + consumerConf = Map( + "security.protocol" -> "SASL_SSL", + "sasl.mechanism" -> "PLAIN", + "sasl.jaas.config" -> s"""org.apache.kafka.common.security.plain.PlainLoginModule required username=\"$$ConnectionString\" password=\"${System + .getenv(outputHubKeyEnv)}\";""", + "group.id" -> s"test-transformer-consumer-${UUID.randomUUID()}", + "enable.auto.commit" -> "true", + "auto.offset.reset" -> "latest" + ) + ) + + private def createProducer(): Resource[IO, Queue.Producer[IO]] = + KafkaProducer + .producer[IO]( + bootstrapServers = System.getenv(eventHubsUrlEnv), + topicName = System.getenv(inputHubNameEnv), + producerConf = Map( + "security.protocol" -> "SASL_SSL", + "sasl.mechanism" -> "PLAIN", + "sasl.jaas.config" -> s"""org.apache.kafka.common.security.plain.PlainLoginModule required username=\"$$ConnectionString\" password=\"${System + .getenv(inputHubKeyEnv)}\";""" + ) + ) +} + +object AzureTransformerSpecification { + val blobStorageUrlEnv = "TEST_TRANSFORMER_BLOB_STORAGE_URL" + val eventHubsUrlEnv = "TEST_TRANSFORMER_EVENTHUBS_URL" + val inputHubNameEnv = "TEST_TRANSFORMER_INPUT_HUB_NAME" + val outputHubNameEnv = "TEST_TRANSFORMER_OUTPUT_HUB_NAME" + val inputHubKeyEnv = "TEST_TRANSFORMER_INPUT_HUB_KEY" + val outputHubKeyEnv = "TEST_TRANSFORMER_OUTPUT_HUB_KEY" + + val requiredEnvironmentVariables = List( + blobStorageUrlEnv, + eventHubsUrlEnv, + inputHubNameEnv, + outputHubNameEnv, + inputHubKeyEnv, + outputHubKeyEnv + ) + + def anyEnvironementVariableMissing(): Boolean = + requiredEnvironmentVariables.exists(varName => System.getenv(varName) == null) + +} diff --git a/modules/transformer-kafka/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/kafka/experimental/InputBatch.scala b/modules/transformer-kafka/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/kafka/experimental/InputBatch.scala new file mode 100644 index 000000000..3d1c6cfa2 --- /dev/null +++ b/modules/transformer-kafka/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/kafka/experimental/InputBatch.scala @@ -0,0 +1,48 @@ +package com.snowplowanalytics.snowplow.rdbloader.transformer.stream.kafka.experimental +import com.snowplowanalytics.snowplow.analytics.scalasdk.Event +import com.snowplowanalytics.snowplow.eventgen.enrich.{SdkEvent => EventGenerator} +import com.snowplowanalytics.snowplow.eventgen.protocol.event.{EventFrequencies, UnstructEventFrequencies} +import org.scalacheck.Gen +import org.scalacheck.rng.Seed + +import java.time.Instant +import scala.concurrent.duration.{DurationInt, FiniteDuration} +import scala.util.Random + +final case class InputBatch(content: InputBatch.Content, delay: FiniteDuration = 0.minutes) { + def delayed(value: FiniteDuration) = this.copy(delay = value) +} + +object InputBatch { + + sealed trait Content + object Content { + final case class TextLines(lines: List[String]) extends Content + final case class SdkEvents(events: List[Event]) extends Content + } + + def good(count: Int): InputBatch = InputBatch( + Content.SdkEvents( + EventGenerator + .gen( + eventPerPayloadMin = count, + eventPerPayloadMax = count, + now = Instant.now(), + frequencies = EventFrequencies(1, 1, 1, 1, UnstructEventFrequencies(1, 1, 1)) + ) + .apply(Gen.Parameters.default, Seed(Random.nextLong())) + .get + ) + ) + + def bad(count: Int): InputBatch = InputBatch( + Content.TextLines { + (1 to count).map(idx => s"Some broken input - $idx").toList + } + ) + + def asTextLines(content: Content): List[String] = content match { + case Content.TextLines(lines) => lines + case Content.SdkEvents(events) => events.map(_.toTsv) + } +} diff --git a/modules/transformer-kafka/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/kafka/experimental/OutputDataRowReader.scala b/modules/transformer-kafka/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/kafka/experimental/OutputDataRowReader.scala new file mode 100644 index 000000000..8ad2d4890 --- /dev/null +++ b/modules/transformer-kafka/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/kafka/experimental/OutputDataRowReader.scala @@ -0,0 +1,40 @@ +package com.snowplowanalytics.snowplow.rdbloader.transformer.stream.kafka.experimental +import cats.effect.IO +import com.snowplowanalytics.snowplow.rdbloader.transformer.stream.common.ParquetUtils +import com.snowplowanalytics.snowplow.rdbloader.transformer.stream.kafka.experimental.TransformerSpecification.{Blob, DataRow} +import fs2.io.file.{Files, Path} +import io.circe.parser + +import java.nio.charset.StandardCharsets + +object OutputDataRowReader { + + def fromJson(blob: Blob): IO[List[DataRow]] = + fs2.Stream + .emits[IO, Byte](blob) + .through(fs2.text.decodeWithCharset(StandardCharsets.UTF_8)) + .through(fs2.text.lines) + .map(parser.parse(_).right.get) + .compile + .toList + + // For parquet we fetch all bytes from remote blob storage and store them in the temporary local output. + // Then we use hadoop API (details in the `ParquetUtils`) to decode it and convert to human-readable JSON format. + def fromParquet(blob: Blob): IO[List[DataRow]] = + Files[IO].tempFile + .use { tempOutput => + for { + _ <- saveParquetDataToTemporaryOutput(tempOutput, blob) + outputParquetColumns = ParquetUtils.readFileColumns(tempOutput.toNioPath.toFile) + parquetRows <- ParquetUtils.readParquetRowsAsJsonFrom(tempOutput, outputParquetColumns) + } yield parquetRows + } + + private def saveParquetDataToTemporaryOutput(outputPath: Path, blob: Blob): IO[Unit] = + fs2.Stream + .emits(blob) + .through(Files[IO].writeAll(outputPath)) + .compile + .drain + +} diff --git a/modules/transformer-kafka/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/kafka/experimental/TransformerSpecification.scala b/modules/transformer-kafka/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/kafka/experimental/TransformerSpecification.scala new file mode 100644 index 000000000..5810c7029 --- /dev/null +++ b/modules/transformer-kafka/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/kafka/experimental/TransformerSpecification.scala @@ -0,0 +1,279 @@ +package com.snowplowanalytics.snowplow.rdbloader.transformer.stream.kafka.experimental + +import cats.effect.IO +import cats.effect.unsafe.implicits.global +import cats.implicits._ +import com.snowplowanalytics.snowplow.rdbloader.common.LoaderMessage +import com.snowplowanalytics.snowplow.rdbloader.common.LoaderMessage.TypesInfo +import com.snowplowanalytics.snowplow.rdbloader.common.LoaderMessage.TypesInfo.WideRow.WideRowFormat +import com.snowplowanalytics.snowplow.rdbloader.common.cloud.BlobStorage +import com.snowplowanalytics.snowplow.rdbloader.common.cloud.BlobStorage.{BlobObject, Folder} +import com.snowplowanalytics.snowplow.rdbloader.common.config.TransformerConfig.Compression +import com.snowplowanalytics.snowplow.rdbloader.transformer.stream.kafka.experimental.TransformerSpecification._ +import fs2.Stream +import fs2.compression.{Compression => FS2Compression} +import fs2.concurrent.Signal.SignalOps +import fs2.concurrent.{Signal, SignallingRef} +import io.circe.Json +import org.specs2.matcher.MatchResult +import org.specs2.mutable.Specification + +import java.time.LocalDateTime +import java.time.format.DateTimeFormatter +import java.time.temporal.ChronoUnit +import scala.concurrent.duration.DurationInt + +// format: off +/** + * Generic template for the streaming transformer specification. It produces specified input (could be delayed batches), + * waits for incoming windows (can be more than one) with data and asserts produced by the transformer output. + * It assumes transformer app is already running somewhere with accessible input/output. + * It verifies facts about transformer which should be true for any kind of transformation. + * + * It's an abstract class and can't be run on its own. Specific implementation needs to implement abstract members like: + * - 'description' - details about scenario + * - 'inputBatches' - list of input events. Could be any kind of good/bad data in the TSV format. Delivery of a batch can be delayed if any delay is specified. + * - 'countExpectations' - how many good/bad events are expected on the output. If we reach expected count, test is interrupted and assertions are run. + * - 'requiredAppConfig' - expected configuration details for tested transformer application + * - 'customDataAssertion' - optional, allows to execute more detailed assertions of an output, like checking values for transformed events. + * + * Dependencies like blob storage client and queue consumer/producer need to be supplied by a proper `AppDependencies.Provider` implementation. + * + * What kind of assertions are run: + * - `shredding_complete.json` exists in the storage + * - `shredding_complete.json` consumed from the queue is equal to the one in the storage + * - `shredding_complete.json` count values matches the number of actual output events (good and bad) in the storage (for each window) + * - expected output format is used + * - expected output compression is used + * - windows are emitted with expected frequency + */ +// format: on + +abstract class TransformerSpecification extends Specification with AppDependencies.Provider { + + private val timeout = 15.minutes + + protected def description: String + protected def inputBatches: List[InputBatch] + protected def countExpectations: CountExpectations + protected def requiredAppConfig: AppConfiguration + protected def customDataAssertion: Option[DataAssertion] = None + + s"$description" in { + run().unsafeRunSync() + } + + def run(): IO[MatchResult[Any]] = + createDependencies() + .use { dependencies => + for { + windowsAccumulator <- SignallingRef.of[IO, WindowsAccumulator](WindowsAccumulator(List.empty)) + consuming = consumeAllIncomingWindows(dependencies, countExpectations, windowsAccumulator) + producing = produceInput(inputBatches, dependencies) + _ <- consuming.concurrently(producing).compile.drain + collectedWindows <- windowsAccumulator.get + } yield { + collectedWindows.value.foreach(assertSingleWindow) + assertWindowingFrequency(collectedWindows.value) + + val aggregatedData = aggregateDataFromAllWindows(collectedWindows.value) + assertAggregatedCounts(aggregatedData) + customDataAssertion.fold(ok)(_.apply(aggregatedData)) + } + } + + private def produceInput(batches: List[InputBatch], dependencies: AppDependencies): Stream[IO, Unit] = + Stream.eval { + IO.sleep(10.seconds) *> batches.traverse_ { batch => + IO.sleep(batch.delay) *> InputBatch + .asTextLines(batch.content) + .parTraverse_(dependencies.producer.send) + } + } + + private def consumeAllIncomingWindows( + dependencies: AppDependencies, + countExpectations: CountExpectations, + windowsAccumulator: SignallingRef[IO, WindowsAccumulator] + ): Stream[IO, Unit] = + dependencies.queueConsumer.read + .map(_.content) + .map(parseShreddingCompleteMessage) + .evalMap(readWindowOutput(dependencies.blobClient)) + .evalMap { windowOutput => + windowsAccumulator.update(_.addWindow(windowOutput)) + } + .interruptWhen(allEventsProcessed(windowsAccumulator, countExpectations)) + .interruptAfter(timeout) + + private def assertSingleWindow(output: WindowOutput): MatchResult[Any] = { + output.`shredding_complete.json`.compression must beEqualTo(requiredAppConfig.compression) + output.`shredding_complete.json`.count.get.good must beEqualTo(output.goodEvents.size) + output.`shredding_complete.json`.count.get.bad.get must beEqualTo(output.badEvents.size) + output.`shredding_complete.json`.typesInfo must beLike { case TypesInfo.WideRow(fileFormat, _) => + fileFormat must beEqualTo(requiredAppConfig.fileFormat) + } + } + + private def assertWindowingFrequency(collectedWindows: List[WindowOutput]): Unit = + collectedWindows.groupBy(_.appId).foreach { case (_, windows) => + windows.zip(windows.tail).foreach { case (window1, window2) => + ChronoUnit.MINUTES.between(window1.producedAt, window2.producedAt) must beEqualTo(requiredAppConfig.windowFrequencyMinutes) + } + } + + private def assertAggregatedCounts(aggregatedData: AggregatedData): MatchResult[Any] = { + aggregatedData.good.size must beEqualTo(countExpectations.good) + aggregatedData.bad.size must beEqualTo(countExpectations.bad) + } + + private def aggregateDataFromAllWindows(windows: List[WindowOutput]): AggregatedData = + windows.foldLeft(AggregatedData.empty) { case (aggregated, window) => + val windowTypes = window.`shredding_complete.json`.typesInfo.asInstanceOf[TypesInfo.WideRow].types + AggregatedData( + good = window.goodEvents ::: aggregated.good, + bad = window.badEvents ::: aggregated.bad, + types = windowTypes ::: aggregated.types + ) + } + + private def allEventsProcessed( + windowsAccumulator: SignallingRef[IO, WindowsAccumulator], + countExpectations: CountExpectations + ): Signal[IO, Boolean] = + windowsAccumulator + .map(_.getTotalNumberOfEvents >= countExpectations.total) + + private def readWindowOutput(blobClient: BlobStorage[IO])(message: LoaderMessage.ShreddingComplete): IO[WindowOutput] = + for { + scMessageInStorage <- readSCMessageFromBlobStorage(message, blobClient) + transformedEvents <- readDataRowsFromFolder(scMessageInStorage.base.append("output=good"), blobClient) + badEvents <- readDataRowsFromFolder(scMessageInStorage.base.append("output=bad"), blobClient) + } yield { + + message must beEqualTo(scMessageInStorage) + + // Assuming folder name structure ending like '.../run=yyyy-MM-dd-HH-mm-ss-${UUID}/' + val base = message.base.stripSuffix("/") + val appID = base.takeRight(36) // extract application ID which is represented by UUID at the end of a folder name + val time = base.stripSuffix(appID).stripSuffix("-").takeRight(19) // extract date and time of the window, but without `run=` prefix + val parsedTime = LocalDateTime.parse(time, DateTimeFormatter.ofPattern("yyyy-MM-dd-HH-mm-ss")) + + println( + s"""Received `shredding_complete.json` message: + |- folder - ${message.base} + |- counts - ${message.count} + |""".stripMargin + ) + + WindowOutput(appID, parsedTime, message, transformedEvents, badEvents) + } + + private def readDataRowsFromFolder( + folder: Folder, + blobClient: BlobStorage[IO] + ): IO[List[DataRow]] = + blobClient + .list(folder, recursive = false) + .evalMap(blob => readDataRowsFromBlob(blobClient, blob)) + .flatMap(dataRows => Stream.emits(dataRows)) + .compile + .toList + + private def readDataRowsFromBlob( + blobClient: BlobStorage[IO], + blob: BlobObject + ): IO[List[DataRow]] = + blobClient + .getBytes(blob.key) + .through(decompressIfNeeded(blob.key)) + .compile + .to(Array) + .flatMap(convertBlobToListOfRows(blob.key)) + + private def convertBlobToListOfRows(blobKey: BlobStorage.Key)(blob: Blob): IO[List[DataRow]] = + if (isBlobGoodParquetData(blobKey)) + OutputDataRowReader.fromParquet(blob) + else + OutputDataRowReader.fromJson(blob) + + // Decompress only for: + // - JSON good/bad output + // - Parquet bad output + // Decompression for parquet good data is handled by hadoop API in parquet-oriented scenarios + private def decompressIfNeeded(blobKey: BlobStorage.Key): fs2.Pipe[IO, Byte, Byte] = + requiredAppConfig.compression match { + case Compression.Gzip if !isBlobGoodParquetData(blobKey) => + FS2Compression[IO].gunzip().andThen(_.flatMap(_.content)) + case _ => + identity + } + + private def isBlobGoodParquetData(blobKey: BlobStorage.Key): Boolean = + requiredAppConfig.fileFormat == WideRowFormat.PARQUET && blobKey.contains("output=good") + + private def readSCMessageFromBlobStorage( + message: LoaderMessage.ShreddingComplete, + blobClient: BlobStorage[IO] + ): IO[LoaderMessage.ShreddingComplete] = + scMessageMustExist(message, blobClient) *> fetchSCMessage(message, blobClient) + + private def scMessageMustExist(message: LoaderMessage.ShreddingComplete, blobClient: BlobStorage[IO]) = + blobClient + .keyExists(message.base.withKey("shredding_complete.json")) + .map(messageExists => messageExists must beTrue) + + private def fetchSCMessage(message: LoaderMessage.ShreddingComplete, blobClient: BlobStorage[IO]): IO[LoaderMessage.ShreddingComplete] = + blobClient + .get(message.base.withKey("shredding_complete.json")) + .map(value => parseShreddingCompleteMessage(value.right.get)) + + private def parseShreddingCompleteMessage(message: String): LoaderMessage.ShreddingComplete = + LoaderMessage + .fromString(message) + .right + .get + .asInstanceOf[LoaderMessage.ShreddingComplete] + +} + +object TransformerSpecification { + + type Blob = Array[Byte] + type DataRow = Json + type DataAssertion = AggregatedData => MatchResult[Any] + + final case class CountExpectations(good: Int, bad: Int) { + def total = good + bad + } + + final case class WindowsAccumulator(value: List[WindowOutput]) { + def addWindow(window: WindowOutput): WindowsAccumulator = + WindowsAccumulator(value :+ window) + + def getTotalNumberOfEvents: Long = + value.map { window => + val good = window.`shredding_complete.json`.count.map(_.good).getOrElse(0L) + val bad = window.`shredding_complete.json`.count.flatMap(_.bad).getOrElse(0L) + good + bad + }.sum + } + + final case class WindowOutput( + appId: String, + producedAt: LocalDateTime, + `shredding_complete.json`: LoaderMessage.ShreddingComplete, + goodEvents: List[DataRow], + badEvents: List[DataRow] + ) + + final case class AggregatedData( + good: List[DataRow], + bad: List[DataRow], + types: List[TypesInfo.WideRow.Type] + ) + + object AggregatedData { + val empty = AggregatedData(List.empty, List.empty, List.empty) + } +} diff --git a/modules/transformer-kafka/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/kafka/experimental/scenarios/BadDetailsScenario.scala b/modules/transformer-kafka/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/kafka/experimental/scenarios/BadDetailsScenario.scala new file mode 100644 index 000000000..a4f5b696d --- /dev/null +++ b/modules/transformer-kafka/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/kafka/experimental/scenarios/BadDetailsScenario.scala @@ -0,0 +1,28 @@ +package com.snowplowanalytics.snowplow.rdbloader.transformer.stream.kafka.experimental.scenarios + +import com.snowplowanalytics.snowplow.rdbloader.transformer.stream.kafka.experimental.InputBatch.Content +import com.snowplowanalytics.snowplow.rdbloader.transformer.stream.kafka.experimental.TransformerSpecification.CountExpectations +import com.snowplowanalytics.snowplow.rdbloader.transformer.stream.kafka.experimental.{ + AppConfiguration, + AzureTransformerSpecification, + InputBatch +} + +class BadDetailsScenario extends AzureTransformerSpecification { + + private val badEvent = Content.TextLines(List("Some example bad event")) + + override def description = "Asserting details of output single bad row" + override def requiredAppConfig = AppConfiguration.default + override def inputBatches = List(InputBatch(badEvent)) + override def countExpectations = CountExpectations(good = 0, bad = 1) + + override def customDataAssertion = Some { outputData => + val badRow = outputData.bad.head + badRow.hcursor.get[String]("schema").right.get must beEqualTo( + "iglu:com.snowplowanalytics.snowplow.badrows/loader_parsing_error/jsonschema/2-0-0" + ) + badRow.hcursor.downField("data").get[String]("payload").right.get must beEqualTo("Some example bad event") + badRow.hcursor.downField("data").downField("failure").get[String]("type").right.get must beEqualTo("NotTSV") + } +} diff --git a/modules/transformer-kafka/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/kafka/experimental/scenarios/json.scala b/modules/transformer-kafka/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/kafka/experimental/scenarios/json.scala new file mode 100644 index 000000000..fd248d9a2 --- /dev/null +++ b/modules/transformer-kafka/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/kafka/experimental/scenarios/json.scala @@ -0,0 +1,244 @@ +package com.snowplowanalytics.snowplow.rdbloader.transformer.stream.kafka.experimental.scenarios +import com.snowplowanalytics.snowplow.rdbloader.common.config.TransformerConfig.Compression +import com.snowplowanalytics.snowplow.rdbloader.transformer.stream.kafka.experimental.InputBatch.{Content, bad, good} +import com.snowplowanalytics.snowplow.rdbloader.transformer.stream.kafka.experimental.TransformerSpecification.CountExpectations +import com.snowplowanalytics.snowplow.rdbloader.transformer.stream.kafka.experimental.{ + AppConfiguration, + AzureTransformerSpecification, + InputBatch +} +import io.circe.parser + +import scala.concurrent.duration.DurationInt + +class JsonScenario1 extends AzureTransformerSpecification { + def description = "Input: 1 good, config: JSON, compression, windowing: 1 minute" + def requiredAppConfig = AppConfiguration.default + def inputBatches = List(good(count = 1)) + def countExpectations = CountExpectations(good = 1, bad = 0) +} + +class JsonScenario2 extends AzureTransformerSpecification { + def description = "Input: 1 bad, config: JSON, compression, windowing: 1 minute" + def requiredAppConfig = AppConfiguration.default + def inputBatches = List(bad(count = 1)) + def countExpectations = CountExpectations(good = 0, bad = 1) +} + +class JsonScenario3 extends AzureTransformerSpecification { + def description = "Input: 10000 good, config: JSON, compression, windowing: 1 minute" + def requiredAppConfig = AppConfiguration.default + def inputBatches = List(good(count = 10000)) + def countExpectations = CountExpectations(good = 10000, bad = 0) +} + +class JsonScenario4 extends AzureTransformerSpecification { + def description = "Input: 10000 bad, config: JSON, compression, windowing: 1 minute" + def requiredAppConfig = AppConfiguration.default + def inputBatches = List(bad(count = 10000)) + def countExpectations = CountExpectations(good = 0, bad = 10000) +} + +class JsonScenario5 extends AzureTransformerSpecification { + def description = """Input: mixed 5000 good and 5000 bad, config: JSON, compression, windowing: 1 minute""" + def requiredAppConfig = AppConfiguration.default + def inputBatches = List(good(count = 5000), bad(count = 5000)) + def countExpectations = CountExpectations(good = 5000, bad = 5000) +} + +//Changed defualt windowing to 2 minutes +class JsonScenario6 extends AzureTransformerSpecification { + def description = """Input: mixed 5000 good and 5000 bad, config: JSON, compression, windowing: 2 minutes""" + def requiredAppConfig = AppConfiguration.default.copy(windowFrequencyMinutes = 2) + def inputBatches = List(good(count = 5000), good(count = 5000).delayed(2.minutes)) // force new window by delaying second input batch + def countExpectations = CountExpectations(good = 10000, bad = 0) +} + +//No compression +class JsonScenario7 extends AzureTransformerSpecification { + def description = """Input: mixed 5000 good and 5000 bad, config: JSON, no compression, windowing: 1 minute""" + def requiredAppConfig = AppConfiguration.default.copy(compression = Compression.None) + def inputBatches = List(good(count = 5000), bad(count = 5000)) + def countExpectations = CountExpectations(good = 5000, bad = 5000) +} + +//Checking details of JSON output +class JsonOutputDetailsScenario extends AzureTransformerSpecification { + + private val goodEvent = Content.TextLines( + List( + """snowplowweb web 2014-06-01 14:04:11.639 2014-05-29 18:16:35.000 2014-05-29 18:16:35.967 unstruct 2b1b25a4-c0df-4859-8201-cf21492ad61b 114221 clojure js-2.0.0-M2 clj-0.6.0-tom-0.0.4 hadoop-0.5.0-common-0.4.0 68.42.204.218 1242058182 58df65c46e1ac937 11 437ad25b-2006-455e-b5d8-d664b74df8f3 US MI Holland 49423 42.742294 -86.0661 http://snowplowanalytics.com/blog/ https://www.google.com/ http snowplowanalytics.com 80 /blog/ https www.google.com 80 / search Google {"schema":"iglu:com.snowplowanalytics.snowplow/contexts/jsonschema/1-0-0","data":[{"schema":"iglu:org.schema/WebPage/jsonschema/1-0-0","data":{"datePublished":"2014-07-23T00:00:00Z","author":"Jonathan Almeida","inLanguage":"en-US","genre":"blog","breadcrumb":["blog","releases"],"keywords":["snowplow","analytics","java","jvm","tracker"]}}]} {"schema":"iglu:com.snowplowanalytics.snowplow/unstruct_event/jsonschema/1-0-0","data":{"schema":"iglu:com.snowplowanalytics.snowplow/link_click/jsonschema/1-0-0","data":{"targetUrl":"http://snowplowanalytics.com/blog/page2","elementClasses":["next"]}}} Mozilla/5.0 (Macintosh; Intel Mac OS X 10_9_3) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/35.0.1916.114 Safari/537.36 Chrome Chrome Browser WEBKIT en-US 1 1 1 0 1 0 0 0 1 1 24 1241 806 Mac OS Mac OS Apple Inc. America/New_York Computer 0 1440 900 UTF-8 """ + ) + ) + override def description = "Asserting details of a single JSON transformed event" + override def requiredAppConfig = AppConfiguration.default + override def inputBatches = List(InputBatch(goodEvent)) + override def countExpectations = CountExpectations(good = 1, bad = 0) + + override def customDataAssertion = Some { outputData => + val transformedEvent = outputData.good.head + val expectedEvent = parser + .parse(""" + |{ + | "page_urlhost": "snowplowanalytics.com", + | "br_features_realplayer": false, + | "etl_tstamp": "2014-06-01T14:04:11.639Z", + | "dvce_ismobile": false, + | "geo_latitude": 42.742294, + | "refr_medium": "search", + | "ti_orderid": null, + | "br_version": null, + | "base_currency": null, + | "v_collector": "clj-0.6.0-tom-0.0.4", + | "mkt_content": null, + | "collector_tstamp": "2014-05-29T18:16:35Z", + | "os_family": "Mac OS", + | "ti_sku": null, + | "event_vendor": null, + | "network_userid": "437ad25b-2006-455e-b5d8-d664b74df8f3", + | "br_renderengine": "WEBKIT", + | "br_lang": "en-US", + | "tr_affiliation": null, + | "ti_quantity": null, + | "ti_currency": null, + | "geo_country": "US", + | "user_fingerprint": "1242058182", + | "mkt_medium": null, + | "page_urlscheme": "http", + | "ti_category": null, + | "pp_yoffset_min": null, + | "br_features_quicktime": true, + | "event": "unstruct", + | "refr_urlhost": "www.google.com", + | "user_ipaddress": "68.42.204.218", + | "br_features_pdf": true, + | "page_referrer": "https://www.google.com/", + | "doc_height": null, + | "refr_urlscheme": "https", + | "geo_region": "MI", + | "geo_timezone": null, + | "page_urlfragment": null, + | "br_features_flash": true, + | "os_manufacturer": "Apple Inc.", + | "mkt_clickid": null, + | "ti_price": null, + | "br_colordepth": "24", + | "event_format": null, + | "tr_total": null, + | "contexts_org_schema_web_page_1": [ + | { + | "datePublished": "2014-07-23T00:00:00Z", + | "author": "Jonathan Almeida", + | "inLanguage": "en-US", + | "genre": "blog", + | "breadcrumb": [ + | "blog", + | "releases" + | ], + | "keywords": [ + | "snowplow", + | "analytics", + | "java", + | "jvm", + | "tracker" + | ] + | } + | ], + | "pp_xoffset_min": null, + | "doc_width": null, + | "geo_zipcode": "49423", + | "br_family": "Chrome", + | "tr_currency": null, + | "useragent": "Mozilla/5.0 (Macintosh; Intel Mac OS X 10_9_3) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/35.0.1916.114 Safari/537.36", + | "event_name": null, + | "os_name": "Mac OS", + | "page_urlpath": "/blog/", + | "br_name": "Chrome", + | "ip_netspeed": null, + | "page_title": null, + | "ip_organization": null, + | "dvce_created_tstamp": "2014-05-29T18:16:35.967Z", + | "br_features_gears": false, + | "dvce_type": "Computer", + | "dvce_sent_tstamp": null, + | "se_action": null, + | "br_features_director": false, + | "se_category": null, + | "ti_name": null, + | "user_id": null, + | "refr_urlquery": null, + | "true_tstamp": null, + | "geo_longitude": -86.0661, + | "mkt_term": null, + | "v_tracker": "js-2.0.0-M2", + | "os_timezone": "America/New_York", + | "br_type": "Browser", + | "br_features_windowsmedia": false, + | "event_version": null, + | "dvce_screenwidth": 1440, + | "refr_dvce_tstamp": null, + | "se_label": null, + | "domain_sessionid": null, + | "domain_userid": "58df65c46e1ac937", + | "page_urlquery": null, + | "geo_location": "42.742294,-86.0661", + | "refr_term": null, + | "name_tracker": "clojure", + | "tr_tax_base": null, + | "dvce_screenheight": 900, + | "mkt_campaign": null, + | "refr_urlfragment": null, + | "tr_shipping": null, + | "tr_shipping_base": null, + | "br_features_java": true, + | "br_viewwidth": 1241, + | "geo_city": "Holland", + | "unstruct_event_com_snowplowanalytics_snowplow_link_click_1": { + | "targetUrl": "http://snowplowanalytics.com/blog/page2", + | "elementClasses": [ + | "next" + | ] + | }, + | "br_viewheight": 806, + | "refr_domain_userid": null, + | "br_features_silverlight": true, + | "ti_price_base": null, + | "tr_tax": null, + | "br_cookies": true, + | "tr_total_base": null, + | "refr_urlport": 80, + | "derived_tstamp": null, + | "app_id": "snowplowweb", + | "ip_isp": null, + | "geo_region_name": null, + | "pp_yoffset_max": null, + | "ip_domain": null, + | "domain_sessionidx": 11, + | "pp_xoffset_max": null, + | "mkt_source": null, + | "page_urlport": 80, + | "se_property": null, + | "platform": "web", + | "event_id": "2b1b25a4-c0df-4859-8201-cf21492ad61b", + | "refr_urlpath": "/", + | "mkt_network": null, + | "se_value": null, + | "page_url": "http://snowplowanalytics.com/blog/", + | "etl_tags": null, + | "tr_orderid": null, + | "tr_state": null, + | "txn_id": 114221, + | "refr_source": "Google", + | "tr_country": null, + | "tr_city": null, + | "doc_charset": "UTF-8", + | "event_fingerprint": null, + | "v_etl": "hadoop-0.5.0-common-0.4.0" + |} + |""".stripMargin) + .right + .get + + transformedEvent must beEqualTo(expectedEvent) + } +} diff --git a/modules/transformer-kafka/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/kafka/experimental/scenarios/parquet.scala b/modules/transformer-kafka/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/kafka/experimental/scenarios/parquet.scala new file mode 100644 index 000000000..3b45153f8 --- /dev/null +++ b/modules/transformer-kafka/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/kafka/experimental/scenarios/parquet.scala @@ -0,0 +1,309 @@ +package com.snowplowanalytics.snowplow.rdbloader.transformer.stream.kafka.experimental.scenarios + +import com.snowplowanalytics.snowplow.rdbloader.common.LoaderMessage.TypesInfo.WideRow.WideRowFormat +import com.snowplowanalytics.snowplow.rdbloader.common.config.TransformerConfig.Compression +import com.snowplowanalytics.snowplow.rdbloader.transformer.stream.kafka.experimental.InputBatch.{Content, bad, good} +import com.snowplowanalytics.snowplow.rdbloader.transformer.stream.kafka.experimental.TransformerSpecification.CountExpectations +import com.snowplowanalytics.snowplow.rdbloader.transformer.stream.kafka.experimental.{ + AppConfiguration, + AzureTransformerSpecification, + InputBatch +} +import io.circe.parser + +import scala.concurrent.duration.DurationInt + +class ParquetScenario1 extends AzureTransformerSpecification { + def description = "Input: 1 good, config: PARQUET, compression, windowing: 1 minute" + def requiredAppConfig = AppConfiguration.default.copy(fileFormat = WideRowFormat.PARQUET) + def inputBatches = List(good(count = 1)) + def countExpectations = CountExpectations(good = 1, bad = 0) +} + +class ParquetScenario2 extends AzureTransformerSpecification { + def description = "Input: 1 bad, config: PARQUET, compression, windowing: 1 minute" + def requiredAppConfig = AppConfiguration.default.copy(fileFormat = WideRowFormat.PARQUET) + def inputBatches = List(bad(count = 1)) + def countExpectations = CountExpectations(good = 0, bad = 1) +} + +class ParquetScenario3 extends AzureTransformerSpecification { + def description = "Input: 10000 good, config: PARQUET, compression, windowing: 1 minute" + def requiredAppConfig = AppConfiguration.default.copy(fileFormat = WideRowFormat.PARQUET) + def inputBatches = List(good(count = 10000)) + def countExpectations = CountExpectations(good = 10000, bad = 0) +} + +class ParquetScenario4 extends AzureTransformerSpecification { + def description = "Input: 10000 bad, config: PARQUET, compression, windowing: 1 minute" + def requiredAppConfig = AppConfiguration.default.copy(fileFormat = WideRowFormat.PARQUET) + def inputBatches = List(bad(count = 10000)) + def countExpectations = CountExpectations(good = 0, bad = 10000) +} + +class ParquetScenario5 extends AzureTransformerSpecification { + def description = """Input: mixed 5000 good and 5000 bad, config: PARQUET, compression, windowing: 1 minute""" + def requiredAppConfig = AppConfiguration.default.copy(fileFormat = WideRowFormat.PARQUET) + def inputBatches = List(good(count = 5000), bad(count = 5000)) + def countExpectations = CountExpectations(good = 5000, bad = 5000) +} + +//Changed defualt windowing to 2 minutes +class ParquetScenario6 extends AzureTransformerSpecification { + def description = """Input: mixed 5000 good and 5000 bad, config: PARQUET, compression, windowing: 2 minutes""" + def requiredAppConfig = AppConfiguration.default.copy(fileFormat = WideRowFormat.PARQUET, windowFrequencyMinutes = 2) + def inputBatches = List(good(count = 5000), good(count = 5000).delayed(2.minutes)) // force new window by delaying second input batch + def countExpectations = CountExpectations(good = 10000, bad = 0) +} + +//No compression +class ParquetScenario7 extends AzureTransformerSpecification { + def description = """Input: mixed 5000 good and 5000 bad, config: PARQUET, no compression, windowing: 1 minute""" + def requiredAppConfig = AppConfiguration.default.copy(fileFormat = WideRowFormat.PARQUET, compression = Compression.None) + def inputBatches = List(good(count = 5000), bad(count = 5000)) + def countExpectations = CountExpectations(good = 5000, bad = 5000) +} + +//Checking details of parquet output +class ParquetOutputDetailsScenario extends AzureTransformerSpecification { + + private val goodEvent = Content.TextLines( + List( + """spirit-walk web 2021-10-05 11:04:24.773579202 2021-10-05 11:04:06.799579202 unstruct a111920f-a2a3-45dc-8ae6-d8d86a383f5b datacap js-2.5.3-m1 ssc-2.2.1-pubsub beam-enrich-1.2.0-common-1.1.0 ada.blackjack@iglu.com 0:7fff:9ad9:7fff:cd64:8000:af61:eb6d a0925757-3894-4631-9e6e-e572820bde76 9462 0791c60d-3c62-49c5-87e4-42581e909a85 http://e.net/lfek Elit do sit consectetur ipsum adipiscing lorem dolor sed amet https://google.fr/morwmjx http e.net lfek https google.fr morwmjx search Google email openemail igloosforall {"schema":"iglu:com.snowplowanalytics.snowplow/contexts/jsonschema/1-0-0","data":[{"schema":"iglu:com.snowplowanalytics.snowplow/web_page/jsonschema/1-0-0","data":{"id":"15108107-7c6f-4df1-94cf-9d8d1229095c"}},{"schema":"iglu:com.snowplowanalytics.snowplow/geolocation_context/jsonschema/1-0-0","data":{"latitude":13.47832218152621,"longitude":180.0}},{"schema":"iglu:org.w3/PerformanceTiming/jsonschema/1-0-0","data":{"requestStart":100000,"chromeFirstPaint":4561,"unloadEventStart":79959,"fetchStart":1,"domainLookupStart":1,"requestEnd":1,"unloadEventEnd":100000,"loadEventStart":7718,"secureConnectionStart":1,"redirectEnd":100000,"domContentLoadedEventStart":96121,"navigationStart":12431,"proxyStart":100000,"responseStart":50588,"proxyEnd":1,"connectStart":53541,"msFirstPaint":100000,"domContentLoadedEventEnd":95815,"loadEventEnd":12163,"responseEnd":1,"connectEnd":1,"domInteractive":1,"redirectStart":97743,"domComplete":78711,"domainLookupEnd":1,"domLoading":3987}},{"schema":"iglu:com.snowplowanalytics.snowplow/client_session/jsonschema/1-0-1","data":{"userId":"0a4d7fb4-1b41-4377-b777-43c84df3b432","sessionId":"c57ab828-5186-48f6-880a-2b00d1d68d12","sessionIndex":1299327581,"previousSessionId":"10f52998-686c-4186-83e6-87d4bf044360","storageMechanism":"SQLITE"}}]} {"schema":"iglu:com.snowplowanalytics.snowplow/unstruct_event/jsonschema/1-0-0","data":{"schema":"iglu:com.snowplowanalytics.snowplow/link_click/jsonschema/1-0-1","data":{"targetUrl":"http://www.W.ru/q"}}} Mozilla/5.0 (Macintosh; Intel Mac OS X 10_10_5) AppleWebKit/601.1.56 (KHTML, like Gecko) Version/9.0 Safari/601.1.56 {"schema":"iglu:com.snowplowanalytics.snowplow/contexts/jsonschema/1-0-0","data":[{"schema":"iglu:com.mparticle.snowplow/pushregistration_event/jsonschema/1-0-0","data":{"name":"s","registrationToken":"x"}},{"schema":"iglu:com.segment/screen/jsonschema/1-0-0","data":{"name":"nn5y1uOtbmT54qhatagqg6pjpfydxdzv"}},{"schema":"iglu:com.snowplowanalytics.snowplow/consent_withdrawn/jsonschema/1-0-0","data":{"all":false}},{"schema":"iglu:com.mparticle.snowplow/session_context/jsonschema/1-0-0","data":{"id":"w"}},{"schema":"iglu:com.optimizely.optimizelyx/summary/jsonschema/1-0-0","data":{"experimentId":86409,"variationName":"z0gdkswjpyckuA","variation":94199,"visitorId":"guHo9jXgVkkqgvttdbx"}},{"schema":"iglu:com.optimizely/variation/jsonschema/1-0-0","data":{"id":"aygp70c7plcgfFkletcxePqzjsrfg","name":"b","code":"cm"}},{"schema":"iglu:com.optimizely/state/jsonschema/1-0-0","data":{"experimentId":"y","isActive":false,"variationIndex":15901,"variationId":null,"variationName":"uxhiptya1afmklYxovdeWkyym"}},{"schema":"iglu:com.optimizely/visitor/jsonschema/1-0-0","data":{"browser":"H","browserVersion":null,"device":"dhqxkb2xhcxif6kb","deviceType":"bwajz3uPbvgkolzu","mobile":true}},{"schema":"iglu:com.google.analytics/private/jsonschema/1-0-0","data":{"v":null,"s":561759,"u":"4SRvufkxnwvdvacwEedknvvtihtv7t4thdwffzbkt9nGpxhLcvliwCKpasetbbylw9hoxdajvbsci00ujotb2ntK3kvgrjqwoT7chiyvoxviqawkgdmmZe8shxiuplspCgki8kliptqnjsjpasFmdkhTzfmnlMGspowzbNawTlfegkfezEadqlmnbvv3qjtrEueqsagjbrucamlmwndnw2skrabwwT7hvreyckyvwgpchjAgzuml4rfxji7je233chSsmeutdxlbZonaFtoywafl1gyaeZl77odhhd9xretxiVndvrqgcxusmelrio6xowtkqfoyuwmeasls4DzmqesVt6igsesvxRRjyu6YqymoPpwfyf3idankobecpm5nndrhyiwc37p2oqku1yirYxqawehvsv3nlr0pzizcR9vorhdbwfbe2nqhi8wvwd","gid":"n7yse4eCgtm","r":103569}},{"schema":"iglu:com.google.analytics/cookies/jsonschema/1-0-0","data":{"__utma":"E","__utmb":"lWcbfgEoyr","__utmc":"rqqjwkixOpg","__utmv":"l","__utmz":"k4Kn","_ga":"tvjkHopok"}},{"schema":"iglu:org.ietf/http_cookie/jsonschema/1-0-0","data":{"name":"NoygypmvyarvZbsknVdbskuwoBaU3qcL","value":"jlhaofcMybVj33qrdumlnt5qoktdabaw"}},{"schema":"iglu:org.ietf/http_header/jsonschema/1-0-0","data":{"name":"7r72a6d4","value":"o5cVtuyjtorn4vfo"}},{"schema":"iglu:com.snowplowanalytics.snowplow/ua_parser_context/jsonschema/1-0-0","data":{"useragentMinor":"6","useragentFamily":"Firefox","useragentMajor":"7","osFamily":"Mac OS X","deviceFamily":"Mac"}},{"schema":"iglu:com.snowplowanalytics.snowplow/desktop_context/jsonschema/1-0-0","data":{"osType":"Linux","osVersion":"49365250426432071725495","osIs64Bit":false,"osServicePack":"c","deviceManufacturer":"176003457107290991384576784569789099","deviceModel":"203936335876967844347234142039077985","deviceProcessorCount":15}},{"schema":"iglu:com.snowplowanalytics.snowplow/consent_document/jsonschema/1-0-0","data":{"id":"pfprxpoi","version":"b","description":"c"}},{"schema":"iglu:com.snowplowanalytics.snowplow/client_session/jsonschema/1-0-1","data":{"userId":"7707e20a-7ddd-4535-b6be-b27bcd2a9557","sessionId":"ec98ef04-d521-4c1c-adb4-9a5878ac6ca9","sessionIndex":203173097,"previousSessionId":"bf76d087-4890-4434-b8fb-86a7ef5c6def","storageMechanism":"SQLITE"}},{"schema":"iglu:com.snowplowanalytics.snowplow/change_form/jsonschema/1-0-0","data":{"formId":"mxzy","elementId":"dtcfxsmqgwv","nodeName":"TEXTAREA","type":"text","value":"f"}}]} 90ea775c-1aa0-4d40-a473-0b5796b44509 2021-09-17 09:05:28.590000001 com.snowplowanalytics.snowplow link_click jsonschema 1-0-1 2d3de1febeeaf6bdcfcbbdfddd0e2b9b """ + ) + ) + + override def description = "Asserting details of a single PARQUET transformed event" + override def requiredAppConfig = AppConfiguration.default.copy(fileFormat = WideRowFormat.PARQUET) + override def inputBatches = List(InputBatch(goodEvent)) + override def countExpectations = CountExpectations(good = 1, bad = 0) + + override def customDataAssertion = Some { outputData => + outputData.good.head must beEqualTo(expectedOutputParquetDataAsJson) + } + + lazy val expectedOutputParquetDataAsJson = parser + .parse( + """ + |{ + | "app_id" : "spirit-walk", + | "platform" : "web", + | "etl_tstamp" : "2021-10-05T11:04:24.773Z", + | "collector_tstamp" : "2021-10-05T11:04:06.799Z", + | "event" : "unstruct", + | "event_id" : "a111920f-a2a3-45dc-8ae6-d8d86a383f5b", + | "name_tracker" : "datacap", + | "v_tracker" : "js-2.5.3-m1", + | "v_collector" : "ssc-2.2.1-pubsub", + | "v_etl" : "beam-enrich-1.2.0-common-1.1.0", + | "user_id" : "ada.blackjack@iglu.com", + | "user_ipaddress" : "0:7fff:9ad9:7fff:cd64:8000:af61:eb6d", + | "domain_userid" : "a0925757-3894-4631-9e6e-e572820bde76", + | "domain_sessionidx" : 9462, + | "network_userid" : "0791c60d-3c62-49c5-87e4-42581e909a85", + | "page_url" : "http://e.net/lfek", + | "page_title" : "Elit do sit consectetur ipsum adipiscing lorem dolor sed amet", + | "page_referrer" : "https://google.fr/morwmjx", + | "page_urlscheme" : "http", + | "page_urlhost" : "e.net", + | "page_urlpath" : "lfek", + | "refr_urlscheme" : "https", + | "refr_urlhost" : "google.fr", + | "refr_urlpath" : "morwmjx", + | "refr_medium" : "search", + | "refr_source" : "Google", + | "mkt_medium" : "email", + | "mkt_source" : "openemail", + | "mkt_campaign" : "igloosforall", + | "useragent" : "Mozilla/5.0 (Macintosh; Intel Mac OS X 10_10_5) AppleWebKit/601.1.56 (KHTML, like Gecko) Version/9.0 Safari/601.1.56", + | "domain_sessionid" : "90ea775c-1aa0-4d40-a473-0b5796b44509", + | "derived_tstamp" : "2021-09-17T09:05:28.590Z", + | "event_vendor" : "com.snowplowanalytics.snowplow", + | "event_name" : "link_click", + | "event_format" : "jsonschema", + | "event_version" : "1-0-1", + | "event_fingerprint" : "2d3de1febeeaf6bdcfcbbdfddd0e2b9b", + | "contexts_com_google_analytics_cookies_1" : [ + | { + | "__utma" : "E", + | "__utmb" : "lWcbfgEoyr", + | "__utmc" : "rqqjwkixOpg", + | "__utmv" : "l", + | "__utmz" : "k4Kn", + | "_ga" : "tvjkHopok" + | } + | ], + | "contexts_com_google_analytics_private_1" : [ + | { + | "gid" : "n7yse4eCgtm", + | "r" : 103569, + | "s" : 561759, + | "u" : "4SRvufkxnwvdvacwEedknvvtihtv7t4thdwffzbkt9nGpxhLcvliwCKpasetbbylw9hoxdajvbsci00ujotb2ntK3kvgrjqwoT7chiyvoxviqawkgdmmZe8shxiuplspCgki8kliptqnjsjpasFmdkhTzfmnlMGspowzbNawTlfegkfezEadqlmnbvv3qjtrEueqsagjbrucamlmwndnw2skrabwwT7hvreyckyvwgpchjAgzuml4rfxji7je233chSsmeutdxlbZonaFtoywafl1gyaeZl77odhhd9xretxiVndvrqgcxusmelrio6xowtkqfoyuwmeasls4DzmqesVt6igsesvxRRjyu6YqymoPpwfyf3idankobecpm5nndrhyiwc37p2oqku1yirYxqawehvsv3nlr0pzizcR9vorhdbwfbe2nqhi8wvwd" + | } + | ], + | "contexts_com_mparticle_snowplow_pushregistration_event_1" : [ + | { + | "name" : "s", + | "registration_token" : "x" + | } + | ], + | "contexts_com_mparticle_snowplow_session_context_1" : [ + | { + | "id" : "w" + | } + | ], + | "contexts_com_optimizely_state_1" : [ + | { + | "experiment_id" : "y", + | "is_active" : false, + | "variation_index" : 15901, + | "variation_name" : "uxhiptya1afmklYxovdeWkyym" + | } + | ], + | "contexts_com_optimizely_variation_1" : [ + | { + | "code" : "cm", + | "id" : "aygp70c7plcgfFkletcxePqzjsrfg", + | "name" : "b" + | } + | ], + | "contexts_com_optimizely_visitor_1" : [ + | { + | "browser" : "H", + | "device" : "dhqxkb2xhcxif6kb", + | "device_type" : "bwajz3uPbvgkolzu", + | "mobile" : true + | } + | ], + | "contexts_com_optimizely_optimizelyx_summary_1" : [ + | { + | "experiment_id" : 86409, + | "variation" : 94199, + | "variation_name" : "z0gdkswjpyckuA", + | "visitor_id" : "guHo9jXgVkkqgvttdbx" + | } + | ], + | "contexts_com_segment_screen_1" : [ + | { + | "name" : "nn5y1uOtbmT54qhatagqg6pjpfydxdzv" + | } + | ], + | "contexts_com_snowplowanalytics_snowplow_change_form_1" : [ + | { + | "element_id" : "dtcfxsmqgwv", + | "form_id" : "mxzy", + | "node_name" : "TEXTAREA", + | "type" : "text", + | "value" : "f" + | } + | ], + | "contexts_com_snowplowanalytics_snowplow_client_session_1" : [ + | { + | "previous_session_id" : "10f52998-686c-4186-83e6-87d4bf044360", + | "session_id" : "c57ab828-5186-48f6-880a-2b00d1d68d12", + | "session_index" : 1299327581, + | "storage_mechanism" : "SQLITE", + | "user_id" : "0a4d7fb4-1b41-4377-b777-43c84df3b432" + | }, + | { + | "previous_session_id" : "bf76d087-4890-4434-b8fb-86a7ef5c6def", + | "session_id" : "ec98ef04-d521-4c1c-adb4-9a5878ac6ca9", + | "session_index" : 203173097, + | "storage_mechanism" : "SQLITE", + | "user_id" : "7707e20a-7ddd-4535-b6be-b27bcd2a9557" + | } + | ], + | "contexts_com_snowplowanalytics_snowplow_consent_document_1" : [ + | { + | "description" : "c", + | "id" : "pfprxpoi", + | "version" : "b" + | } + | ], + | "contexts_com_snowplowanalytics_snowplow_consent_withdrawn_1" : [ + | { + | "all" : false + | } + | ], + | "contexts_com_snowplowanalytics_snowplow_desktop_context_1" : [ + | { + | "device_manufacturer" : "176003457107290991384576784569789099", + | "device_model" : "203936335876967844347234142039077985", + | "device_processor_count" : 15.0, + | "os_is64_bit" : false, + | "os_service_pack" : "c", + | "os_type" : "Linux", + | "os_version" : "49365250426432071725495" + | } + | ], + | "contexts_com_snowplowanalytics_snowplow_geolocation_context_1" : [ + | { + | "latitude" : 13.47832218152621, + | "longitude" : 180.0 + | } + | ], + | "contexts_com_snowplowanalytics_snowplow_ua_parser_context_1" : [ + | { + | "device_family" : "Mac", + | "os_family" : "Mac OS X", + | "useragent_family" : "Firefox", + | "useragent_major" : "7", + | "useragent_minor" : "6" + | } + | ], + | "contexts_com_snowplowanalytics_snowplow_web_page_1" : [ + | { + | "id" : "15108107-7c6f-4df1-94cf-9d8d1229095c" + | } + | ], + | "contexts_org_ietf_http_cookie_1" : [ + | { + | "name" : "NoygypmvyarvZbsknVdbskuwoBaU3qcL", + | "value" : "jlhaofcMybVj33qrdumlnt5qoktdabaw" + | } + | ], + | "contexts_org_ietf_http_header_1" : [ + | { + | "name" : "7r72a6d4", + | "value" : "o5cVtuyjtorn4vfo" + | } + | ], + | "contexts_org_w3_performance_timing_1" : [ + | { + | "chrome_first_paint" : 4561, + | "connect_end" : 1, + | "connect_start" : 53541, + | "dom_complete" : 78711, + | "dom_content_loaded_event_end" : 95815, + | "dom_content_loaded_event_start" : 96121, + | "dom_interactive" : 1, + | "dom_loading" : 3987, + | "domain_lookup_end" : 1, + | "domain_lookup_start" : 1, + | "fetch_start" : 1, + | "load_event_end" : 12163, + | "load_event_start" : 7718, + | "ms_first_paint" : 100000, + | "navigation_start" : 12431, + | "proxy_end" : 1, + | "proxy_start" : 100000, + | "redirect_end" : 100000, + | "redirect_start" : 97743, + | "request_end" : 1, + | "request_start" : 100000, + | "response_end" : 1, + | "response_start" : 50588, + | "secure_connection_start" : 1, + | "unload_event_end" : 100000, + | "unload_event_start" : 79959 + | } + | ], + | "unstruct_event_com_snowplowanalytics_snowplow_link_click_1" : { + | "target_url" : "http://www.W.ru/q" + | } + |} + |""".stripMargin + ) + .right + .get +} diff --git a/project/Dependencies.scala b/project/Dependencies.scala index dc2ad5f0d..921d141e2 100644 --- a/project/Dependencies.scala +++ b/project/Dependencies.scala @@ -80,7 +80,8 @@ object Dependencies { val resolutionRepos = Seq( // Redshift native driver - ("redshift" at "http://redshift-maven-repository.s3-website-us-east-1.amazonaws.com/release").withAllowInsecureProtocol(true) + ("redshift" at "http://redshift-maven-repository.s3-website-us-east-1.amazonaws.com/release").withAllowInsecureProtocol(true), + ("Snowplow Analytics Maven repo" at "http://maven.snplow.com/releases/").withAllowInsecureProtocol(true) ) // Scala (Common) @@ -125,6 +126,7 @@ object Dependencies { val http4sClient = "org.http4s" %% "http4s-blaze-client" % V.http4sBlaze val scalaTracker = "com.snowplowanalytics" %% "snowplow-scala-tracker-core" % V.scalaTracker val scalaTrackerEmit = "com.snowplowanalytics" %% "snowplow-scala-tracker-emitter-http4s" % V.scalaTracker + val eventGenerator = "com.snowplowanalytics" %% "snowplow-event-generator-core" % "0.4.0" % Test // Scala (Shredder) val eventsManifest = "com.snowplowanalytics" %% "snowplow-events-manifest" % V.eventsManifest @@ -337,7 +339,8 @@ object Dependencies { ) val transformerKafkaDependencies = Seq( - hadoopAzure + hadoopAzure, + eventGenerator ) val commonStreamTransformerExclusions = From bdd6cd5af06c7758848051fd0b2560892d8d8497 Mon Sep 17 00:00:00 2001 From: spenes Date: Thu, 6 Jul 2023 11:02:25 +0300 Subject: [PATCH 14/14] Snowflake Loader: add semi-automatic test scenerios using cloud resources --- .../integrationtestutils}/InputBatch.scala | 29 ++++- .../integrationtestutils/ItUtils.scala | 104 ++++++++++++++++ .../experimental/AzureTestResources.scala | 76 ++++++++++++ .../experimental/CloudResources.scala | 27 ++++ .../experimental/LoaderSpecification.scala | 116 ++++++++++++++++++ .../experimental/StorageTargetProvider.scala | 22 ++++ .../rdbloader/experimental/TestDAO.scala | 35 ++++++ .../it/AzureSnowflakeLoaderSpecs.scala | 66 ++++++++++ .../snowflake/it/SnowflakeTestResources.scala | 92 ++++++++++++++ .../TransformerSpecification.scala | 88 +++---------- .../scenarios/BadDetailsScenario.scala | 11 +- .../kafka/experimental/scenarios/json.scala | 12 +- .../experimental/scenarios/parquet.scala | 13 +- project/Dependencies.scala | 6 +- 14 files changed, 596 insertions(+), 101 deletions(-) rename modules/{transformer-kafka/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/kafka/experimental => common/src/test/scala/com.snowplowanalytics.snowplow.rdbloader.common/integrationtestutils}/InputBatch.scala (65%) create mode 100644 modules/common/src/test/scala/com.snowplowanalytics.snowplow.rdbloader.common/integrationtestutils/ItUtils.scala create mode 100644 modules/loader/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/experimental/AzureTestResources.scala create mode 100644 modules/loader/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/experimental/CloudResources.scala create mode 100644 modules/loader/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/experimental/LoaderSpecification.scala create mode 100644 modules/loader/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/experimental/StorageTargetProvider.scala create mode 100644 modules/loader/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/experimental/TestDAO.scala create mode 100644 modules/snowflake-loader/src/test/scala/com/snowplowanalytics/snowplow/loader/snowflake/it/AzureSnowflakeLoaderSpecs.scala create mode 100644 modules/snowflake-loader/src/test/scala/com/snowplowanalytics/snowplow/loader/snowflake/it/SnowflakeTestResources.scala diff --git a/modules/transformer-kafka/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/kafka/experimental/InputBatch.scala b/modules/common/src/test/scala/com.snowplowanalytics.snowplow.rdbloader.common/integrationtestutils/InputBatch.scala similarity index 65% rename from modules/transformer-kafka/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/kafka/experimental/InputBatch.scala rename to modules/common/src/test/scala/com.snowplowanalytics.snowplow.rdbloader.common/integrationtestutils/InputBatch.scala index 3d1c6cfa2..cc01b38dc 100644 --- a/modules/transformer-kafka/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/kafka/experimental/InputBatch.scala +++ b/modules/common/src/test/scala/com.snowplowanalytics.snowplow.rdbloader.common/integrationtestutils/InputBatch.scala @@ -1,14 +1,31 @@ -package com.snowplowanalytics.snowplow.rdbloader.transformer.stream.kafka.experimental -import com.snowplowanalytics.snowplow.analytics.scalasdk.Event -import com.snowplowanalytics.snowplow.eventgen.enrich.{SdkEvent => EventGenerator} -import com.snowplowanalytics.snowplow.eventgen.protocol.event.{EventFrequencies, UnstructEventFrequencies} -import org.scalacheck.Gen -import org.scalacheck.rng.Seed +/* + * Copyright (c) 2012-2023 Snowplow Analytics Ltd. All rights reserved. + * + * This program is licensed to you under the Apache License Version 2.0, + * and you may not use this file except in compliance with the Apache License Version 2.0. + * You may obtain a copy of the Apache License Version 2.0 at + * http://www.apache.org/licenses/LICENSE-2.0. + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the Apache License Version 2.0 is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the Apache License Version 2.0 for the specific language governing permissions and + * limitations there under. + */ +package com.snowplowanalytics.snowplow.rdbloader.common.integrationtestutils import java.time.Instant + import scala.concurrent.duration.{DurationInt, FiniteDuration} import scala.util.Random +import org.scalacheck.Gen +import org.scalacheck.rng.Seed + +import com.snowplowanalytics.snowplow.analytics.scalasdk.Event +import com.snowplowanalytics.snowplow.eventgen.enrich.{SdkEvent => EventGenerator} +import com.snowplowanalytics.snowplow.eventgen.protocol.event.{EventFrequencies, UnstructEventFrequencies} + final case class InputBatch(content: InputBatch.Content, delay: FiniteDuration = 0.minutes) { def delayed(value: FiniteDuration) = this.copy(delay = value) } diff --git a/modules/common/src/test/scala/com.snowplowanalytics.snowplow.rdbloader.common/integrationtestutils/ItUtils.scala b/modules/common/src/test/scala/com.snowplowanalytics.snowplow.rdbloader.common/integrationtestutils/ItUtils.scala new file mode 100644 index 000000000..4af66ca58 --- /dev/null +++ b/modules/common/src/test/scala/com.snowplowanalytics.snowplow.rdbloader.common/integrationtestutils/ItUtils.scala @@ -0,0 +1,104 @@ +/* + * Copyright (c) 2012-2023 Snowplow Analytics Ltd. All rights reserved. + * + * This program is licensed to you under the Apache License Version 2.0, + * and you may not use this file except in compliance with the Apache License Version 2.0. + * You may obtain a copy of the Apache License Version 2.0 at + * http://www.apache.org/licenses/LICENSE-2.0. + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the Apache License Version 2.0 is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the Apache License Version 2.0 for the specific language governing permissions and + * limitations there under. + */ +package com.snowplowanalytics.snowplow.rdbloader.common.integrationtestutils + +import scala.concurrent.duration.DurationInt + +import cats.implicits._ +import cats.effect.IO + +import retry._ + +import fs2.Stream +import fs2.concurrent.{Signal, SignallingRef} + +import com.snowplowanalytics.snowplow.rdbloader.common.LoaderMessage +import com.snowplowanalytics.snowplow.rdbloader.common.cloud.Queue + +object ItUtils { + + private val timeout = 15.minutes + + def produceInput(batches: List[InputBatch], producer: Queue.Producer[IO]): Stream[IO, Unit] = + Stream.eval { + IO.sleep(10.seconds) *> batches.traverse_ { batch => + IO.sleep(batch.delay) *> InputBatch + .asTextLines(batch.content) + .parTraverse_(producer.send) + } + } + + def consumeAllIncomingWindows[A <: GetShreddingComplete]( + queueConsumer: Queue.Consumer[IO], + countExpectations: CountExpectations, + windowsAccumulator: SignallingRef[IO, WindowsAccumulator[A]], + getWindowOutput: LoaderMessage.ShreddingComplete => IO[A] + ): Stream[IO, Unit] = + queueConsumer.read + .map(_.content) + .map(parseShreddingCompleteMessage) + .evalMap(getWindowOutput(_)) + .evalMap { windowOutput => + windowsAccumulator.update(_.addWindow(windowOutput)) + } + .interruptWhen(allEventsProcessed(windowsAccumulator, countExpectations)) + .interruptAfter(timeout) + + private def allEventsProcessed[A <: GetShreddingComplete]( + windowsAccumulator: SignallingRef[IO, WindowsAccumulator[A]], + countExpectations: CountExpectations + ): Signal[IO, Boolean] = + windowsAccumulator + .map(_.getTotalNumberOfEvents >= countExpectations.total) + + def parseShreddingCompleteMessage(message: String): LoaderMessage.ShreddingComplete = + LoaderMessage + .fromString(message) + .right + .get + .asInstanceOf[LoaderMessage.ShreddingComplete] + + def retryUntilNonEmpty[A](io: IO[List[A]]): IO[List[A]] = + retryingOnFailures[List[A]]( + policy = RetryPolicies.capDelay[IO](15.minutes, RetryPolicies.constantDelay[IO](30.seconds)), + wasSuccessful = items => IO.delay(items.nonEmpty), + onFailure = (r, d) => IO.delay(println(s"$r - $d")) + )(io) + + final case class CountExpectations(good: Int, bad: Int) { + def total = good + bad + } + + final case class WindowsAccumulator[A <: GetShreddingComplete](value: List[A]) { + def addWindow(window: A): WindowsAccumulator[A] = + WindowsAccumulator(value :+ window) + + def getTotalNumberOfEvents: Long = + value.map { window => + val good = window.shredding_complete.count.map(_.good).getOrElse(0L) + val bad = window.shredding_complete.count.flatMap(_.bad).getOrElse(0L) + good + bad + }.sum + } + + trait GetShreddingComplete { + def shredding_complete: LoaderMessage.ShreddingComplete + } + + implicit class ManifestItemListUtils(items: List[LoaderMessage.ManifestItem]) { + def totalGood: Long = + items.foldLeft(0L)((acc, i) => acc + i.count.map(_.good).getOrElse(0L)) + } +} diff --git a/modules/loader/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/experimental/AzureTestResources.scala b/modules/loader/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/experimental/AzureTestResources.scala new file mode 100644 index 000000000..37666e02b --- /dev/null +++ b/modules/loader/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/experimental/AzureTestResources.scala @@ -0,0 +1,76 @@ +/* + * Copyright (c) 2012-2023 Snowplow Analytics Ltd. All rights reserved. + * + * This program is licensed to you under the Apache License Version 2.0, + * and you may not use this file except in compliance with the Apache License Version 2.0. + * You may obtain a copy of the Apache License Version 2.0 at + * http://www.apache.org/licenses/LICENSE-2.0. + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the Apache License Version 2.0 is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the Apache License Version 2.0 for the specific language governing permissions and + * limitations there under. + */ +package com.snowplowanalytics.snowplow.rdbloader.experimental + +import java.util.UUID + +import cats.effect.{IO, Resource} + +import com.snowplowanalytics.snowplow.rdbloader.azure.{AzureKeyVault, KafkaConsumer, KafkaProducer} +import com.snowplowanalytics.snowplow.rdbloader.common.cloud.{Queue, SecretStore} + +trait AzureTestResources extends CloudResources { + val eventHubsUrlEnv = "TEST_LOADER_EVENTHUBS_URL" + val inputHubNameEnv = "TEST_LOADER_INPUT_HUB_NAME" + val outputHubNameEnv = "TEST_LOADER_OUTPUT_HUB_NAME" + val inputHubKeyEnv = "TEST_LOADER_INPUT_HUB_KEY" + val outputHubKeyEnv = "TEST_LOADER_OUTPUT_HUB_KEY" + val azureKeyVaultNameEnv = "TEST_LOADER_AZURE_KEY_VAULT_NAME" + + def createConsumer: Resource[IO, Queue.Consumer[IO]] = + KafkaConsumer + .consumer[IO]( + bootstrapServers = System.getenv(eventHubsUrlEnv), + topicName = System.getenv(outputHubNameEnv), + consumerConf = Map( + "security.protocol" -> "SASL_SSL", + "sasl.mechanism" -> "PLAIN", + "sasl.jaas.config" -> + s"""org.apache.kafka.common.security.plain.PlainLoginModule required username=\"$$ConnectionString\" password=\"${System.getenv( + outputHubKeyEnv + )}\";""", + "group.id" -> s"test-consumer-${UUID.randomUUID()}", + "enable.auto.commit" -> "true", + "auto.offset.reset" -> "latest" + ) + ) + + def createProducer: Resource[IO, Queue.Producer[IO]] = + KafkaProducer + .producer[IO]( + bootstrapServers = System.getenv(eventHubsUrlEnv), + topicName = System.getenv(inputHubNameEnv), + producerConf = Map( + "security.protocol" -> "SASL_SSL", + "sasl.mechanism" -> "PLAIN", + "sasl.jaas.config" -> + s"""org.apache.kafka.common.security.plain.PlainLoginModule required username=\"$$ConnectionString\" password=\"${System.getenv( + inputHubKeyEnv + )}\";""" + ) + ) + + def createSecretStore: Resource[IO, SecretStore[IO]] = + AzureKeyVault.create[IO](Some(System.getenv(azureKeyVaultNameEnv))) + + override def getCloudResourcesEnvVars: List[String] = List( + eventHubsUrlEnv, + inputHubNameEnv, + outputHubNameEnv, + inputHubKeyEnv, + outputHubKeyEnv, + azureKeyVaultNameEnv + ) +} diff --git a/modules/loader/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/experimental/CloudResources.scala b/modules/loader/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/experimental/CloudResources.scala new file mode 100644 index 000000000..ad841ca33 --- /dev/null +++ b/modules/loader/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/experimental/CloudResources.scala @@ -0,0 +1,27 @@ +/* + * Copyright (c) 2012-2023 Snowplow Analytics Ltd. All rights reserved. + * + * This program is licensed to you under the Apache License Version 2.0, + * and you may not use this file except in compliance with the Apache License Version 2.0. + * You may obtain a copy of the Apache License Version 2.0 at + * http://www.apache.org/licenses/LICENSE-2.0. + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the Apache License Version 2.0 is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the Apache License Version 2.0 for the specific language governing permissions and + * limitations there under. + */ +package com.snowplowanalytics.snowplow.rdbloader.experimental + +import cats.effect.{IO, Resource} + +import com.snowplowanalytics.snowplow.rdbloader.common.cloud.{Queue, SecretStore} + +trait CloudResources { + + def createConsumer: Resource[IO, Queue.Consumer[IO]] + def createProducer: Resource[IO, Queue.Producer[IO]] + def createSecretStore: Resource[IO, SecretStore[IO]] + def getCloudResourcesEnvVars: List[String] +} diff --git a/modules/loader/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/experimental/LoaderSpecification.scala b/modules/loader/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/experimental/LoaderSpecification.scala new file mode 100644 index 000000000..426a345dc --- /dev/null +++ b/modules/loader/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/experimental/LoaderSpecification.scala @@ -0,0 +1,116 @@ +/* + * Copyright (c) 2012-2023 Snowplow Analytics Ltd. All rights reserved. + * + * This program is licensed to you under the Apache License Version 2.0, + * and you may not use this file except in compliance with the Apache License Version 2.0. + * You may obtain a copy of the Apache License Version 2.0 at + * http://www.apache.org/licenses/LICENSE-2.0. + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the Apache License Version 2.0 is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the Apache License Version 2.0 for the specific language governing permissions and + * limitations there under. + */ +package com.snowplowanalytics.snowplow.rdbloader.experimental + +import scala.concurrent.duration.DurationInt + +import doobie.ConnectionIO + +import cats.effect.{IO, Resource} +import cats.effect.std.Dispatcher +import cats.effect.unsafe.implicits.global + +import fs2.concurrent.SignallingRef + +import org.http4s.blaze.client.BlazeClientBuilder + +import com.snowplowanalytics.snowplow.rdbloader.config.{Config, StorageTarget} +import com.snowplowanalytics.snowplow.rdbloader.dsl.metrics.Metrics +import com.snowplowanalytics.snowplow.rdbloader.dsl.{Logging, Monitoring, Transaction} +import com.snowplowanalytics.snowplow.rdbloader.common.LoaderMessage +import com.snowplowanalytics.snowplow.rdbloader.common.cloud.{Queue, SecretStore} +import com.snowplowanalytics.snowplow.rdbloader.common.integrationtestutils.ItUtils._ +import com.snowplowanalytics.snowplow.rdbloader.common.integrationtestutils.InputBatch + +import org.specs2.mutable.Specification + +abstract class LoaderSpecification extends Specification with TestDAO.Provider with StorageTargetProvider with AzureTestResources { + skipAllIf(anyEnvironmentVariableMissing()) + import LoaderSpecification._ + + def run[A]( + inputBatches: List[InputBatch], + countExpectations: CountExpectations, + dbActions: TestDAO => IO[A] + ): IO[(WindowsAccumulator[WindowOutput], A)] = + createResources + .use { resources => + for { + _ <- resources.testDAO.cleanDb + windowsAccumulator <- SignallingRef.of[IO, WindowsAccumulator[WindowOutput]](WindowsAccumulator(List.empty)) + consuming = consumeAllIncomingWindows[WindowOutput]( + resources.queueConsumer, + countExpectations, + windowsAccumulator, + getWindowOutput = sc => IO.pure(WindowOutput(sc)) + ) + producing = produceInput(inputBatches, resources.producer) + _ <- consuming.concurrently(producing).compile.drain + collectedWindows <- windowsAccumulator.get + dbActionResult <- dbActions(resources.testDAO) + } yield (collectedWindows, dbActionResult) + } + + def createResources: Resource[IO, TestResources] = + for { + consumer <- createConsumer + producer <- createProducer + implicit0(secretStore: SecretStore[IO]) <- createSecretStore + transaction <- createDbTransaction + testDAO = createDAO(transaction) + } yield TestResources(queueConsumer = consumer, producer = producer, testDAO = testDAO) + + def createDbTransaction(implicit secretStore: SecretStore[IO]): Resource[IO, Transaction[IO, ConnectionIO]] = { + val storage: StorageTarget = createStorageTarget + val timeouts: Config.Timeouts = Config.Timeouts( + loading = 15.minutes, + nonLoading = 15.minutes, + sqsVisibility = 15.minutes, + rollbackCommit = 15.minutes, + connectionIsValid = 15.minutes + ) + val readyCheck: Config.Retries = Config.Retries( + strategy = Config.Strategy.Constant, + attempts = Some(3), + backoff = 30.seconds, + cumulativeBound = None + ) + for { + implicit0(dispatcher: Dispatcher[IO]) <- Dispatcher.parallel[IO] + httpClient <- BlazeClientBuilder[IO].withExecutionContext(global.compute).resource + implicit0(logging: Logging[IO]) = Logging.loggingInterpreter[IO](List()) + periodicMetrics <- Resource.eval(Metrics.PeriodicMetrics.init[IO](List.empty, 1.minutes)) + implicit0(monitoring: Monitoring[IO]) = + Monitoring.monitoringInterpreter[IO](None, None, List.empty, None, httpClient, periodicMetrics) + transaction <- Transaction.interpreter[IO](storage, timeouts, readyCheck) + } yield transaction + } + + def anyEnvironmentVariableMissing(): Boolean = + (getCloudResourcesEnvVars ::: getStorageTargetEnvVars).exists(varName => System.getenv(varName) == null) +} + +object LoaderSpecification { + + final case class TestResources( + queueConsumer: Queue.Consumer[IO], + producer: Queue.Producer[IO], + testDAO: TestDAO + ) + + final case class WindowOutput( + shredding_complete: LoaderMessage.ShreddingComplete + ) extends GetShreddingComplete +} diff --git a/modules/loader/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/experimental/StorageTargetProvider.scala b/modules/loader/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/experimental/StorageTargetProvider.scala new file mode 100644 index 000000000..c11c64c8a --- /dev/null +++ b/modules/loader/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/experimental/StorageTargetProvider.scala @@ -0,0 +1,22 @@ +/* + * Copyright (c) 2012-2023 Snowplow Analytics Ltd. All rights reserved. + * + * This program is licensed to you under the Apache License Version 2.0, + * and you may not use this file except in compliance with the Apache License Version 2.0. + * You may obtain a copy of the Apache License Version 2.0 at + * http://www.apache.org/licenses/LICENSE-2.0. + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the Apache License Version 2.0 is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the Apache License Version 2.0 for the specific language governing permissions and + * limitations there under. + */ +package com.snowplowanalytics.snowplow.rdbloader.experimental + +import com.snowplowanalytics.snowplow.rdbloader.config.StorageTarget + +trait StorageTargetProvider { + def createStorageTarget: StorageTarget + def getStorageTargetEnvVars: List[String] +} diff --git a/modules/loader/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/experimental/TestDAO.scala b/modules/loader/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/experimental/TestDAO.scala new file mode 100644 index 000000000..85cdff147 --- /dev/null +++ b/modules/loader/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/experimental/TestDAO.scala @@ -0,0 +1,35 @@ +/* + * Copyright (c) 2012-2023 Snowplow Analytics Ltd. All rights reserved. + * + * This program is licensed to you under the Apache License Version 2.0, + * and you may not use this file except in compliance with the Apache License Version 2.0. + * You may obtain a copy of the Apache License Version 2.0 at + * http://www.apache.org/licenses/LICENSE-2.0. + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the Apache License Version 2.0 is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the Apache License Version 2.0 for the specific language governing permissions and + * limitations there under. + */ +package com.snowplowanalytics.snowplow.rdbloader.experimental + +import cats.effect.IO + +import doobie.ConnectionIO + +import com.snowplowanalytics.snowplow.rdbloader.common.LoaderMessage +import com.snowplowanalytics.snowplow.rdbloader.dsl.Transaction + +trait TestDAO { + def cleanDb: IO[Unit] + def queryManifest: IO[List[LoaderMessage.ManifestItem]] + def queryEventIds: IO[List[String]] +} + +object TestDAO { + + trait Provider { + def createDAO(transaction: Transaction[IO, ConnectionIO]): TestDAO + } +} diff --git a/modules/snowflake-loader/src/test/scala/com/snowplowanalytics/snowplow/loader/snowflake/it/AzureSnowflakeLoaderSpecs.scala b/modules/snowflake-loader/src/test/scala/com/snowplowanalytics/snowplow/loader/snowflake/it/AzureSnowflakeLoaderSpecs.scala new file mode 100644 index 000000000..d020a09fc --- /dev/null +++ b/modules/snowflake-loader/src/test/scala/com/snowplowanalytics/snowplow/loader/snowflake/it/AzureSnowflakeLoaderSpecs.scala @@ -0,0 +1,66 @@ +/* + * Copyright (c) 2012-2023 Snowplow Analytics Ltd. All rights reserved. + * + * This program is licensed to you under the Apache License Version 2.0, + * and you may not use this file except in compliance with the Apache License Version 2.0. + * You may obtain a copy of the Apache License Version 2.0 at + * http://www.apache.org/licenses/LICENSE-2.0. + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the Apache License Version 2.0 is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the Apache License Version 2.0 for the specific language governing permissions and + * limitations there under. + */ +package com.snowplowanalytics.snowplow.loader.snowflake.it + +import cats.effect.unsafe.implicits.global + +import com.snowplowanalytics.snowplow.rdbloader.common.LoaderMessage +import com.snowplowanalytics.snowplow.rdbloader.common.integrationtestutils.InputBatch +import com.snowplowanalytics.snowplow.rdbloader.common.integrationtestutils.ItUtils._ +import com.snowplowanalytics.snowplow.rdbloader.experimental.{AzureTestResources, LoaderSpecification} + +import org.specs2.matcher.MatchResult + +class AzureSnowflakeLoaderSpecs extends LoaderSpecification with SnowflakeTestResources with AzureTestResources { + sequential + + "Scenario 1" in { + val goodEvent = InputBatch.Content.TextLines( + List( + """snowplowweb web 2014-06-01 14:04:11.639 2014-05-29 18:16:35.000 2014-05-29 18:16:35.967 unstruct 2b1b25a4-c0df-4859-8201-cf21492ad61b 114221 clojure js-2.0.0-M2 clj-0.6.0-tom-0.0.4 hadoop-0.5.0-common-0.4.0 68.42.204.218 1242058182 58df65c46e1ac937 11 437ad25b-2006-455e-b5d8-d664b74df8f3 US MI Holland 49423 42.742294 -86.0661 http://snowplowanalytics.com/blog/ https://www.google.com/ http snowplowanalytics.com 80 /blog/ https www.google.com 80 / search Google {"schema":"iglu:com.snowplowanalytics.snowplow/contexts/jsonschema/1-0-0","data":[{"schema":"iglu:org.schema/WebPage/jsonschema/1-0-0","data":{"datePublished":"2014-07-23T00:00:00Z","author":"Jonathan Almeida","inLanguage":"en-US","genre":"blog","breadcrumb":["blog","releases"],"keywords":["snowplow","analytics","java","jvm","tracker"]}}]} {"schema":"iglu:com.snowplowanalytics.snowplow/unstruct_event/jsonschema/1-0-0","data":{"schema":"iglu:com.snowplowanalytics.snowplow/link_click/jsonschema/1-0-0","data":{"targetUrl":"http://snowplowanalytics.com/blog/page2","elementClasses":["next"]}}} Mozilla/5.0 (Macintosh; Intel Mac OS X 10_9_3) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/35.0.1916.114 Safari/537.36 Chrome Chrome Browser WEBKIT en-US 1 1 1 0 1 0 0 0 1 1 24 1241 806 Mac OS Mac OS Apple Inc. America/New_York Computer 0 1440 900 UTF-8 """ + ) + ) + val inputBatches = List(InputBatch(goodEvent)) + val countExpectations = CountExpectations(good = 1, bad = 0) + eventCountCheck(inputBatches, countExpectations) + } + + "Scenario 2" in { + val countExpectations = CountExpectations(good = 100, bad = 0) + val inputBatches = List(InputBatch.good(countExpectations.good)) + eventCountCheck(inputBatches, countExpectations) + } + + def eventCountCheck(inputBatches: List[InputBatch], countExpectations: CountExpectations): MatchResult[Any] = { + case class DbActionResult(manifestItems: List[LoaderMessage.ManifestItem], eventIds: List[String]) + val res = for { + (windowAcc, dbActionResult) <- run[DbActionResult]( + inputBatches, + countExpectations, + dbActions = testDAO => + for { + manifestItems <- retryUntilNonEmpty(testDAO.queryManifest) + eventIds <- testDAO.queryEventIds + } yield DbActionResult(manifestItems, eventIds) + ) + } yield { + windowAcc.value.size must beEqualTo(dbActionResult.manifestItems.size) + dbActionResult.eventIds.size must beEqualTo(countExpectations.good) + dbActionResult.manifestItems.totalGood must beEqualTo(countExpectations.good) + } + res.unsafeRunSync() + } + +} diff --git a/modules/snowflake-loader/src/test/scala/com/snowplowanalytics/snowplow/loader/snowflake/it/SnowflakeTestResources.scala b/modules/snowflake-loader/src/test/scala/com/snowplowanalytics/snowplow/loader/snowflake/it/SnowflakeTestResources.scala new file mode 100644 index 000000000..baefb37fd --- /dev/null +++ b/modules/snowflake-loader/src/test/scala/com/snowplowanalytics/snowplow/loader/snowflake/it/SnowflakeTestResources.scala @@ -0,0 +1,92 @@ +/* + * Copyright (c) 2012-2023 Snowplow Analytics Ltd. All rights reserved. + * + * This program is licensed to you under the Apache License Version 2.0, + * and you may not use this file except in compliance with the Apache License Version 2.0. + * You may obtain a copy of the Apache License Version 2.0 at + * http://www.apache.org/licenses/LICENSE-2.0. + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the Apache License Version 2.0 is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the Apache License Version 2.0 for the specific language governing permissions and + * limitations there under. + */ +package com.snowplowanalytics.snowplow.loader.snowflake.it + +import scala.concurrent.duration.DurationInt + +import cats.effect.IO + +import doobie.ConnectionIO +import doobie.implicits._ + +import com.snowplowanalytics.snowplow.rdbloader.common.LoaderMessage +import com.snowplowanalytics.snowplow.rdbloader.config.StorageTarget +import com.snowplowanalytics.snowplow.rdbloader.config.StorageTarget.LoadAuthMethod +import com.snowplowanalytics.snowplow.rdbloader.dsl.Transaction +import com.snowplowanalytics.snowplow.rdbloader.experimental.{StorageTargetProvider, TestDAO} + +trait SnowflakeTestResources extends TestDAO.Provider with StorageTargetProvider { + + val secretStoreParameterNameEnv = "TEST_LOADER_SECRET_STORE_PARAMETER" + val snowflakeRegionEnv = "TEST_LOADER_SNOWFLAKE_REGION" + val snowflakeUsernameEnv = "TEST_LOADER_SNOWFLAKE_USERNAME" + val snowflakeRoleEnv = "TEST_LOADER_SNOWFLAKE_ROLE" + val snowflakeAccountEnv = "TEST_LOADER_SNOWFLAKE_ACCOUNT" + val snowflakeWarehouseEnv = "TEST_LOADER_SNOWFLAKE_WAREHOUSE" + val snowflakeDatabaseEnv = "TEST_LOADER_SNOWFLAKE_DATABASE" + val snowflakeSchemaEnv = "TEST_LOADER_SNOWFLAKE_SCHEMA" + + override def createDAO(transaction: Transaction[IO, ConnectionIO]): TestDAO = new TestDAO { + override def cleanDb: IO[Unit] = + for { + _ <- transaction.run(sql"delete from atomic.events".update.run) + _ <- transaction.run(sql"delete from atomic.manifest".update.run) + } yield () + + override def queryManifest: IO[List[LoaderMessage.ManifestItem]] = + transaction.run( + sql"""select base, types, shredding_started, shredding_completed, + min_collector_tstamp, max_collector_tstamp, + compression, processor_artifact, processor_version, count_good + FROM atomic.manifest""".query[LoaderMessage.ManifestItem].stream.compile.toList + ) + + override def queryEventIds: IO[List[String]] = + for { + res <- transaction.run(sql"select event_id from atomic.events".query[String].stream.compile.toList) + } yield res + } + + override def createStorageTarget: StorageTarget = + StorageTarget.Snowflake( + snowflakeRegion = Some(System.getenv(snowflakeRegionEnv)), + username = System.getenv(snowflakeUsernameEnv), + role = Some(System.getenv(snowflakeRoleEnv)), + password = StorageTarget.PasswordConfig.EncryptedKey(StorageTarget.EncryptedConfig(System.getenv(secretStoreParameterNameEnv))), + account = Some(System.getenv(snowflakeAccountEnv)), + warehouse = System.getenv(snowflakeWarehouseEnv), + database = System.getenv(snowflakeDatabaseEnv), + schema = System.getenv(snowflakeSchemaEnv), + transformedStage = None, + appName = "loader-test", + folderMonitoringStage = None, + jdbcHost = None, + loadAuthMethod = LoadAuthMethod.TempCreds.AzureTempCreds(15.minutes), + readyCheck = StorageTarget.Snowflake.ResumeWarehouse + ) + + // For some reason, using variables above doesn't work here. + // Therefore, we've used directly strings in here. + override def getStorageTargetEnvVars: List[String] = List( + "TEST_LOADER_SECRET_STORE_PARAMETER", + "TEST_LOADER_SNOWFLAKE_REGION", + "TEST_LOADER_SNOWFLAKE_USERNAME", + "TEST_LOADER_SNOWFLAKE_ROLE", + "TEST_LOADER_SNOWFLAKE_ACCOUNT", + "TEST_LOADER_SNOWFLAKE_WAREHOUSE", + "TEST_LOADER_SNOWFLAKE_DATABASE", + "TEST_LOADER_SNOWFLAKE_SCHEMA" + ) +} diff --git a/modules/transformer-kafka/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/kafka/experimental/TransformerSpecification.scala b/modules/transformer-kafka/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/kafka/experimental/TransformerSpecification.scala index 5810c7029..3fc4a0c0c 100644 --- a/modules/transformer-kafka/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/kafka/experimental/TransformerSpecification.scala +++ b/modules/transformer-kafka/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/kafka/experimental/TransformerSpecification.scala @@ -2,18 +2,18 @@ package com.snowplowanalytics.snowplow.rdbloader.transformer.stream.kafka.experi import cats.effect.IO import cats.effect.unsafe.implicits.global -import cats.implicits._ import com.snowplowanalytics.snowplow.rdbloader.common.LoaderMessage import com.snowplowanalytics.snowplow.rdbloader.common.LoaderMessage.TypesInfo import com.snowplowanalytics.snowplow.rdbloader.common.LoaderMessage.TypesInfo.WideRow.WideRowFormat import com.snowplowanalytics.snowplow.rdbloader.common.cloud.BlobStorage import com.snowplowanalytics.snowplow.rdbloader.common.cloud.BlobStorage.{BlobObject, Folder} import com.snowplowanalytics.snowplow.rdbloader.common.config.TransformerConfig.Compression +import com.snowplowanalytics.snowplow.rdbloader.common.integrationtestutils.InputBatch +import com.snowplowanalytics.snowplow.rdbloader.common.integrationtestutils.ItUtils._ import com.snowplowanalytics.snowplow.rdbloader.transformer.stream.kafka.experimental.TransformerSpecification._ import fs2.Stream import fs2.compression.{Compression => FS2Compression} -import fs2.concurrent.Signal.SignalOps -import fs2.concurrent.{Signal, SignallingRef} +import fs2.concurrent.SignallingRef import io.circe.Json import org.specs2.matcher.MatchResult import org.specs2.mutable.Specification @@ -21,7 +21,6 @@ import org.specs2.mutable.Specification import java.time.LocalDateTime import java.time.format.DateTimeFormatter import java.time.temporal.ChronoUnit -import scala.concurrent.duration.DurationInt // format: off /** @@ -51,8 +50,6 @@ import scala.concurrent.duration.DurationInt abstract class TransformerSpecification extends Specification with AppDependencies.Provider { - private val timeout = 15.minutes - protected def description: String protected def inputBatches: List[InputBatch] protected def countExpectations: CountExpectations @@ -67,9 +64,14 @@ abstract class TransformerSpecification extends Specification with AppDependenci createDependencies() .use { dependencies => for { - windowsAccumulator <- SignallingRef.of[IO, WindowsAccumulator](WindowsAccumulator(List.empty)) - consuming = consumeAllIncomingWindows(dependencies, countExpectations, windowsAccumulator) - producing = produceInput(inputBatches, dependencies) + windowsAccumulator <- SignallingRef.of[IO, WindowsAccumulator[WindowOutput]](WindowsAccumulator(List.empty)) + consuming = consumeAllIncomingWindows[WindowOutput]( + dependencies.queueConsumer, + countExpectations, + windowsAccumulator, + getWindowOutput = readWindowOutput(dependencies.blobClient) + ) + producing = produceInput(inputBatches, dependencies.producer) _ <- consuming.concurrently(producing).compile.drain collectedWindows <- windowsAccumulator.get } yield { @@ -82,35 +84,11 @@ abstract class TransformerSpecification extends Specification with AppDependenci } } - private def produceInput(batches: List[InputBatch], dependencies: AppDependencies): Stream[IO, Unit] = - Stream.eval { - IO.sleep(10.seconds) *> batches.traverse_ { batch => - IO.sleep(batch.delay) *> InputBatch - .asTextLines(batch.content) - .parTraverse_(dependencies.producer.send) - } - } - - private def consumeAllIncomingWindows( - dependencies: AppDependencies, - countExpectations: CountExpectations, - windowsAccumulator: SignallingRef[IO, WindowsAccumulator] - ): Stream[IO, Unit] = - dependencies.queueConsumer.read - .map(_.content) - .map(parseShreddingCompleteMessage) - .evalMap(readWindowOutput(dependencies.blobClient)) - .evalMap { windowOutput => - windowsAccumulator.update(_.addWindow(windowOutput)) - } - .interruptWhen(allEventsProcessed(windowsAccumulator, countExpectations)) - .interruptAfter(timeout) - private def assertSingleWindow(output: WindowOutput): MatchResult[Any] = { - output.`shredding_complete.json`.compression must beEqualTo(requiredAppConfig.compression) - output.`shredding_complete.json`.count.get.good must beEqualTo(output.goodEvents.size) - output.`shredding_complete.json`.count.get.bad.get must beEqualTo(output.badEvents.size) - output.`shredding_complete.json`.typesInfo must beLike { case TypesInfo.WideRow(fileFormat, _) => + output.shredding_complete.compression must beEqualTo(requiredAppConfig.compression) + output.shredding_complete.count.get.good must beEqualTo(output.goodEvents.size) + output.shredding_complete.count.get.bad.get must beEqualTo(output.badEvents.size) + output.shredding_complete.typesInfo must beLike { case TypesInfo.WideRow(fileFormat, _) => fileFormat must beEqualTo(requiredAppConfig.fileFormat) } } @@ -129,7 +107,7 @@ abstract class TransformerSpecification extends Specification with AppDependenci private def aggregateDataFromAllWindows(windows: List[WindowOutput]): AggregatedData = windows.foldLeft(AggregatedData.empty) { case (aggregated, window) => - val windowTypes = window.`shredding_complete.json`.typesInfo.asInstanceOf[TypesInfo.WideRow].types + val windowTypes = window.shredding_complete.typesInfo.asInstanceOf[TypesInfo.WideRow].types AggregatedData( good = window.goodEvents ::: aggregated.good, bad = window.badEvents ::: aggregated.bad, @@ -137,13 +115,6 @@ abstract class TransformerSpecification extends Specification with AppDependenci ) } - private def allEventsProcessed( - windowsAccumulator: SignallingRef[IO, WindowsAccumulator], - countExpectations: CountExpectations - ): Signal[IO, Boolean] = - windowsAccumulator - .map(_.getTotalNumberOfEvents >= countExpectations.total) - private def readWindowOutput(blobClient: BlobStorage[IO])(message: LoaderMessage.ShreddingComplete): IO[WindowOutput] = for { scMessageInStorage <- readSCMessageFromBlobStorage(message, blobClient) @@ -228,13 +199,6 @@ abstract class TransformerSpecification extends Specification with AppDependenci .get(message.base.withKey("shredding_complete.json")) .map(value => parseShreddingCompleteMessage(value.right.get)) - private def parseShreddingCompleteMessage(message: String): LoaderMessage.ShreddingComplete = - LoaderMessage - .fromString(message) - .right - .get - .asInstanceOf[LoaderMessage.ShreddingComplete] - } object TransformerSpecification { @@ -243,29 +207,13 @@ object TransformerSpecification { type DataRow = Json type DataAssertion = AggregatedData => MatchResult[Any] - final case class CountExpectations(good: Int, bad: Int) { - def total = good + bad - } - - final case class WindowsAccumulator(value: List[WindowOutput]) { - def addWindow(window: WindowOutput): WindowsAccumulator = - WindowsAccumulator(value :+ window) - - def getTotalNumberOfEvents: Long = - value.map { window => - val good = window.`shredding_complete.json`.count.map(_.good).getOrElse(0L) - val bad = window.`shredding_complete.json`.count.flatMap(_.bad).getOrElse(0L) - good + bad - }.sum - } - final case class WindowOutput( appId: String, producedAt: LocalDateTime, - `shredding_complete.json`: LoaderMessage.ShreddingComplete, + shredding_complete: LoaderMessage.ShreddingComplete, goodEvents: List[DataRow], badEvents: List[DataRow] - ) + ) extends GetShreddingComplete final case class AggregatedData( good: List[DataRow], diff --git a/modules/transformer-kafka/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/kafka/experimental/scenarios/BadDetailsScenario.scala b/modules/transformer-kafka/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/kafka/experimental/scenarios/BadDetailsScenario.scala index a4f5b696d..2f7289265 100644 --- a/modules/transformer-kafka/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/kafka/experimental/scenarios/BadDetailsScenario.scala +++ b/modules/transformer-kafka/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/kafka/experimental/scenarios/BadDetailsScenario.scala @@ -1,12 +1,9 @@ package com.snowplowanalytics.snowplow.rdbloader.transformer.stream.kafka.experimental.scenarios -import com.snowplowanalytics.snowplow.rdbloader.transformer.stream.kafka.experimental.InputBatch.Content -import com.snowplowanalytics.snowplow.rdbloader.transformer.stream.kafka.experimental.TransformerSpecification.CountExpectations -import com.snowplowanalytics.snowplow.rdbloader.transformer.stream.kafka.experimental.{ - AppConfiguration, - AzureTransformerSpecification, - InputBatch -} +import com.snowplowanalytics.snowplow.rdbloader.common.integrationtestutils.InputBatch +import com.snowplowanalytics.snowplow.rdbloader.common.integrationtestutils.InputBatch.Content +import com.snowplowanalytics.snowplow.rdbloader.common.integrationtestutils.ItUtils._ +import com.snowplowanalytics.snowplow.rdbloader.transformer.stream.kafka.experimental.{AppConfiguration, AzureTransformerSpecification} class BadDetailsScenario extends AzureTransformerSpecification { diff --git a/modules/transformer-kafka/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/kafka/experimental/scenarios/json.scala b/modules/transformer-kafka/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/kafka/experimental/scenarios/json.scala index fd248d9a2..9bd30602c 100644 --- a/modules/transformer-kafka/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/kafka/experimental/scenarios/json.scala +++ b/modules/transformer-kafka/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/kafka/experimental/scenarios/json.scala @@ -1,12 +1,10 @@ package com.snowplowanalytics.snowplow.rdbloader.transformer.stream.kafka.experimental.scenarios + import com.snowplowanalytics.snowplow.rdbloader.common.config.TransformerConfig.Compression -import com.snowplowanalytics.snowplow.rdbloader.transformer.stream.kafka.experimental.InputBatch.{Content, bad, good} -import com.snowplowanalytics.snowplow.rdbloader.transformer.stream.kafka.experimental.TransformerSpecification.CountExpectations -import com.snowplowanalytics.snowplow.rdbloader.transformer.stream.kafka.experimental.{ - AppConfiguration, - AzureTransformerSpecification, - InputBatch -} +import com.snowplowanalytics.snowplow.rdbloader.common.integrationtestutils.InputBatch +import com.snowplowanalytics.snowplow.rdbloader.common.integrationtestutils.InputBatch.{Content, bad, good} +import com.snowplowanalytics.snowplow.rdbloader.common.integrationtestutils.ItUtils._ +import com.snowplowanalytics.snowplow.rdbloader.transformer.stream.kafka.experimental.{AppConfiguration, AzureTransformerSpecification} import io.circe.parser import scala.concurrent.duration.DurationInt diff --git a/modules/transformer-kafka/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/kafka/experimental/scenarios/parquet.scala b/modules/transformer-kafka/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/kafka/experimental/scenarios/parquet.scala index 3b45153f8..481266629 100644 --- a/modules/transformer-kafka/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/kafka/experimental/scenarios/parquet.scala +++ b/modules/transformer-kafka/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/kafka/experimental/scenarios/parquet.scala @@ -1,14 +1,11 @@ package com.snowplowanalytics.snowplow.rdbloader.transformer.stream.kafka.experimental.scenarios -import com.snowplowanalytics.snowplow.rdbloader.common.LoaderMessage.TypesInfo.WideRow.WideRowFormat import com.snowplowanalytics.snowplow.rdbloader.common.config.TransformerConfig.Compression -import com.snowplowanalytics.snowplow.rdbloader.transformer.stream.kafka.experimental.InputBatch.{Content, bad, good} -import com.snowplowanalytics.snowplow.rdbloader.transformer.stream.kafka.experimental.TransformerSpecification.CountExpectations -import com.snowplowanalytics.snowplow.rdbloader.transformer.stream.kafka.experimental.{ - AppConfiguration, - AzureTransformerSpecification, - InputBatch -} +import com.snowplowanalytics.snowplow.rdbloader.common.LoaderMessage.TypesInfo.WideRow.WideRowFormat +import com.snowplowanalytics.snowplow.rdbloader.common.integrationtestutils.InputBatch +import com.snowplowanalytics.snowplow.rdbloader.common.integrationtestutils.InputBatch.{Content, bad, good} +import com.snowplowanalytics.snowplow.rdbloader.common.integrationtestutils.ItUtils._ +import com.snowplowanalytics.snowplow.rdbloader.transformer.stream.kafka.experimental.{AppConfiguration, AzureTransformerSpecification} import io.circe.parser import scala.concurrent.duration.DurationInt diff --git a/project/Dependencies.scala b/project/Dependencies.scala index 921d141e2..ac6332130 100644 --- a/project/Dependencies.scala +++ b/project/Dependencies.scala @@ -253,7 +253,8 @@ object Dependencies { scalaTrackerEmit, http4sClient, slf4jApi, - sentry + sentry, + eventGenerator ) val loaderDependencies = Seq( @@ -339,8 +340,7 @@ object Dependencies { ) val transformerKafkaDependencies = Seq( - hadoopAzure, - eventGenerator + hadoopAzure ) val commonStreamTransformerExclusions =