diff --git a/config/loader/aws/redshift.config.reference.hocon b/config/loader/aws/redshift.config.reference.hocon index 0547b466e..2a25fd8ba 100644 --- a/config/loader/aws/redshift.config.reference.hocon +++ b/config/loader/aws/redshift.config.reference.hocon @@ -266,4 +266,14 @@ # 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": { + # List of SchemaKey with partial SchemaVer to disable migration for, redshift only + # Redshift Loader will disable all migration and recovery table creation for the tables + # which belongs to provided schema keys + # e.g. [ "iglu:com.example/myschema1/jsonschema/1-*-*", "iglu:com.example/myschema2/jsonschema/1-*-*"] + # Optional, empty list by default + "disableMigration": [] + } } diff --git a/modules/common-transformer-stream/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/common/Processing.scala b/modules/common-transformer-stream/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/common/Processing.scala index 5953ca044..fb705a3a9 100644 --- a/modules/common-transformer-stream/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/common/Processing.scala +++ b/modules/common-transformer-stream/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/common/Processing.scala @@ -68,7 +68,7 @@ object Processing { implicit val lookup: RegistryLookup[F] = resources.registryLookup val transformer: Transformer[F] = config.formats match { case f: TransformerConfig.Formats.Shred => - Transformer.ShredTransformer(resources.igluResolver, resources.propertiesCache, f, processor) + Transformer.ShredTransformer(resources.igluResolver, resources.shredModelCache, f, processor) case f: TransformerConfig.Formats.WideRow => Transformer.WideRowTransformer(resources.igluResolver, f, processor) } @@ -257,7 +257,9 @@ object Processing { implicit class TransformedOps(t: Transformed) { def getPath: SinkPath = t match { case p: Transformed.Shredded => - val suffix = Some(s"vendor=${p.vendor}/name=${p.name}/format=${p.format.path.toLowerCase}/model=${p.model}/") + val suffix = Some( + s"vendor=${p.vendor}/name=${p.name}/format=${p.format.path.toLowerCase}/model=${p.model}/revision=${p.revision}/addition=${p.addition}" + ) val pathType = if (p.isGood) SinkPath.PathType.Good else SinkPath.PathType.Bad SinkPath(suffix, pathType) case p: Transformed.WideRow => 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 dc0205790..5b22ea033 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 @@ -8,44 +8,37 @@ package com.snowplowanalytics.snowplow.rdbloader.transformer.stream.common import java.util.UUID - import scala.concurrent.ExecutionContext - import org.typelevel.log4cats.slf4j.Slf4jLogger import org.typelevel.log4cats.Logger - import io.circe.Json - import cats.{Applicative, Monad, MonadThrow} import cats.implicits._ import cats.effect._ import cats.effect.std.Random - import io.sentry.SentryClient import com.snowplowanalytics.iglu.client.resolver.{CreateResolverCache, Resolver} import com.snowplowanalytics.iglu.client.resolver.Resolver.ResolverConfig import com.snowplowanalytics.iglu.client.resolver.registries.{Http4sRegistryLookup, RegistryLookup} -import com.snowplowanalytics.iglu.schemaddl.Properties +import com.snowplowanalytics.iglu.schemaddl.redshift.ShredModel import com.snowplowanalytics.lrumap.CreateLruMap import com.snowplowanalytics.snowplow.analytics.scalasdk.Event import com.snowplowanalytics.snowplow.rdbloader.common.Sentry 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.common.transformation.{EventUtils, ShredModelCache, ShredModelCacheKey} 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 import com.snowplowanalytics.snowplow.rdbloader.transformer.stream.common.sources.Checkpointer - import com.snowplowanalytics.snowplow.scalatracker.Tracking - import org.http4s.blaze.client.BlazeClientBuilder case class Resources[F[_], C]( igluResolver: Resolver[F], - propertiesCache: PropertiesCache[F], + shredModelCache: ShredModelCache[F], eventParser: EventParser, producer: Queue.Producer[F], instanceId: String, @@ -81,7 +74,7 @@ object Resources { producer <- mkQueue(config.queue) resolverConfig <- mkResolverConfig(igluConfig) resolver <- mkResolver(resolverConfig) - propertiesCache <- Resource.eval(CreateLruMap[F, PropertiesKey, Properties].create(resolverConfig.cacheSize)) + shredModelCache <- Resource.eval(CreateLruMap[F, ShredModelCacheKey, ShredModel].create(resolverConfig.cacheSize)) httpClient <- BlazeClientBuilder[F].withExecutionContext(executionContext).resource implicit0(registryLookup: RegistryLookup[F]) <- Resource.pure(Http4sRegistryLookup[F](httpClient)) eventParser <- mkEventParser(resolver, config) @@ -103,7 +96,7 @@ object Resources { badSink <- mkBadSink(config, mkBadQueue) } yield Resources( resolver, - propertiesCache, + shredModelCache, eventParser, producer, instanceId.toString, diff --git a/modules/common-transformer-stream/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/common/Transformer.scala b/modules/common-transformer-stream/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/common/Transformer.scala index 9fb962bc4..7de06ac72 100644 --- a/modules/common-transformer-stream/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/common/Transformer.scala +++ b/modules/common-transformer-stream/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/common/Transformer.scala @@ -20,7 +20,7 @@ import com.snowplowanalytics.snowplow.rdbloader.common.Common import com.snowplowanalytics.snowplow.rdbloader.common.LoaderMessage.{SnowplowEntity, TypesInfo} import com.snowplowanalytics.snowplow.rdbloader.common.config.TransformerConfig.Formats import com.snowplowanalytics.snowplow.rdbloader.common.config.TransformerConfig.Formats.WideRow -import com.snowplowanalytics.snowplow.rdbloader.common.transformation.{PropertiesCache, Transformed} +import com.snowplowanalytics.snowplow.rdbloader.common.transformation.{ShredModelCache, Transformed} import com.snowplowanalytics.snowplow.rdbloader.common.transformation.parquet.fields.AllFields import com.snowplowanalytics.snowplow.rdbloader.common.transformation.parquet.{ AtomicFieldsProvider, @@ -40,7 +40,7 @@ sealed trait Transformer[F[_]] extends Product with Serializable { object Transformer { case class ShredTransformer[F[_]: Monad: RegistryLookup: Clock]( igluResolver: Resolver[F], - propertiesCache: PropertiesCache[F], + shredModelCache: ShredModelCache[F], formats: Formats.Shred, processor: Processor ) extends Transformer[F] { @@ -54,12 +54,12 @@ object Transformer { else TypesInfo.Shredded.ShreddedFormat.JSON def goodTransform(event: Event): EitherT[F, BadRow, List[Transformed]] = - Transformed.shredEvent[F](igluResolver, propertiesCache, isTabular, processor)(event) + Transformed.shredEvent[F](igluResolver, shredModelCache, isTabular, processor)(event) def badTransform(badRow: BadRow): Transformed = { - val SchemaKey(vendor, name, _, SchemaVer.Full(model, _, _)) = badRow.schemaKey + val SchemaKey(vendor, name, _, SchemaVer.Full(model, revision, addition)) = badRow.schemaKey val data = Transformed.Data.DString(badRow.compact) - Transformed.Shredded.Json(false, vendor, name, model, data) + Transformed.Shredded.Json(false, vendor, name, model, revision, addition, data) } def typesInfo(types: Set[Data.ShreddedType]): TypesInfo = { diff --git a/modules/common-transformer-stream/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/common/processing/ShredTsvProcessingSpec.scala b/modules/common-transformer-stream/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/common/processing/ShredTsvProcessingSpec.scala index 05b9f41f0..066fcd358 100644 --- a/modules/common-transformer-stream/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/common/processing/ShredTsvProcessingSpec.scala +++ b/modules/common-transformer-stream/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/common/processing/ShredTsvProcessingSpec.scala @@ -32,28 +32,28 @@ class ShredTsvProcessingSpec extends BaseProcessingSpec { readStringRowsFrom( Path( outputDirectory.toString + - s"/run=1970-01-01-10-30-00-${AppId.appId}/output=good/vendor=com.snowplowanalytics.snowplow/name=atomic/format=tsv/model=1" + s"/run=1970-01-01-10-30-00-${AppId.appId}/output=good/vendor=com.snowplowanalytics.snowplow/name=atomic/format=tsv/model=1/revision=0/addition=0" ) ) actualOptimizelyRows <- readStringRowsFrom( Path( outputDirectory.toString + - s"/run=1970-01-01-10-30-00-${AppId.appId}/output=good/vendor=com.optimizely/name=state/format=tsv/model=1" + s"/run=1970-01-01-10-30-00-${AppId.appId}/output=good/vendor=com.optimizely/name=state/format=tsv/model=1/revision=0/addition=0" ) ) actualConsentRows <- readStringRowsFrom( Path( outputDirectory.toString + - s"/run=1970-01-01-10-30-00-${AppId.appId}/output=good/vendor=com.snowplowanalytics.snowplow/name=consent_document/format=tsv/model=1" + s"/run=1970-01-01-10-30-00-${AppId.appId}/output=good/vendor=com.snowplowanalytics.snowplow/name=consent_document/format=tsv/model=1/revision=0/addition=0" ) ) actualBadRows <- readStringRowsFrom( Path( outputDirectory.toString + - s"/run=1970-01-01-10-30-00-${AppId.appId}/output=bad/vendor=com.snowplowanalytics.snowplow.badrows/name=loader_parsing_error/format=json/model=2/" + s"/run=1970-01-01-10-30-00-${AppId.appId}/output=bad/vendor=com.snowplowanalytics.snowplow.badrows/name=loader_parsing_error/format=json/model=2/revision=0/addition=0" ) ) @@ -92,14 +92,14 @@ class ShredTsvProcessingSpec extends BaseProcessingSpec { readStringRowsFrom( Path( outputDirectory.toString + - s"/run=1970-01-01-10-30-00-${AppId.appId}/output=good/vendor=com.snowplowanalytics.snowplow/name=atomic/format=tsv/model=1" + s"/run=1970-01-01-10-30-00-${AppId.appId}/output=good/vendor=com.snowplowanalytics.snowplow/name=atomic/format=tsv/model=1/revision=0/addition=0" ) ) actualBadRows <- readStringRowsFrom( Path( outputDirectory.toString + - s"/run=1970-01-01-10-30-00-${AppId.appId}/output=bad/vendor=com.snowplowanalytics.snowplow.badrows/name=loader_iglu_error/format=json/model=2/" + s"/run=1970-01-01-10-30-00-${AppId.appId}/output=bad/vendor=com.snowplowanalytics.snowplow.badrows/name=loader_iglu_error/format=json/model=2/revision=0/addition=0" ) ) diff --git a/modules/common-transformer-stream/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/common/sinks/TransformingSpec.scala b/modules/common-transformer-stream/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/common/sinks/TransformingSpec.scala index 96b00901b..8f3e158ea 100644 --- a/modules/common-transformer-stream/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/common/sinks/TransformingSpec.scala +++ b/modules/common-transformer-stream/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/stream/common/sinks/TransformingSpec.scala @@ -20,13 +20,13 @@ import io.circe.parser.{parse => parseCirce} import org.http4s.client.{Client => Http4sClient} import com.snowplowanalytics.iglu.client.Resolver import com.snowplowanalytics.iglu.client.resolver.registries.{Http4sRegistryLookup, Registry, RegistryLookup} -import com.snowplowanalytics.iglu.schemaddl.Properties +import com.snowplowanalytics.iglu.schemaddl.redshift.ShredModel import com.snowplowanalytics.lrumap.CreateLruMap import com.snowplowanalytics.snowplow.analytics.scalasdk.Event import com.snowplowanalytics.snowplow.rdbloader.generated.BuildInfo import com.snowplowanalytics.snowplow.rdbloader.common.LoaderMessage import com.snowplowanalytics.snowplow.rdbloader.common.config.TransformerConfig -import com.snowplowanalytics.snowplow.rdbloader.common.transformation.{PropertiesCache, PropertiesKey, Transformed} +import com.snowplowanalytics.snowplow.rdbloader.common.transformation.{ShredModelCache, ShredModelCacheKey, Transformed} import com.snowplowanalytics.snowplow.rdbloader.transformer.stream.common.{Processing, Transformer} import com.snowplowanalytics.snowplow.rdbloader.transformer.stream.common.sources.{Checkpointer, ParsedC} import com.snowplowanalytics.snowplow.rdbloader.transformer.stream.common.processing.TestApplication._ @@ -41,12 +41,12 @@ class TransformingSpec extends Specification { val testFileNameMap = List( Transformed.Shredded - .Tabular("com.snowplowanalytics.snowplow", "atomic", 1, dummyTransformedData) + .Tabular("com.snowplowanalytics.snowplow", "atomic", 1, 0, 0, dummyTransformedData) .getPath -> "com.snowplowanalytics.snowplow-atomic", Transformed.Shredded - .Tabular("com.snowplowanalytics.snowplow", "consent_document", 1, dummyTransformedData) + .Tabular("com.snowplowanalytics.snowplow", "consent_document", 1, 0, 0, dummyTransformedData) .getPath -> "com.snowplowanalytics.snowplow-consent_document", - Transformed.Shredded.Tabular("com.optimizely", "state", 1, dummyTransformedData).getPath -> "com.optimizely-state" + Transformed.Shredded.Tabular("com.optimizely", "state", 1, 0, 0, dummyTransformedData).getPath -> "com.optimizely-state" ).toMap val expectedTransformedMap = @@ -128,12 +128,12 @@ object TransformingSpec { val defaultWindow = Window(1, 1, 1, 1, 1) val dummyTransformedData = Transformed.Data.DString("") - def propertiesCache: PropertiesCache[IO] = CreateLruMap[IO, PropertiesKey, Properties].create(100).unsafeRunSync() + def shredModelCache: ShredModelCache[IO] = CreateLruMap[IO, ShredModelCacheKey, ShredModel].create(100).unsafeRunSync() def createTransformer(formats: TransformerConfig.Formats): Transformer[IO] = formats match { case f: TransformerConfig.Formats.Shred => - Transformer.ShredTransformer(defaultIgluResolver, propertiesCache, f, TestProcessor) + Transformer.ShredTransformer(defaultIgluResolver, shredModelCache, f, TestProcessor) case f: TransformerConfig.Formats.WideRow => Transformer.WideRowTransformer(defaultIgluResolver, f, TestProcessor) } diff --git a/modules/common/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/common/Common.scala b/modules/common/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/common/Common.scala index 49e34759e..65314f0ff 100644 --- a/modules/common/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/common/Common.scala +++ b/modules/common/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/common/Common.scala @@ -32,7 +32,7 @@ object Common { DateTimeFormatter.ofPattern("yyyy-MM-dd-HH-mm-ss").withZone(ZoneId.from(ZoneOffset.UTC)) def entityPath(entity: TypesInfo.Shredded.Type) = - s"$GoodPrefix/vendor=${entity.schemaKey.vendor}/name=${entity.schemaKey.name}/format=${entity.format.path}/model=${entity.schemaKey.version.model}" + s"$GoodPrefix/vendor=${entity.schemaKey.vendor}/name=${entity.schemaKey.name}/format=${entity.format.path}/model=${entity.schemaKey.version.model}/revision=${entity.schemaKey.version.revision}/addition=${entity.schemaKey.version.addition}" def entityPathFull(base: BlobStorage.Folder, entity: TypesInfo.Shredded.Type): BlobStorage.Folder = BlobStorage.Folder.append(base, entityPath(entity)) diff --git a/modules/common/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/common/SchemaProvider.scala b/modules/common/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/common/SchemaProvider.scala index 1bca050c3..506ee8028 100644 --- a/modules/common/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/common/SchemaProvider.scala +++ b/modules/common/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/common/SchemaProvider.scala @@ -16,7 +16,7 @@ import com.snowplowanalytics.iglu.client.{ClientError, Resolver} import com.snowplowanalytics.iglu.core.{SchemaCriterion, SchemaKey} import com.snowplowanalytics.iglu.schemaddl.jsonschema.Schema import com.snowplowanalytics.iglu.schemaddl.jsonschema.circe.implicits.toSchema -import com.snowplowanalytics.snowplow.badrows.FailureDetails +import com.snowplowanalytics.snowplow.badrows.FailureDetails.LoaderIgluError object SchemaProvider { @@ -31,7 +31,7 @@ object SchemaProvider { def getSchema[F[_]: Clock: Monad: RegistryLookup]( resolver: Resolver[F], schemaKey: SchemaKey - ): EitherT[F, FailureDetails.LoaderIgluError, Schema] = + ): EitherT[F, LoaderIgluError, Schema] = for { json <- EitherT(resolver.lookupSchema(schemaKey)).leftMap(resolverBadRow(schemaKey)) schema <- EitherT.fromOption[F](Schema.parse(json), parseSchemaBadRow(schemaKey)) @@ -40,7 +40,7 @@ object SchemaProvider { def fetchSchemasWithSameModel[F[_]: Clock: Monad: RegistryLookup]( resolver: Resolver[F], schemaKey: SchemaKey - ): EitherT[F, FailureDetails.LoaderIgluError, List[SchemaWithKey]] = + ): EitherT[F, LoaderIgluError, List[SchemaWithKey]] = EitherT(resolver.listSchemasLike(schemaKey)) .leftMap(resolverFetchBadRow(schemaKey.vendor, schemaKey.name, schemaKey.format, schemaKey.version.model)) .map(_.schemas) @@ -59,13 +59,13 @@ object SchemaProvider { model: Int )( e: ClientError.ResolutionError - ): FailureDetails.LoaderIgluError = - FailureDetails.LoaderIgluError.SchemaListNotFound(SchemaCriterion(vendor = vendor, name = name, format = format, model = model), e) + ): LoaderIgluError = + LoaderIgluError.SchemaListNotFound(SchemaCriterion(vendor = vendor, name = name, format = format, model = model), e) - private def resolverBadRow(schemaKey: SchemaKey)(e: ClientError.ResolutionError): FailureDetails.LoaderIgluError = - FailureDetails.LoaderIgluError.IgluError(schemaKey, e) + private def resolverBadRow(schemaKey: SchemaKey)(e: ClientError.ResolutionError): LoaderIgluError = + LoaderIgluError.IgluError(schemaKey, e) - private def parseSchemaBadRow(schemaKey: SchemaKey): FailureDetails.LoaderIgluError = - FailureDetails.LoaderIgluError.InvalidSchema(schemaKey, "Cannot be parsed as JSON Schema AST") + private def parseSchemaBadRow(schemaKey: SchemaKey): LoaderIgluError = + LoaderIgluError.InvalidSchema(schemaKey, "Cannot be parsed as JSON Schema AST") } diff --git a/modules/common/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/common/transformation/EventUtils.scala b/modules/common/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/common/transformation/EventUtils.scala index ec2109fc9..6b36198b3 100644 --- a/modules/common/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/common/transformation/EventUtils.scala +++ b/modules/common/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/common/transformation/EventUtils.scala @@ -13,7 +13,7 @@ import java.time.format.DateTimeParseException import io.circe.Json import cats.Monad -import cats.data.{EitherT, NonEmptyList} +import cats.data.NonEmptyList import cats.implicits._ import cats.effect.Clock @@ -23,16 +23,13 @@ import com.snowplowanalytics.iglu.core._ import com.snowplowanalytics.iglu.client.{ClientError, Resolver} import com.snowplowanalytics.iglu.client.resolver.registries.RegistryLookup -import com.snowplowanalytics.iglu.schemaddl.migrations.FlatData import com.snowplowanalytics.iglu.schemaddl.jsonschema.Schema import com.snowplowanalytics.iglu.schemaddl.jsonschema.circe.implicits._ import com.snowplowanalytics.snowplow.analytics.scalasdk.{Event, ParsingError} import com.snowplowanalytics.snowplow.badrows.{BadRow, Failure, FailureDetails, Payload, Processor} -import com.snowplowanalytics.snowplow.rdbloader.common.Common -import Flattening.NullCharacter -import com.snowplowanalytics.iglu.schemaddl.Properties import com.snowplowanalytics.snowplow.analytics.scalasdk.decode.TSVParser +import com.snowplowanalytics.snowplow.rdbloader.common.Common object EventUtils { @@ -131,34 +128,6 @@ object EventUtils { "events" ) - /** - * Transform a self-desribing entity into tabular format, using its known schemas to get a correct - * order of columns - * @param resolver - * Iglu resolver to get list of known schemas - * @param instance - * self-describing JSON that needs to be transformed - * @return - * list of columns or flattening error - */ - def flatten[F[_]: Monad: Clock: RegistryLookup]( - resolver: Resolver[F], - propertiesCache: PropertiesCache[F], - instance: SelfDescribingData[Json] - ): EitherT[F, FailureDetails.LoaderIgluError, List[String]] = - Flattening - .getDdlProperties(resolver, propertiesCache, instance.schema) - .map(props => mapProperties(props, instance)) - - private def mapProperties(props: Properties, instance: SelfDescribingData[Json]) = - props - .map { case (pointer, _) => - FlatData.getPath(pointer.forData, instance.data, getString, NullCharacter) - } - - def getString(json: Json): String = - json.fold(NullCharacter, transformBool, _ => json.show, escape, _ => escape(json.noSpaces), _ => escape(json.noSpaces)) - def getEntities(event: Event): List[SelfDescribingData[Json]] = event.unstruct_event.data.toList ++ event.derived_contexts.data ++ @@ -171,11 +140,6 @@ object EventUtils { case _ => 0 } - /** Prevents data with newlines and tabs from breaking the loading process */ - private def escape(s: String): String = - if (s == NullCharacter) "\\\\N" - else s.replace('\t', ' ').replace('\n', ' ') - private def getLength(schema: Schema): Option[Int] = schema.maxLength.map(_.value.toInt) } diff --git a/modules/common/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/common/transformation/Flattening.scala b/modules/common/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/common/transformation/Flattening.scala deleted file mode 100644 index d6632342a..000000000 --- a/modules/common/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/common/transformation/Flattening.scala +++ /dev/null @@ -1,116 +0,0 @@ -/* - * Copyright (c) 2012-present Snowplow Analytics Ltd. All rights reserved. - * - * This program is licensed to you under the Snowplow Community License Version 1.0, - * and you may not use this file except in compliance with the Snowplow Community License Version 1.0. - * You may obtain a copy of the Snowplow Community License Version 1.0 at https://docs.snowplow.io/community-license-1.0 - */ -package com.snowplowanalytics.snowplow.rdbloader.common.transformation - -import io.circe.Json -import cats.Monad -import cats.data.EitherT -import cats.syntax.either._ -import cats.effect.Clock -import com.snowplowanalytics.iglu.client.resolver.Resolver.{ResolverResult, SchemaListKey} -import com.snowplowanalytics.iglu.core._ -import com.snowplowanalytics.iglu.core.circe.implicits._ -import com.snowplowanalytics.iglu.client.resolver.registries.RegistryLookup -import com.snowplowanalytics.iglu.client.{ClientError, Resolver} -import com.snowplowanalytics.iglu.schemaddl.{IgluSchema, Properties} -import com.snowplowanalytics.iglu.schemaddl.migrations.{FlatSchema, SchemaList => DdlSchemaList} -import com.snowplowanalytics.iglu.schemaddl.jsonschema.Schema -import com.snowplowanalytics.iglu.schemaddl.jsonschema.circe.implicits._ -import com.snowplowanalytics.snowplow.badrows.FailureDetails - -object Flattening { - - /** Redshift default NULL string */ - val NullCharacter: String = "\\N" - - val MetaSchema = SchemaKey("com.snowplowanalyics.self-desc", "schema", "jsonschema", SchemaVer.Full(1, 0, 0)) - - def getOrdered[F[_]: Monad: RegistryLookup: Clock]( - resolver: Resolver[F], - vendor: String, - name: String, - model: Int - ): EitherT[F, FailureDetails.LoaderIgluError, DdlSchemaList] = { - val criterion = SchemaCriterion(vendor, name, "jsonschema", Some(model), None, None) - val schemaList = resolver.listSchemas(vendor, name, model) - for { - schemaList <- EitherT[F, ClientError.ResolutionError, SchemaList](schemaList).leftMap(error => - FailureDetails.LoaderIgluError.SchemaListNotFound(criterion, error) - ) - ordered <- DdlSchemaList.fromSchemaList(schemaList, fetch(resolver)) - } yield ordered - } - - def getDdlProperties[F[_]: Monad: Clock: RegistryLookup]( - resolver: Resolver[F], - propertiesCache: PropertiesCache[F], - schemaKey: SchemaKey - ): EitherT[F, FailureDetails.LoaderIgluError, Properties] = { - val criterion = SchemaCriterion(schemaKey.vendor, schemaKey.name, "jsonschema", Some(schemaKey.version.model), None, None) - - EitherT(resolver.listSchemasResult(schemaKey.vendor, schemaKey.name, schemaKey.version.model)) - .leftMap(error => FailureDetails.LoaderIgluError.SchemaListNotFound(criterion, error)) - .flatMap { - case cached: ResolverResult.Cached[SchemaListKey, SchemaList] => - lookupInCache(resolver, propertiesCache, cached) - case ResolverResult.NotCached(schemaList) => - evaluateProperties(schemaList, resolver) - } - } - - def fetch[F[_]: Monad: RegistryLookup: Clock]( - resolver: Resolver[F] - )( - key: SchemaKey - ): EitherT[F, FailureDetails.LoaderIgluError, IgluSchema] = - for { - json <- EitherT(resolver.lookupSchema(key)).leftMap(error => FailureDetails.LoaderIgluError.IgluError(key, error)) - schema <- EitherT.fromEither(parseSchema(json)) - } yield schema - - private def lookupInCache[F[_]: Monad: RegistryLookup: Clock]( - resolver: Resolver[F], - propertiesCache: PropertiesCache[F], - resolvedSchemaList: ResolverResult.Cached[SchemaListKey, SchemaList] - ): EitherT[F, FailureDetails.LoaderIgluError, Properties] = { - val propertiesKey = (resolvedSchemaList.key, resolvedSchemaList.timestamp) - - EitherT.liftF(propertiesCache.get(propertiesKey)).flatMap { - case Some(properties) => - EitherT.pure[F, FailureDetails.LoaderIgluError](properties) - case None => - evaluateProperties(resolvedSchemaList.value, resolver) - .semiflatTap(props => propertiesCache.put(propertiesKey, props)) - } - } - - private def evaluateProperties[F[_]: Monad: RegistryLookup: Clock]( - schemaList: SchemaList, - resolver: Resolver[F] - ): EitherT[F, FailureDetails.LoaderIgluError, Properties] = - DdlSchemaList - .fromSchemaList(schemaList, fetch(resolver)) - .map(FlatSchema.extractProperties) - - /** Parse JSON into self-describing schema, or return `FlatteningError` */ - private def parseSchema(json: Json): Either[FailureDetails.LoaderIgluError, IgluSchema] = - for { - selfDescribing <- SelfDescribingSchema.parse(json).leftMap(invalidSchema(json)) - parsed <- Schema.parse(selfDescribing.schema).toRight(invalidSchema(selfDescribing)) - } yield SelfDescribingSchema(selfDescribing.self, parsed) - - private def invalidSchema(json: Json)(code: ParseError): FailureDetails.LoaderIgluError = { - val error = s"Cannot parse ${json.noSpaces} as self-describing schema, ${code.code}" - FailureDetails.LoaderIgluError.InvalidSchema(MetaSchema, error) - } - - private def invalidSchema(schema: SelfDescribingSchema[_]): FailureDetails.LoaderIgluError = { - val error = s"Cannot be parsed as JSON Schema AST" - FailureDetails.LoaderIgluError.InvalidSchema(schema.self.schemaKey, error) - } -} diff --git a/modules/common/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/common/transformation/Transformed.scala b/modules/common/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/common/transformation/Transformed.scala index 427edb054..66d5f42a8 100644 --- a/modules/common/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/common/transformation/Transformed.scala +++ b/modules/common/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/common/transformation/Transformed.scala @@ -13,12 +13,19 @@ import cats.data.{EitherT, NonEmptyList} import cats.effect.Clock import com.snowplowanalytics.iglu.client.Resolver import com.snowplowanalytics.iglu.client.resolver.registries.RegistryLookup -import com.snowplowanalytics.iglu.core.SchemaKey +import com.snowplowanalytics.iglu.client.resolver.Resolver.{ResolverResult, SchemaListKey} +import com.snowplowanalytics.iglu.core.{SchemaCriterion, SchemaKey, SchemaList, SchemaMap, SelfDescribingSchema} +import com.snowplowanalytics.iglu.schemaddl.jsonschema.Schema import com.snowplowanalytics.iglu.schemaddl.parquet.{Field, FieldValue} +import com.snowplowanalytics.iglu.schemaddl.redshift._ import com.snowplowanalytics.snowplow.analytics.scalasdk.Event import com.snowplowanalytics.snowplow.badrows.{BadRow, FailureDetails, Processor} +import com.snowplowanalytics.snowplow.badrows.FailureDetails.LoaderIgluError +import com.snowplowanalytics.snowplow.badrows.FailureDetails.LoaderIgluError.SchemaListNotFound import com.snowplowanalytics.snowplow.rdbloader.common.Common.AtomicSchema import com.snowplowanalytics.snowplow.rdbloader.common.LoaderMessage.TypesInfo.Shredded.ShreddedFormat +import com.snowplowanalytics.snowplow.rdbloader.common.SchemaProvider +import com.snowplowanalytics.snowplow.rdbloader.common.SchemaProvider.SchemaWithKey /** Represents transformed data in blob storage */ @@ -48,6 +55,8 @@ object Transformed { def name: String def format: ShreddedFormat def model: Int + def revision: Int + def addition: Int def data: Data.DString } @@ -61,6 +70,8 @@ object Transformed { vendor: String, name: String, model: Int, + revision: Int, + addition: Int, data: Data.DString ) extends Shredded { val format = ShreddedFormat.JSON @@ -71,6 +82,8 @@ object Transformed { vendor: String, name: String, model: Int, + revision: Int, + addition: Int, data: Data.DString ) extends Shredded { val isGood = true // We don't support TSV shredding for bad data @@ -104,7 +117,7 @@ object Transformed { */ def shredEvent[F[_]: Monad: Clock: RegistryLookup]( igluResolver: Resolver[F], - propertiesCache: PropertiesCache[F], + shredModelCache: ShredModelCache[F], isTabular: SchemaKey => Boolean, processor: Processor )( @@ -114,18 +127,85 @@ object Transformed { .fromEvent(event) .traverse { hierarchy => val tabular = isTabular(hierarchy.entity.schema) - fromHierarchy(tabular, igluResolver, propertiesCache)(hierarchy) + fromHierarchy(tabular, igluResolver, shredModelCache)(hierarchy) } .leftMap(error => EventUtils.shreddingBadRow(event, processor)(NonEmptyList.one(error))) .map { shredded => val data = EventUtils.alterEnrichedEvent(event) - val atomic = Shredded.Tabular(AtomicSchema.vendor, AtomicSchema.name, AtomicSchema.version.model, Transformed.Data.DString(data)) + val atomic = Shredded.Tabular( + AtomicSchema.vendor, + AtomicSchema.name, + AtomicSchema.version.model, + AtomicSchema.version.revision, + AtomicSchema.version.addition, + Transformed.Data.DString(data) + ) atomic :: shredded } def wideRowEvent(event: Event): Transformed = WideRow(good = true, Transformed.Data.DString(event.toJson(true).noSpaces)) + def getShredModel[F[_]: Monad: Clock: RegistryLookup]( + schemaKey: SchemaKey, + schemaKeys: List[SchemaKey], + resolver: Resolver[F] + ): EitherT[F, LoaderIgluError, ShredModel] = + schemaKeys + .traverse { sk => + SchemaProvider + .getSchema(resolver, sk) + .map(schema => SchemaWithKey(sk, schema)) + } + .flatMap { schemaWithKeyList => + EitherT + .fromOption[F][FailureDetails.LoaderIgluError, NonEmptyList[SchemaWithKey]]( + NonEmptyList.fromList(schemaWithKeyList), + FailureDetails.LoaderIgluError.InvalidSchema(schemaKey, s"Empty resolver response for $schemaKey") + ) + .map { nel => + val schemas = nel.map(swk => SelfDescribingSchema[Schema](SchemaMap(swk.schemaKey), swk.schema)) + foldMapRedshiftSchemas(schemas)(schemaKey) + } + } + + /** + * Lookup ShredModel for given SchemaKey and evaluate if not found + */ + def lookupShredModel[F[_]: Monad: Clock: RegistryLookup]( + schemaKey: SchemaKey, + shredModelCache: ShredModelCache[F], + resolver: => Resolver[F] + ): EitherT[F, LoaderIgluError, ShredModel] = { + val criterion = SchemaCriterion(schemaKey.vendor, schemaKey.name, schemaKey.format, Some(schemaKey.version.model), None, None) + + EitherT(resolver.listSchemasResult(schemaKey.vendor, schemaKey.name, schemaKey.version.model, Some(schemaKey))) + .leftMap(error => SchemaListNotFound(criterion, error)) + .flatMap { + case cached: ResolverResult.Cached[SchemaListKey, SchemaList] => + lookupInCache(schemaKey, resolver, shredModelCache, cached) + case ResolverResult.NotCached(schemaList) => + val schemaKeys = schemaList.schemas + getShredModel(schemaKey, schemaKeys, resolver) + } + } + + def lookupInCache[F[_]: Monad: Clock: RegistryLookup]( + schemaKey: SchemaKey, + resolver: Resolver[F], + shredModelCache: ShredModelCache[F], + cached: ResolverResult.Cached[SchemaListKey, SchemaList] + ) = { + val key = (schemaKey, cached.timestamp) + EitherT.liftF(shredModelCache.get(key)).flatMap { + case Some(model) => + EitherT.pure[F, FailureDetails.LoaderIgluError](model) + case None => + getShredModel(schemaKey, cached.value.schemas, resolver) + .semiflatTap(props => shredModelCache.put(key, props)) + } + } + /** * Transform JSON `Hierarchy`, extracted from enriched into a `Shredded` entity, specifying how it * should look like in destination: JSON or TSV If flattening algorithm failed at any point - it @@ -141,20 +221,37 @@ object Transformed { private def fromHierarchy[F[_]: Monad: Clock: RegistryLookup]( tabular: Boolean, resolver: => Resolver[F], - propertiesCache: PropertiesCache[F] + shredModelCache: ShredModelCache[F] )( hierarchy: Hierarchy ): EitherT[F, FailureDetails.LoaderIgluError, Transformed] = { val vendor = hierarchy.entity.schema.vendor val name = hierarchy.entity.schema.name + val meta = EventUtils.buildMetadata(hierarchy.eventId, hierarchy.collectorTstamp, hierarchy.entity.schema) if (tabular) { - EventUtils.flatten(resolver, propertiesCache, hierarchy.entity).map { columns => - val meta = EventUtils.buildMetadata(hierarchy.eventId, hierarchy.collectorTstamp, hierarchy.entity.schema) - Shredded.Tabular(vendor, name, hierarchy.entity.schema.version.model, Transformed.Data.DString((meta ++ columns).mkString("\t"))) - } + lookupShredModel(hierarchy.entity.schema, shredModelCache, resolver) + .map { shredModel => + val columns = shredModel.jsonToStrings(hierarchy.entity.data) + Shredded.Tabular( + vendor, + name, + hierarchy.entity.schema.version.model, + hierarchy.entity.schema.version.revision, + hierarchy.entity.schema.version.addition, + Transformed.Data.DString((meta ++ columns).mkString("\t")) + ) + } } else EitherT.pure[F, FailureDetails.LoaderIgluError]( - Shredded.Json(true, vendor, name, hierarchy.entity.schema.version.model, Transformed.Data.DString(hierarchy.dumpJson)) + Shredded.Json( + true, + vendor, + name, + hierarchy.entity.schema.version.model, + hierarchy.entity.schema.version.revision, + hierarchy.entity.schema.version.addition, + Transformed.Data.DString(hierarchy.dumpJson) + ) ) } } diff --git a/modules/common/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/common/transformation/package.scala b/modules/common/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/common/transformation/package.scala index 43b177ea0..ded0f5923 100644 --- a/modules/common/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/common/transformation/package.scala +++ b/modules/common/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/common/transformation/package.scala @@ -7,20 +7,20 @@ */ package com.snowplowanalytics.snowplow.rdbloader.common -import com.snowplowanalytics.iglu.client.resolver.Resolver.SchemaListKey import com.snowplowanalytics.iglu.client.resolver.StorageTime +import com.snowplowanalytics.iglu.core.SchemaKey +import com.snowplowanalytics.iglu.schemaddl.redshift.ShredModel import java.time.{Instant, ZoneOffset} import java.time.format.DateTimeFormatter import com.snowplowanalytics.lrumap.LruMap -import com.snowplowanalytics.iglu.schemaddl.Properties package object transformation { private val Formatter = DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss.SSS") - type PropertiesKey = (SchemaListKey, StorageTime) - type PropertiesCache[F[_]] = LruMap[F, PropertiesKey, Properties] + type ShredModelCacheKey = (SchemaKey, StorageTime) + type ShredModelCache[F[_]] = LruMap[F, ShredModelCacheKey, ShredModel] implicit class InstantOps(time: Instant) { def formatted: String = diff --git a/modules/common/src/test/scala/com.snowplowanalytics.snowplow.rdbloader.common/CachedFlatteningSpec.scala b/modules/common/src/test/scala/com.snowplowanalytics.snowplow.rdbloader.common/CachedFlatteningSpec.scala deleted file mode 100644 index 2d12e6965..000000000 --- a/modules/common/src/test/scala/com.snowplowanalytics.snowplow.rdbloader.common/CachedFlatteningSpec.scala +++ /dev/null @@ -1,214 +0,0 @@ -/* - * Copyright (c) 2012-present Snowplow Analytics Ltd. All rights reserved. - * - * This program is licensed to you under the Snowplow Community License Version 1.0, - * and you may not use this file except in compliance with the Snowplow Community License Version 1.0. - * You may obtain a copy of the Snowplow Community License Version 1.0 at https://docs.snowplow.io/community-license-1.0 - */ -package com.snowplowanalytics.snowplow.rdbloader.common - -import cats.effect.Clock -import cats.{Applicative, Id, Monad} -import com.snowplowanalytics.iglu.client.resolver.Resolver -import com.snowplowanalytics.iglu.client.resolver.registries.{Registry, RegistryError, RegistryLookup} -import com.snowplowanalytics.iglu.core.{SchemaKey, SchemaList, SelfDescribingData} -import com.snowplowanalytics.iglu.schemaddl.Properties -import com.snowplowanalytics.lrumap.CreateLruMap -import com.snowplowanalytics.snowplow.rdbloader.common.transformation.{EventUtils, PropertiesCache, PropertiesKey} -import io.circe.Json -import io.circe.literal.JsonStringContext -import org.specs2.mutable.Specification - -import scala.concurrent.duration.{DurationInt, FiniteDuration} - -import java.util.concurrent.TimeUnit - -class CachedFlatteningSpec extends Specification { - - // single 'field1' field - val `original schema - 1 field`: Json = - json""" - { - "$$schema": "http://iglucentral.com/schemas/com.snowplowanalytics.self-desc/schema/jsonschema/1-0-0#", - "description": "Test schema 1", - "self": { - "vendor": "com.snowplowanalytics.snowplow", - "name": "test_schema", - "format": "jsonschema", - "version": "1-0-0" - }, - - "type": "object", - "properties": { - "field1": { "type": "string"} - } - } - """ - - // same key as schema1, but with additional `field2` field. - val `patched schema - 2 fields`: Json = - json""" - { - "$$schema": "http://iglucentral.com/schemas/com.snowplowanalytics.self-desc/schema/jsonschema/1-0-0#", - "description": "Test schema 1", - "self": { - "vendor": "com.snowplowanalytics.snowplow", - "name": "test_schema", - "format": "jsonschema", - "version": "1-0-0" - }, - - "type": "object", - "properties": { - "field1": { "type": "string" }, - "field2": { "type": "integer" } - } - } - """ - - val cacheTtl = 10.seconds - val dataToFlatten = json"""{ "field1": "1", "field2": 2 }""" - val schemaKey = "iglu:com.snowplowanalytics.snowplow/test_schema/jsonschema/1-0-0" - - // (vendor, name, model) - val propertiesKey = ("com.snowplowanalytics.snowplow", "test_schema", 1) - - "Cached properties during flattening should be in sync with cached schemas/lists in iglu client" >> { - - "(1) original schema only, 1 flatten call => 1 field flattened" in { - val propertiesCache = getCache - val result = flatten(propertiesCache, getResolver)( - currentTime = 1000, // ms - schemaInRegistry = `original schema - 1 field` - ) - - result must beEqualTo(List("1")) - - // Properties are cached after first call (1 second) - propertiesCache.get((propertiesKey, 1.second)) must beSome - } - - "(2) original schema is patched between calls, no delay => original schema is still cached => 1 field flattened" in { - val propertiesCache = getCache - val resolver = getResolver - - // first call - flatten(propertiesCache, resolver)( - currentTime = 1000, // ms - schemaInRegistry = `original schema - 1 field` - ) - - // second call, same time - val result = flatten(propertiesCache, resolver)( - currentTime = 1000, // ms - schemaInRegistry = `patched schema - 2 fields` // different schema with the same key! - ) - - // no data from patched schema - result must beEqualTo(List("1")) - - // Properties are cached after first call (1 second) - propertiesCache.get((propertiesKey, 1.second)) must beSome - } - - "(3) schema is patched, delay between flatten calls is less than cache TTL => original schema is still cached => 1 field flattened" in { - val propertiesCache = getCache - val resolver = getResolver - - // first call - flatten(propertiesCache, resolver)( - currentTime = 1000, // ms - schemaInRegistry = `original schema - 1 field` - ) - - // second call, 2s later, less than 10s TTL - val result = flatten(propertiesCache, resolver)( - currentTime = 3000, // ms - schemaInRegistry = `patched schema - 2 fields` // different schema with the same key! - ) - - // no data from patched schema - result must beEqualTo(List("1")) - - // Properties are cached after first call (1 second) - propertiesCache.get((propertiesKey, 1.second)) must beSome - - // Properties are not cached after second call (3 seconds) - propertiesCache.get((propertiesKey, 3.second)) must beNone - } - - "(4) schema is patched, delay between flatten calls is greater than cache TTL => original schema is expired => using patched schema => 2 field flattened" in { - val propertiesCache = getCache - val resolver = getResolver - - // first call - flatten(propertiesCache, resolver)( - currentTime = 1000, // ms - schemaInRegistry = `original schema - 1 field` - ) - - // second call, 12s later, greater than 10s TTL - val result = flatten(propertiesCache, resolver)( - currentTime = 13000, // ms - schemaInRegistry = `patched schema - 2 fields` // different schema with the same key! - ) - - // Cache content expired, patched schema is fetched => 2 fields flattened - result must beEqualTo(List("1", "2")) - - // Properties are cached after first call (1 second) - propertiesCache.get((propertiesKey, 1.second)) must beSome - - // Properties are cached after second call (13 seconds) - propertiesCache.get((propertiesKey, 13.second)) must beSome - } - } - - // Helper method to wire all test dependencies and execute EventUtils.flatten - private def flatten( - propertiesCache: PropertiesCache[Id], - resolver: Resolver[Id] - )( - currentTime: Long, - schemaInRegistry: Json - ): List[String] = { - - // To return value stored in the schemaInRegistry variable, passed registry is ignored - val testRegistryLookup: RegistryLookup[Id] = new RegistryLookup[Id] { - override def lookup(registry: Registry, schemaKey: SchemaKey): Id[Either[RegistryError, Json]] = - Right(schemaInRegistry) - - override def list( - registry: Registry, - vendor: String, - name: String, - model: Int - ): Id[Either[RegistryError, SchemaList]] = - Right(SchemaList(List(SchemaKey.fromUri("iglu:com.snowplowanalytics.snowplow/test_schema/jsonschema/1-0-0").right.get))) - } - - val staticClock: Clock[Id] = new Clock[Id] { - override def applicative: Applicative[Id] = Applicative[Id] - - override def monotonic: Id[FiniteDuration] = FiniteDuration(currentTime * 1000000, TimeUnit.NANOSECONDS) - - override def realTime: Id[FiniteDuration] = FiniteDuration(currentTime, TimeUnit.MILLISECONDS) - } - - val data = SelfDescribingData(schema = SchemaKey.fromUri(schemaKey).right.get, data = dataToFlatten) - - EventUtils.flatten(resolver, propertiesCache, data)(Monad[Id], staticClock, testRegistryLookup).value.right.get - } - - private def getCache: PropertiesCache[Id] = CreateLruMap[Id, PropertiesKey, Properties].create(100) - - private def getResolver: Resolver[Id] = - Resolver.init[Id]( - cacheSize = 10, - cacheTtl = Some(cacheTtl), - refs = Registry.Embedded( // not used in test as we fix returned schema in custom test RegistryLookup - Registry.Config("Test", 0, List.empty), - path = "/fake" - ) - ) -} diff --git a/modules/common/src/test/scala/com.snowplowanalytics.snowplow.rdbloader.common/CachedShredModelSpec.scala b/modules/common/src/test/scala/com.snowplowanalytics.snowplow.rdbloader.common/CachedShredModelSpec.scala new file mode 100644 index 000000000..db7320dd8 --- /dev/null +++ b/modules/common/src/test/scala/com.snowplowanalytics.snowplow.rdbloader.common/CachedShredModelSpec.scala @@ -0,0 +1,97 @@ +package com.snowplowanalytics.snowplow.rdbloader.common + +import cats.{Applicative, Id, Monad} +import cats.effect.Clock +import com.snowplowanalytics.iglu.client.resolver.registries.{Registry, RegistryError, RegistryLookup} +import com.snowplowanalytics.iglu.client.resolver.{Resolver, StorageTime} +import com.snowplowanalytics.iglu.core.{SchemaKey, SchemaList, SchemaVer} +import com.snowplowanalytics.iglu.schemaddl.redshift.ShredModel +import com.snowplowanalytics.lrumap.CreateLruMap +import com.snowplowanalytics.snowplow.rdbloader.common.transformation.{ShredModelCache, Transformed} +import io.circe.Json +import io.circe.literal.JsonStringContext +import org.specs2.mutable.Specification + +import scala.concurrent.duration.{DurationInt, FiniteDuration} + +import java.util.concurrent.TimeUnit + +class CachedShredModelSpec extends Specification { + + val schemaKey = SchemaKey("com.snowplowanalytics.snowplow", "test_schema", "jsonschema", SchemaVer.Full(1, 0, 0)) + + val data = json"""{ "field1": "1" }""" + + val schema100: Json = + json""" + { + "$$schema": "http://iglucentral.com/schemas/com.snowplowanalytics.self-desc/schema/jsonschema/1-0-0#", + "description": "Test schema 1", + "self": { + "vendor": "com.snowplowanalytics.snowplow", + "name": "test_schema", + "format": "jsonschema", + "version": "1-0-0" + }, + + "type": "object", + "properties": { + "field1": { "type": "string"} + } + } + """ + + "Cached shred model" should { + "original schema only, 1 field flattened" in { + + val cache = getCache + val result: ShredModel = helper(schemaKey, cache, 1000, schema100) + + result.jsonToStrings(data) must beEqualTo(List("1")) + cache.get((schemaKey, FiniteDuration(1000, TimeUnit.MILLISECONDS))) must beSome + } + } + private def helper( + schemaKey: SchemaKey, + shredModelCache: ShredModelCache[Id], + currentTime: Long, + schemaInRegistry: Json + ): ShredModel = { + + // To return value stored in the schemaInRegistry variable, passed registry is ignored + val testRegistryLookup: RegistryLookup[Id] = new RegistryLookup[Id] { + override def lookup(registry: Registry, schemaKey: SchemaKey): Id[Either[RegistryError, Json]] = + Right(schemaInRegistry) + + override def list( + registry: Registry, + vendor: String, + name: String, + model: Int + ): Id[Either[RegistryError, SchemaList]] = + Right(SchemaList(List(SchemaKey.fromUri("iglu:com.snowplowanalytics.snowplow/test_schema/jsonschema/1-0-0").right.get))) + } + + val clock: Clock[Id] = new Clock[Id] { + override def applicative: Applicative[Id] = Applicative[Id] + + override def monotonic: Id[FiniteDuration] = FiniteDuration(currentTime * 1000000, TimeUnit.NANOSECONDS) + + override def realTime: Id[FiniteDuration] = FiniteDuration(currentTime, TimeUnit.MILLISECONDS) + } + + Transformed.lookupShredModel[Id](schemaKey, shredModelCache, getResolver)(Monad[Id], clock, testRegistryLookup).value.right.get + } + + private def getCache: ShredModelCache[Id] = CreateLruMap[Id, (SchemaKey, StorageTime), ShredModel].create(100) + + private def getResolver: Resolver[Id] = + Resolver.init[Id]( + cacheSize = 10, + cacheTtl = Some(10.seconds), + refs = Registry.Embedded( // not used in test as we fix returned schema in custom test RegistryLookup + Registry.Config("Test", 0, List.empty), + path = "/fake" + ) + ) +} 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 696ddac64..765684a98 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 @@ -12,8 +12,8 @@ import cats.data.NonEmptyList import doobie.Fragment import doobie.implicits._ import io.circe.syntax._ -import com.snowplowanalytics.iglu.core.SchemaKey -import com.snowplowanalytics.iglu.schemaddl.migrations.{Migration, SchemaList} +import com.snowplowanalytics.iglu.core.{SchemaCriterion, SchemaKey} +import com.snowplowanalytics.iglu.schemaddl.redshift.ShredModel import com.snowplowanalytics.snowplow.rdbloader.azure.AzureBlobStorage import com.snowplowanalytics.snowplow.rdbloader.LoadStatements import com.snowplowanalytics.snowplow.rdbloader.common.cloud.BlobStorage.Folder @@ -39,15 +39,16 @@ object Databricks { override val requiresEventsColumns: Boolean = true - override def updateTable(migration: Migration): Block = - Block(Nil, Nil, Entity.Table(tgt.schema, SchemaKey(migration.vendor, migration.name, "jsonschema", migration.to))) + override def updateTable(shredModel: ShredModel.GoodModel, currentSchemaKey: SchemaKey): Block = + Block(Nil, Nil, Entity.Table(tgt.schema, currentSchemaKey, shredModel.tableName)) - override def extendTable(info: ShreddedType.Info): Option[Block] = None + override def extendTable(info: ShreddedType.Info): List[Block] = List.empty override def getLoadStatements( discovery: DataDiscovery, eventTableColumns: EventTableColumns, - i: Unit + i: Unit, + disableMigration: List[SchemaCriterion] ): LoadStatements = { val toCopy = columnsToCopyFromDiscoveredData(discovery) val toSkip = ColumnsToSkip(getEntityColumnsPresentInDbOnly(eventTableColumns, toCopy)) @@ -59,7 +60,8 @@ object Databricks { override def initQuery[F[_]: DAO: Monad]: F[Unit] = Monad[F].unit - override def createTable(schemas: SchemaList): Block = Block(Nil, Nil, Entity.Table(tgt.schema, schemas.latest.schemaKey)) + override def createTable(shredModel: ShredModel): Block = + throw new IllegalStateException("createTable should never be called for Databricks") override def getManifest: Statement = Statement.CreateTable( @@ -241,7 +243,7 @@ object Databricks { } private def columnsToCopyFromDiscoveredData(discovery: DataDiscovery): ColumnsToCopy = { - val shredTypeColumns = discovery.columns.map(ColumnName.apply) + val shredTypeColumns = discovery.wideColumns.map(ColumnName.apply) ColumnsToCopy(AtomicColumns.Columns ::: shredTypeColumns) } } 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 f1daf843f..2906f0318 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 @@ -8,6 +8,7 @@ package com.snowplowanalytics.snowplow.loader.databricks import cats.data.NonEmptyList +import com.snowplowanalytics.iglu.core.SchemaVer import com.snowplowanalytics.snowplow.rdbloader.common.LoaderMessage.TypesInfo.WideRow.WideRowFormat.PARQUET import com.snowplowanalytics.snowplow.rdbloader.discovery.{DataDiscovery, ShreddedType} import com.snowplowanalytics.snowplow.rdbloader.common.config.TransformerConfig.Compression @@ -42,10 +43,10 @@ class DatabricksSpec extends Specification { ).map(ColumnName) val shreddedTypes = List( - ShreddedType.Widerow(ShreddedType.Info(baseFolder, "com_acme", "aaa", 1, SnowplowEntity.SelfDescribingEvent)), - ShreddedType.Widerow(ShreddedType.Info(baseFolder, "com_acme", "ccc", 1, SnowplowEntity.SelfDescribingEvent)), - ShreddedType.Widerow(ShreddedType.Info(baseFolder, "com_acme", "yyy", 1, SnowplowEntity.Context)), - ShreddedType.Widerow(ShreddedType.Info(baseFolder, "com_acme", "zzz", 1, SnowplowEntity.Context)) + ShreddedType.Widerow(ShreddedType.Info(baseFolder, "com_acme", "aaa", SchemaVer.Full(1, 0, 0), SnowplowEntity.SelfDescribingEvent)), + ShreddedType.Widerow(ShreddedType.Info(baseFolder, "com_acme", "ccc", SchemaVer.Full(1, 0, 0), SnowplowEntity.SelfDescribingEvent)), + ShreddedType.Widerow(ShreddedType.Info(baseFolder, "com_acme", "yyy", SchemaVer.Full(1, 0, 0), SnowplowEntity.Context)), + ShreddedType.Widerow(ShreddedType.Info(baseFolder, "com_acme", "zzz", SchemaVer.Full(1, 0, 0), SnowplowEntity.Context)) ) val discovery = DataDiscovery( @@ -58,11 +59,12 @@ class DatabricksSpec extends Specification { "unstruct_event_com_acme_ccc_1", "contexts_com_acme_yyy_1", "contexts_com_acme_zzz_1" - ) + ), + Map.empty ) val results = target - .getLoadStatements(discovery, eventsColumns, ()) + .getLoadStatements(discovery, eventsColumns, (), Nil) .map(f => f(LoadAuthMethod.NoCreds)) results should be like { case NonEmptyList(Statement.EventsCopy(path, compression, columnsToCopy, columnsToSkip, _, _, _), Nil) => @@ -241,7 +243,7 @@ object DatabricksSpec { Config.Retries(Config.Strategy.Constant, None, 1.minute, None), Config.Retries(Config.Strategy.Constant, None, 1.minute, None), Config.Retries(Config.Strategy.Constant, None, 1.minute, None), - Config.FeatureFlags(addLoadTstampColumn = true), + Config.FeatureFlags(addLoadTstampColumn = true, disableMigration = Nil), exampleTelemetry ) diff --git a/modules/loader/src/main/resources/application.conf b/modules/loader/src/main/resources/application.conf index c3678cd17..fbd998e6d 100644 --- a/modules/loader/src/main/resources/application.conf +++ b/modules/loader/src/main/resources/application.conf @@ -45,7 +45,8 @@ "connectionIsValid": "30 seconds" }, "featureFlags": { - "addLoadTstampColumn": true + "addLoadTstampColumn": true, + "disableMigration": [] } "telemetry": { "disable": false 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 4a6272c70..87ebe0568 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 @@ -207,7 +207,8 @@ object Loader { for { start <- Clock[F].realTimeInstant _ <- discovery.origin.timestamps.min.map(t => Monitoring[F].periodicMetrics.setEarliestKnownUnloadedData(t)).sequence.void - result <- Load.load[F, C, I](setStageC, incrementAttemptsC, discovery, initQueryResult, target) + result <- + Load.load[F, C, I](setStageC, incrementAttemptsC, discovery, initQueryResult, target, config.featureFlags.disableMigration) attempts <- control.getAndResetAttempts _ <- result match { case Load.LoadSuccess(ingested) => diff --git a/modules/loader/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/cloud/JsonPathDiscovery.scala b/modules/loader/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/cloud/JsonPathDiscovery.scala index 67c8241ff..c1bab8f7f 100644 --- a/modules/loader/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/cloud/JsonPathDiscovery.scala +++ b/modules/loader/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/cloud/JsonPathDiscovery.scala @@ -47,7 +47,7 @@ object JsonPathDiscovery { * full valid s3 path (with `s3://` prefix) */ override def discoverJsonPath(jsonpathAssets: Option[BlobStorage.Folder], shreddedType: Info): DiscoveryAction[F, BlobStorage.Key] = { - val filename = s"""${toSnakeCase(shreddedType.name)}_${shreddedType.model}.json""" + val filename = s"""${toSnakeCase(shreddedType.name)}_${shreddedType.version.model}.json""" val key = s"${shreddedType.vendor}/$filename" Cache[F].getCache(key).flatMap { 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 b7a47dc4a..ec6dbb956 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 @@ -13,6 +13,7 @@ import cats.effect.Sync import cats.data.EitherT import cats.syntax.either._ import cats.syntax.option._ +import com.snowplowanalytics.iglu.core.SchemaCriterion import io.circe._ import io.circe.generic.semiauto._ import org.http4s.{ParseFailure, Uri} @@ -123,7 +124,7 @@ object Config { backoff: FiniteDuration, cumulativeBound: Option[FiniteDuration] ) - final case class FeatureFlags(addLoadTstampColumn: Boolean) + final case class FeatureFlags(addLoadTstampColumn: Boolean, disableMigration: List[SchemaCriterion]) sealed trait Strategy object Strategy { @@ -267,6 +268,9 @@ object Config { implicit val configDecoder: Decoder[Config[StorageTarget]] = deriveDecoder[Config[StorageTarget]].ensure(validateConfig) + implicit val disableMigrationConfigDecoder: Decoder[SchemaCriterion] = + Decoder[String].emap(s => SchemaCriterion.parse(s).toRight(s"[$s] is not a valid schema criterion")) + implicit val featureFlagsConfigDecoder: Decoder[FeatureFlags] = deriveDecoder[FeatureFlags] diff --git a/modules/loader/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/db/Migration.scala b/modules/loader/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/db/Migration.scala index 8b2648171..4766db2e2 100644 --- a/modules/loader/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/db/Migration.scala +++ b/modules/loader/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/db/Migration.scala @@ -10,16 +10,18 @@ package com.snowplowanalytics.snowplow.rdbloader.db import cats.data.EitherT import cats.implicits._ import cats.{Applicative, Monad, MonadThrow} -import com.snowplowanalytics.iglu.core.{SchemaKey, SchemaMap, SchemaVer} -import com.snowplowanalytics.iglu.schemaddl.StringUtils -import com.snowplowanalytics.iglu.schemaddl.migrations.{Migration => SchemaMigration, SchemaList} -import com.snowplowanalytics.snowplow.rdbloader.db.Columns.ColumnName -import com.snowplowanalytics.snowplow.rdbloader.discovery.{DataDiscovery, DiscoveryFailure, ShreddedType} +import com.snowplowanalytics.iglu.core.{SchemaCriterion, SchemaKey, SchemaVer} +import com.snowplowanalytics.iglu.core.SchemaKey.ordering +import com.snowplowanalytics.iglu.schemaddl.redshift._ +import com.snowplowanalytics.iglu.schemaddl.redshift.ShredModel.RecoveryModel +import com.snowplowanalytics.snowplow.rdbloader.discovery.{DataDiscovery, ShreddedType} import com.snowplowanalytics.snowplow.rdbloader.dsl.{DAO, Iglu, Logging, Transaction} import com.snowplowanalytics.snowplow.rdbloader.loading.EventsTable import com.snowplowanalytics.snowplow.rdbloader.{LoaderAction, LoaderError, readSchemaKey} import doobie.Fragment +import scala.math.Ordered.orderingToOrdered + /** * Sequences of DDL statement executions that have to be applied to a DB in order to make it * compatible with a certain `DataDiscovery` (batch of data) Unlike `Block`, which is set of @@ -79,7 +81,7 @@ object Migration { def getCommentOn: Statement.CommentOn = entity match { - case Entity.Table(_, schemaKey) => + case Entity.Table(_, schemaKey, _) => Statement.CommentOn(getName, schemaKey.toSchemaUri) case Entity.Column(info) => Statement.CommentOn(getName, info.getNameFull) @@ -95,7 +97,7 @@ object Migration { * Works with separate tables (both create and update) and does support migration (hence all * schema info) */ - final case class Table(schemaList: SchemaList) extends Description + final case class Table(mergeResult: MergeRedshiftSchemasResult) extends Description /** Works with only `events` table, creating a column for every new schema */ final case class WideRow(shreddedType: ShreddedType.Info) extends Description @@ -113,7 +115,7 @@ object Migration { * @note * since all [[Item]]s contain `Fragment` there's no safe `equals` operations */ - sealed trait Item { + sealed trait Item extends Product with Serializable { def statement: Statement } @@ -139,18 +141,25 @@ object Migration { final case class CreateTable(createTable: Fragment) extends Item { val statement: Statement = Statement.CreateTable(createTable) } + + /** COMMENT ON. Can be combined with [[AddColumn]] in [[Block]] */ + final case class CommentOn(tableName: String, comment: String) extends Item { + val statement: Statement = Statement.CommentOn(tableName, comment) + } } /** Inspect DB state and create a [[Migration]] object that contains all necessary actions */ def build[F[_]: Transaction[*[_], C]: MonadThrow: Iglu, C[_]: MonadThrow: Logging: DAO, I]( discovery: DataDiscovery, - target: Target[I] + target: Target[I], + disableMigration: List[SchemaCriterion] ): F[Migration[C]] = { val descriptions: LoaderAction[F, List[Description]] = discovery.shreddedTypes.filterNot(_.isAtomic).traverse { case s: ShreddedType.Tabular => - val ShreddedType.Info(_, vendor, name, model, _) = s.info - EitherT(Iglu[F].getSchemas(vendor, name, model)).map(Description.Table) + if (!disableMigration.contains(s.info.toCriterion)) + EitherT.rightT[F, LoaderError](Description.Table(discovery.shredModels(s.info.getSchemaKey))) + else EitherT.rightT[F, LoaderError](Description.NoMigration) case ShreddedType.Widerow(info) => EitherT.rightT[F, LoaderError](Description.WideRow(info)) case ShreddedType.Json(_, _) => @@ -159,11 +168,11 @@ object Migration { val transaction: C[Migration[C]] = Transaction[F, C].arrowBack(descriptions.value).flatMap { - case Right(schemaList) => + case Right(descriptionList) => // Duplicate schemas cause migration vector to double failing the second migration. Therefore deduplication // with toSet.toList - schemaList.toSet.toList - .traverseFilter(buildBlock[C, I](_, target)) + descriptionList.toSet.toList + .flatTraverse(buildBlock[C, I](_, target)) .flatMap(blocks => Migration.fromBlocks[C](blocks)) case Left(error) => MonadThrow[C].raiseError[Migration[C]](error) @@ -176,54 +185,50 @@ object Migration { def empty[F[_]: Applicative]: Migration[F] = Migration[F](Nil, Applicative[F].unit) - def buildBlock[F[_]: MonadThrow: DAO, I](description: Description, target: Target[I]): F[Option[Block]] = + implicit val ord: Ordering[SchemaKey] = ordering + + def createMissingRecoveryTables[F[_]: Monad: DAO, I]( + target: Target[I], + recoveryModels: Map[SchemaKey, RecoveryModel] + ): F[List[Block]] = + recoveryModels.toList + .traverseFilter[F, Block] { case (_, rm) => + Control.tableExists[F](rm.tableName).ifM(Applicative[F].pure(None), Applicative[F].pure(Some(target.createTable(rm)))) + } + + def updateGoodTable[F[_]: Monad: DAO, I]( + target: Target[I], + goodModel: ShredModel.GoodModel + ): F[List[Block]] = + for { + schemaKeyInTable <- getVersion[F](goodModel.tableName) + updateNeeded = schemaKeyInTable < goodModel.schemaKey + block <- if (updateNeeded) Monad[F].pure(List(target.updateTable(goodModel, schemaKeyInTable))) + else Monad[F].pure(Nil) + } yield block + + def buildBlock[F[_]: Monad: DAO, I](description: Description, target: Target[I]): F[List[Block]] = description match { - case Description.Table(schemas) => - val tableName = StringUtils.getTableName(schemas.latest) - - val migrate: F[Option[Block]] = for { - schemaKey <- getVersion[F](tableName) - matches = schemas.latest.schemaKey == schemaKey - block <- if (matches) emptyBlock[F] - else - Control.getColumns[F](tableName).flatMap { (columns: List[ColumnName]) => - migrateTable(target, schemaKey, columns, schemas) match { - case Left(migrationError) => MonadThrow[F].raiseError[Option[Block]](migrationError) - case Right(block) => MonadThrow[F].pure(block.some) - } - } - } yield block - - Control.tableExists[F](tableName).ifM(migrate, Monad[F].pure(target.createTable(schemas).some)) + case Description.Table(mergeResult) => + val goodModel = mergeResult.goodModel + + val migrate = for { + ugt <- updateGoodTable[F, I](target, goodModel) + createRecovery <- createMissingRecoveryTables[F, I](target, mergeResult.recoveryModels) + } yield ugt ::: createRecovery + + val createTables: F[List[Block]] = + for { + createGood <- Monad[F].pure(target.createTable(goodModel)) + createRecovery <- createMissingRecoveryTables[F, I](target, mergeResult.recoveryModels) + } yield createGood :: createRecovery + + Control.tableExists[F](goodModel.tableName).ifM(migrate, createTables) + case Description.WideRow(info) => Monad[F].pure(target.extendTable(info)) case Description.NoMigration => - Monad[F].pure(none[Block]) - } - - def migrateTable[I]( - target: Target[I], - current: SchemaKey, - columns: List[ColumnName], - schemaList: SchemaList - ): Either[LoaderError, Block] = - schemaList match { - case s: SchemaList.Full => - val migrations = s.extractSegments.map(SchemaMigration.fromSegment) - migrations.find(_.from == current.version) match { - case Some(schemaMigration) => - target.updateTable(schemaMigration).asRight - case None => - val message = - s"Table's schema key '${current.toSchemaUri}' cannot be found in fetched schemas $schemaList. Migration cannot be created" - DiscoveryFailure.IgluError(message).toLoaderError.asLeft - } - case s: SchemaList.Single => - val message = - s"Illegal State: updateTable called for a table with known single schema [${s.schema.self.schemaKey.toSchemaUri}]\ncolumns: ${columns - .map(_.value) - .mkString(", ")}\nstate: $schemaList" - LoaderError.MigrationError(message).asLeft + Monad[F].pure(Nil) } def fromBlocks[F[_]: Monad: DAO: Logging](blocks: List[Block]): F[Migration[F]] = @@ -256,9 +261,7 @@ object Migration { Logging[F].info(s"${b.getName} migration completed") migration.addInTransaction(inAction) - case (migration, b @ Block(pre, Nil, Entity.Table(_, _))) => - // Uses two-step pre-transaction, because first step can result in a closed connection if - // we catch and ignore an exception + case (migration, b @ Block(pre, Nil, Entity.Table(_, _, _))) => val preAction = Logging[F].info(s"Migrating ${b.getName} (pre-transaction)") *> pre.traverse_(item => DAO[F].executeUpdate(item.statement, DAO.Purpose.NonLoading).void) val commentAction = @@ -282,30 +285,29 @@ object Migration { items.traverse_(item => DAO[F].executeUpdate(item.statement, DAO.Purpose.NonLoading).void) else Monad[F].unit - def emptyBlock[F[_]: Monad]: F[Option[Block]] = - Monad[F].pure(None) - /** Find the latest schema version in the table and confirm that it is the latest in `schemas` */ def getVersion[F[_]: DAO](tableName: String): F[SchemaKey] = DAO[F].executeQuery[SchemaKey](Statement.GetVersion(tableName))(readSchemaKey) sealed trait Entity { def getName: String = this match { - case Entity.Table(dbSchema, schemaKey) => - val tableName = StringUtils.getTableName(SchemaMap(schemaKey)) - s"$dbSchema.$tableName" + case Entity.Table(schema, _, tableName) => s"$schema.$tableName" case Entity.Column(info) => info.getNameFull } def getInfo: SchemaKey = this match { - case Entity.Table(_, schemaKey) => schemaKey - case Entity.Column(info) => SchemaKey(info.vendor, info.name, "jsonschema", SchemaVer.Full(info.model, 0, 0)) + case Entity.Table(_, schemaKey, _) => schemaKey + case Entity.Column(info) => SchemaKey(info.vendor, info.name, "jsonschema", SchemaVer.Full(info.version.model, 0, 0)) } } object Entity { - final case class Table(dbSchema: String, schemaKey: SchemaKey) extends Entity + final case class Table( + schema: String, + schemaKey: SchemaKey, + tableName: String + ) extends Entity final case class Column(info: ShreddedType.Info) extends Entity } @@ -325,7 +327,7 @@ object Migration { .executeQueryList[String](Statement.GetColumns(EventsTable.MainName)) .map { columns => (entity: Entity) => entity match { - case Entity.Table(_, _) => false + case Entity.Table(_, _, _) => false case Entity.Column(info) => val f = !columns.map(_.toLowerCase).contains(info.getNameFull.toLowerCase) if (f) { 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 6880eb520..9bd0fa467 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 @@ -7,6 +7,7 @@ */ package com.snowplowanalytics.snowplow.rdbloader.db +import com.snowplowanalytics.iglu.schemaddl.redshift.ShredModel import com.snowplowanalytics.snowplow.rdbloader.common.LoaderMessage.TypesInfo import doobie.Fragment import com.snowplowanalytics.snowplow.rdbloader.common.LoaderMessage @@ -69,7 +70,9 @@ object Statement { case class ShreddedCopy( shreddedType: ShreddedType, compression: Compression, - loadAuthMethod: LoadAuthMethod + loadAuthMethod: LoadAuthMethod, + shredModel: ShredModel, + tableName: String ) extends Statement with Loading { def table: String = shreddedType.info.getName diff --git a/modules/loader/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/db/Target.scala b/modules/loader/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/db/Target.scala index 11379b4b0..b9c5ee157 100644 --- a/modules/loader/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/db/Target.scala +++ b/modules/loader/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/db/Target.scala @@ -8,8 +8,8 @@ package com.snowplowanalytics.snowplow.rdbloader.db import cats.Monad - -import com.snowplowanalytics.iglu.schemaddl.migrations.{Migration => SchemaMigration, SchemaList} +import com.snowplowanalytics.iglu.core.{SchemaCriterion, SchemaKey} +import com.snowplowanalytics.iglu.schemaddl.redshift.ShredModel import com.snowplowanalytics.snowplow.rdbloader.LoadStatements import com.snowplowanalytics.snowplow.rdbloader.db.Columns.EventTableColumns import com.snowplowanalytics.snowplow.rdbloader.db.Migration.Block @@ -43,7 +43,8 @@ trait Target[I] { def getLoadStatements( discovery: DataDiscovery, eventTableColumns: EventTableColumns, - initQueryResult: I + initQueryResult: I, + disableMigration: List[SchemaCriterion] ): LoadStatements /** Get DDL of a manifest table */ @@ -52,10 +53,13 @@ trait Target[I] { def getEventTable: Statement /** Generate a DB-specification migration Block for updating a *separate* table */ - def updateTable(migration: SchemaMigration): Block + def updateTable( + goodModel: ShredModel.GoodModel, + currentSchemaKey: SchemaKey + ): Block /** Create a table with columns dervived from list of Iglu schemas */ - def createTable(schemas: SchemaList): Block + def createTable(shredModel: ShredModel): Block /** Query to get necessary bits from the warehouse during initialization of the application */ def initQuery[F[_]: DAO: Monad]: F[I] @@ -64,7 +68,7 @@ trait Target[I] { * Add a new column into `events`, i.e. extend a wide row. Unlike `updateTable` it always operates * on `events` table */ - def extendTable(info: ShreddedType.Info): Option[Block] + def extendTable(info: ShreddedType.Info): List[Block] /** * Prepare a temporary table to be used for folder monitoring diff --git a/modules/loader/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/discovery/DataDiscovery.scala b/modules/loader/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/discovery/DataDiscovery.scala index 8095f49d4..8b0595a8e 100644 --- a/modules/loader/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/discovery/DataDiscovery.scala +++ b/modules/loader/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/discovery/DataDiscovery.scala @@ -10,6 +10,9 @@ package com.snowplowanalytics.snowplow.rdbloader.discovery import cats._ import cats.data.EitherT import cats.implicits._ +import com.snowplowanalytics.iglu.core.SchemaKey +import com.snowplowanalytics.iglu.core.SchemaKey.ordering +import com.snowplowanalytics.iglu.schemaddl.redshift.{MergeRedshiftSchemasResult, foldMapMergeRedshiftSchemas} import com.snowplowanalytics.snowplow.rdbloader.cloud.JsonPathDiscovery import com.snowplowanalytics.snowplow.rdbloader.common.LoaderMessage.TypesInfo import com.snowplowanalytics.snowplow.rdbloader.{DiscoveryStream, LoaderAction, LoaderError} @@ -22,6 +25,8 @@ import com.snowplowanalytics.snowplow.rdbloader.common.config.TransformerConfig. import com.snowplowanalytics.snowplow.rdbloader.discovery.DiscoveryFailure.IgluError import com.snowplowanalytics.snowplow.rdbloader.state.State +import scala.math.Ordered.orderingToOrdered + /** * Result of data discovery in transformer output folder It still exists mostly to fulfill legacy * batch-discovery behavior, once Loader entirely switched to RT architecture we can replace it with @@ -36,7 +41,8 @@ case class DataDiscovery( shreddedTypes: List[ShreddedType], compression: Compression, typesInfo: TypesInfo, - columns: List[String] + wideColumns: List[String], + shredModels: Map[SchemaKey, MergeRedshiftSchemasResult] ) { /** ETL id */ @@ -143,20 +149,44 @@ object DataDiscovery { for { types <- LoaderAction[F, List[ShreddedType]](types) - columns <- message.typesInfo match { - case TypesInfo.Shredded(_) => EitherT.pure[F, LoaderError](List.empty[String]) - case TypesInfo.WideRow(fileFormat, types) => - fileFormat match { - case WideRowFormat.JSON => EitherT.pure[F, LoaderError](List.empty[String]) - case WideRowFormat.PARQUET => - Iglu[F] - .fieldNamesFromTypes(types) - .leftMap(er => LoaderError.DiscoveryError(IgluError(s"Error inferring columns names $er"))) + nonAtomicTypes = types.distinct.filterNot(_.isAtomic) + wideColumns <- message.typesInfo match { + case TypesInfo.Shredded(_) => EitherT.pure[F, LoaderError](List.empty[String]) + case TypesInfo.WideRow(fileFormat, types) => + fileFormat match { + case WideRowFormat.JSON => EitherT.pure[F, LoaderError](List.empty[String]) + case WideRowFormat.PARQUET => + Iglu[F] + .fieldNamesFromTypes(types) + .leftMap(er => LoaderError.DiscoveryError(IgluError(s"Error inferring columns names $er"))) + } } - } - } yield DataDiscovery(message.base, types.distinct, message.compression, message.typesInfo, columns) + models <- getShredModels[F](nonAtomicTypes) + } yield DataDiscovery(message.base, types.distinct, message.compression, message.typesInfo, wideColumns, models) } + def getShredModels[F[_]: Monad: Iglu]( + nonAtomicTypes: List[ShreddedType] + ): EitherT[F, LoaderError, Map[SchemaKey, MergeRedshiftSchemasResult]] = { + val maxSchemaKeyPerTableName = getMaxSchemaKeyPerTableName(nonAtomicTypes) + nonAtomicTypes + .traverse { shreddedType => + EitherT(Iglu[F].getSchemasWithSameModel(shreddedType.info.getSchemaKey)).map { schemas => + val maxSchemaKey = maxSchemaKeyPerTableName(shreddedType.info.getName) + val filtered = schemas.filter(_.self.schemaKey <= maxSchemaKey).toNel.get + val mergeRedshiftSchemasResult = foldMapMergeRedshiftSchemas(filtered) + (shreddedType.info.getSchemaKey, mergeRedshiftSchemasResult) + } + } + .map(_.toMap) + } + + implicit val ord: Ordering[SchemaKey] = ordering + + /** Find the maximum SchemaKey for all table names in a given set of shredded types */ + def getMaxSchemaKeyPerTableName(shreddedTypes: List[ShreddedType]): Map[String, SchemaKey] = + shreddedTypes.groupBy(_.info.getName).mapValues(_.maxBy(_.info.version).info.getSchemaKey) + def logAndRaise[F[_]: MonadThrow: Logging](error: LoaderError): F[Option[WithOrigin]] = Logging[F].error(error)("A problem occurred in the loading of SQS message") *> MonadThrow[F].raiseError(error) diff --git a/modules/loader/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/discovery/ShreddedType.scala b/modules/loader/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/discovery/ShreddedType.scala index df731ea55..81f0e6970 100644 --- a/modules/loader/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/discovery/ShreddedType.scala +++ b/modules/loader/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/discovery/ShreddedType.scala @@ -9,11 +9,8 @@ package com.snowplowanalytics.snowplow.rdbloader.discovery import cats.Monad import cats.implicits._ - import com.snowplowanalytics.snowplow.analytics.scalasdk.SnowplowEvent - -import com.snowplowanalytics.iglu.core.SchemaCriterion - +import com.snowplowanalytics.iglu.core.{SchemaCriterion, SchemaKey, SchemaVer} import com.snowplowanalytics.snowplow.rdbloader.DiscoveryStep import com.snowplowanalytics.snowplow.rdbloader.cloud.JsonPathDiscovery import com.snowplowanalytics.snowplow.rdbloader.common.{Common, LoaderMessage} @@ -41,8 +38,8 @@ sealed trait ShreddedType { /** Check if this type is special atomic type */ def isAtomic = this match { - case ShreddedType.Tabular(ShreddedType.Info(_, vendor, name, model, _)) => - vendor == Common.AtomicSchema.vendor && name == Common.AtomicSchema.name && model == Common.AtomicSchema.version.model + case ShreddedType.Tabular(ShreddedType.Info(_, vendor, name, v, _)) => + vendor == Common.AtomicSchema.vendor && name == Common.AtomicSchema.name && v.model == Common.AtomicSchema.version.model && v.revision == Common.AtomicSchema.version.revision && v.addition == Common.AtomicSchema.version.addition case _ => false } @@ -64,7 +61,7 @@ object ShreddedType { */ final case class Json(info: Info, jsonPaths: BlobStorage.Key) extends ShreddedType { def getLoadPath: String = - s"${info.base}${Common.GoodPrefix}/vendor=${info.vendor}/name=${info.name}/format=json/model=${info.model}" + s"${info.base}${Common.GoodPrefix}/vendor=${info.vendor}/name=${info.name}/format=json/model=${info.version.model}/revision=${info.version.revision}/addition=${info.version.addition}" def show: String = s"${info.toCriterion.asString} ($jsonPaths)" } @@ -78,7 +75,7 @@ object ShreddedType { */ final case class Tabular(info: Info) extends ShreddedType { def getLoadPath: String = - s"${info.base}${Common.GoodPrefix}/vendor=${info.vendor}/name=${info.name}/format=tsv/model=${info.model}" + s"${info.base}${Common.GoodPrefix}/vendor=${info.vendor}/name=${info.name}/format=tsv/model=${info.version.model}/revision=${info.version.revision}/addition=${info.version.addition}" def show: String = s"${info.toCriterion.asString} TSV" } @@ -108,17 +105,18 @@ object ShreddedType { base: BlobStorage.Folder, vendor: String, name: String, - model: Int, + version: SchemaVer.Full, entity: LoaderMessage.SnowplowEntity ) { - def toCriterion: SchemaCriterion = SchemaCriterion(vendor, name, "jsonschema", model) + def getSchemaKey: SchemaKey = SchemaKey(vendor, name, "jsonschema", version) + def toCriterion: SchemaCriterion = SchemaCriterion(vendor, name, "jsonschema", version.model) /** Build valid table name for the shredded type */ def getName: String = - s"${toSnakeCase(vendor)}_${toSnakeCase(name)}_$model" + s"${toSnakeCase(vendor)}_${toSnakeCase(name)}_${version.model}" def getNameFull: String = - SnowplowEvent.transformSchema(entity.toSdkProperty, vendor, name, model) + SnowplowEvent.transformSchema(entity.toSdkProperty, vendor, name, version.model) } /** @@ -134,15 +132,15 @@ object ShreddedType { case t: TypesInfo.Shredded => t.types.traverse[F, DiscoveryStep[ShreddedType]] { case TypesInfo.Shredded.Type(schemaKey, TypesInfo.Shredded.ShreddedFormat.TSV, shredProperty) => - val info = Info(base, schemaKey.vendor, schemaKey.name, schemaKey.version.model, shredProperty) + val info = Info(base, schemaKey.vendor, schemaKey.name, schemaKey.version, shredProperty) (Tabular(info): ShreddedType).asRight[DiscoveryFailure].pure[F] case TypesInfo.Shredded.Type(schemaKey, TypesInfo.Shredded.ShreddedFormat.JSON, shredProperty) => - val info = Info(base, schemaKey.vendor, schemaKey.name, schemaKey.version.model, shredProperty) + val info = Info(base, schemaKey.vendor, schemaKey.name, schemaKey.version, shredProperty) Monad[F].map(JsonPathDiscovery[F].discoverJsonPath(jsonpathAssets, info))(_.map(Json(info, _))) } case t: TypesInfo.WideRow => t.types.traverse[F, DiscoveryStep[ShreddedType]] { case TypesInfo.WideRow.Type(schemaKey, shredProperty) => - val info = Info(base, schemaKey.vendor, schemaKey.name, schemaKey.version.model, shredProperty) + val info = Info(base, schemaKey.vendor, schemaKey.name, schemaKey.version, shredProperty) (Widerow(info): ShreddedType).asRight[DiscoveryFailure].pure[F] } } diff --git a/modules/loader/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/dsl/Iglu.scala b/modules/loader/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/dsl/Iglu.scala index 7ad58dfe8..e1df768c7 100644 --- a/modules/loader/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/dsl/Iglu.scala +++ b/modules/loader/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/dsl/Iglu.scala @@ -7,7 +7,7 @@ */ package com.snowplowanalytics.snowplow.rdbloader.dsl -import cats.data.EitherT +import cats.data.{EitherT, NonEmptyList} import cats.~> import cats.implicits._ import cats.effect.{Async, Resource} @@ -16,34 +16,34 @@ import io.circe.syntax._ import org.http4s.client.Client import com.snowplowanalytics.iglu.client.resolver.{Resolver, ResolverCache} import com.snowplowanalytics.iglu.client.resolver.registries.{Http4sRegistryLookup, RegistryLookup} -import com.snowplowanalytics.iglu.schemaddl.migrations.SchemaList +import com.snowplowanalytics.iglu.core.{SchemaKey, SchemaMap, SelfDescribingSchema} +import com.snowplowanalytics.iglu.schemaddl.jsonschema.Schema +import com.snowplowanalytics.iglu.schemaddl.IgluSchema import com.snowplowanalytics.snowplow.badrows.FailureDetails.LoaderIgluError import com.snowplowanalytics.snowplow.rdbloader.LoaderError import com.snowplowanalytics.snowplow.rdbloader.common.LoaderMessage.TypesInfo.WideRow import com.snowplowanalytics.snowplow.rdbloader.common._ -import com.snowplowanalytics.snowplow.rdbloader.common.transformation.Flattening import com.snowplowanalytics.snowplow.rdbloader.common.transformation.parquet.NonAtomicFieldsProvider +import com.snowplowanalytics.snowplow.rdbloader.common.SchemaProvider.SchemaWithKey import com.snowplowanalytics.snowplow.rdbloader.discovery.DiscoveryFailure trait Iglu[F[_]] { self => - /** Retrieve list of schemas from Iglu Server */ - def getSchemas( - vendor: String, - name: String, - model: Int - ): F[Either[LoaderError, SchemaList]] + /** + * Retrieve list of schemas which belongs to the same vendor-name-model combination with the given + * schema key + */ + def getSchemasWithSameModel( + schemaKey: SchemaKey + ): F[Either[LoaderError, NonEmptyList[IgluSchema]]] def fieldNamesFromTypes(types: List[WideRow.Type]): EitherT[F, LoaderIgluError, List[String]] def mapK[G[_]](arrow: F ~> G): Iglu[G] = new Iglu[G] { - /** Retrieve list of schemas from Iglu Server */ - override def getSchemas( - vendor: String, - name: String, - model: Int - ): G[Either[LoaderError, SchemaList]] = arrow(self.getSchemas(vendor, name, model)) + override def getSchemasWithSameModel( + schemaKey: SchemaKey + ): G[Either[LoaderError, NonEmptyList[IgluSchema]]] = arrow(self.getSchemasWithSameModel(schemaKey)) override def fieldNamesFromTypes(types: List[WideRow.Type]): EitherT[G, LoaderIgluError, List[String]] = self.fieldNamesFromTypes(types).mapK(arrow) @@ -68,23 +68,25 @@ object Iglu { Resource.eval(buildResolver).map { resolver => new Iglu[F] { - def getSchemas( - vendor: String, - name: String, - model: Int - ): F[Either[LoaderError, SchemaList]] = { - val attempt = Flattening.getOrdered[F](resolver, vendor, name, model) - attempt - .recoverWith { - case LoaderIgluError.SchemaListNotFound(_, error) if isInputError(error) => - attempt - } - .leftMap { resolutionError => - val message = s"Cannot get schemas for iglu:$vendor/$name/jsonschema/$model-*-* ${resolutionError.asJson.noSpaces}" - LoaderError.DiscoveryError(DiscoveryFailure.IgluError(message)) + override def getSchemasWithSameModel( + schemaKey: SchemaKey + ): F[Either[LoaderError, NonEmptyList[IgluSchema]]] = + SchemaProvider + .fetchSchemasWithSameModel(resolver, schemaKey) + .leftMap(igluErr => LoaderError.DiscoveryError(DiscoveryFailure.IgluError(igluErr.asJson.noSpaces))) + .flatMap { schemaWithKeyList => + EitherT + .fromOption[F][LoaderError, NonEmptyList[SchemaWithKey]]( + NonEmptyList.fromList(schemaWithKeyList), + LoaderError.DiscoveryError( + DiscoveryFailure.IgluError( + s"Empty response from resolver for ${schemaKey.toSchemaUri}" + ) + ) + ) + .map(_.map(swk => SelfDescribingSchema[Schema](SchemaMap(swk.schemaKey), swk.schema))) } .value - } def fieldNamesFromTypes(types: List[WideRow.Type]): EitherT[F, LoaderIgluError, List[String]] = NonAtomicFieldsProvider.fieldsFromTypes(resolver, types).map(_.map(_.field.name)) 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 f17bd0310..c92c0bbcb 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 @@ -11,6 +11,7 @@ import java.time.Instant import cats.{Monad, MonadThrow, Show} import cats.implicits._ import cats.effect.Clock +import com.snowplowanalytics.iglu.core.SchemaCriterion import retry.Sleep // This project @@ -86,14 +87,15 @@ object Load { incrementAttempt: C[Unit], discovery: DataDiscovery.WithOrigin, initQueryResult: I, - target: Target[I] + target: Target[I], + disableMigration: List[SchemaCriterion] ): F[LoadResult] = for { _ <- TargetCheck.prepareTarget[F, C] - migrations <- Migration.build[F, C, I](discovery.discovery, target) + migrations <- Migration.build[F, C, I](discovery.discovery, target, disableMigration) _ <- getPreTransactions(setStage, migrations.preTransaction, incrementAttempt).traverse_(Transaction[F, C].run(_)) result <- Transaction[F, C].transact { - getTransaction[C, I](setStage, discovery, initQueryResult, target)(migrations.inTransaction) + getTransaction[C, I](setStage, discovery, initQueryResult, target, disableMigration)(migrations.inTransaction) .onError { case _: Throwable => incrementAttempt } } } yield result @@ -121,7 +123,8 @@ object Load { setStage: Stage => F[Unit], discovery: DataDiscovery.WithOrigin, initQueryResult: I, - target: Target[I] + target: Target[I], + disableMigration: List[SchemaCriterion] )( inTransactionMigrations: F[Unit] ): F[LoadResult] = @@ -140,7 +143,7 @@ object Load { Logging[F].info(s"Loading transaction for ${discovery.origin.base} has started") *> setStage(Stage.MigrationIn) *> inTransactionMigrations *> - run[F, I](setLoading, discovery.discovery, initQueryResult, target) *> + run[F, I](setLoading, discovery.discovery, initQueryResult, target, disableMigration) *> setStage(Stage.Committing) *> Manifest.add[F](discovery.origin.toManifestItem) *> Manifest @@ -194,12 +197,13 @@ object Load { setLoading: String => F[Unit], discovery: DataDiscovery, initQueryResult: I, - target: Target[I] + target: Target[I], + disableMigration: List[SchemaCriterion] ): F[Unit] = for { _ <- Logging[F].info(s"Loading ${discovery.base}") existingEventTableColumns <- if (target.requiresEventsColumns) Control.getColumns[F](EventsTable.MainName) else Nil.pure[F] - _ <- target.getLoadStatements(discovery, existingEventTableColumns, initQueryResult).traverse_ { genStatement => + _ <- target.getLoadStatements(discovery, existingEventTableColumns, initQueryResult, disableMigration).traverse_ { genStatement => for { loadAuthMethod <- LoadAuthService[F].forLoadingEvents // statement must be generated as late as possible, to have fresh and valid credentials. 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 0699512a3..83f174408 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 @@ -218,7 +218,7 @@ object ConfigSpec { 1.hour ) val exampleInitRetries: Config.Retries = Config.Retries(Config.Strategy.Exponential, Some(3), 30.seconds, Some(1.hour)) - val exampleFeatureFlags: Config.FeatureFlags = Config.FeatureFlags(addLoadTstampColumn = true) + val exampleFeatureFlags: Config.FeatureFlags = Config.FeatureFlags(addLoadTstampColumn = true, disableMigration = Nil) val exampleCloud: Config.Cloud = Config.Cloud.AWS(exampleRegion, exampleMessageQueue) val exampleTelemetry = Telemetry.Config( diff --git a/modules/loader/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/discovery/DataDiscoverySpec.scala b/modules/loader/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/discovery/DataDiscoverySpec.scala index 84c07966d..782cadee7 100644 --- a/modules/loader/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/discovery/DataDiscoverySpec.scala +++ b/modules/loader/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/discovery/DataDiscoverySpec.scala @@ -11,7 +11,9 @@ import java.time.Instant import cats.data.NonEmptyList import cats.syntax.either._ import com.snowplowanalytics.iglu.core.SchemaVer.Full -import com.snowplowanalytics.iglu.core.{SchemaKey, SchemaVer} +import com.snowplowanalytics.iglu.core.{SchemaKey, SchemaMap, SchemaVer, SelfDescribingSchema} +import com.snowplowanalytics.iglu.schemaddl.jsonschema.Schema +import com.snowplowanalytics.iglu.schemaddl.redshift.foldMapMergeRedshiftSchemas import com.snowplowanalytics.snowplow.rdbloader.LoaderError import com.snowplowanalytics.snowplow.rdbloader.cloud.JsonPathDiscovery import com.snowplowanalytics.snowplow.rdbloader.common.LoaderMessage @@ -34,7 +36,7 @@ class DataDiscoverySpec extends Specification { BlobStorage.Folder.coerce("s3://my-bucket/my-path"), "com.acme", "event", - 2, + SchemaVer.Full(2, 0, 0), LoaderMessage.SnowplowEntity.SelfDescribingEvent ), BlobStorage.Key.coerce("s3://assets/event_1.json") @@ -44,7 +46,7 @@ class DataDiscoverySpec extends Specification { BlobStorage.Folder.coerce("s3://my-bucket/my-path"), "com.acme", "context", - 2, + SchemaVer.Full(2, 0, 0), LoaderMessage.SnowplowEntity.SelfDescribingEvent ), BlobStorage.Key.coerce("s3://assets/context_1.json") @@ -57,7 +59,8 @@ class DataDiscoverySpec extends Specification { shreddedTypes, Compression.Gzip, TypesInfo.Shredded(List.empty), - Nil + Nil, + Map.empty ) discovery.show must beEqualTo("""|my-path with following shredded types: | * iglu:com.acme/event/jsonschema/2-*-* (s3://assets/event_1.json) @@ -72,19 +75,37 @@ class DataDiscoverySpec extends Specification { implicit val aws: BlobStorage[Pure] = PureAWS.blobStorage(PureAWS.init) implicit val jsonPathDiscovery: JsonPathDiscovery[Pure] = JsonPathDiscovery.aws[Pure]("eu-central-1") + val s1 = ShreddedType.Tabular( + ShreddedType.Info( + BlobStorage.Folder.coerce("s3://bucket/folder/"), + "com.acme", + "event-a", + SchemaVer.Full(1, 0, 0), + LoaderMessage.SnowplowEntity.SelfDescribingEvent + ) + ) + val s2 = ShreddedType.Tabular( + ShreddedType.Info( + BlobStorage.Folder.coerce("s3://bucket/folder/"), + "com.acme", + "event-a", + SchemaVer.Full(1, 1, 0), + LoaderMessage.SnowplowEntity.SelfDescribingEvent + ) + ) + val shreddedTypes = List(s1, s2) + val shredModels = Map( + s1.info.getSchemaKey -> foldMapMergeRedshiftSchemas( + NonEmptyList.of(SelfDescribingSchema(SchemaMap(s1.info.getSchemaKey), Schema())) + ), + s2.info.getSchemaKey -> foldMapMergeRedshiftSchemas( + NonEmptyList.of(SelfDescribingSchema(SchemaMap(s2.info.getSchemaKey), Schema())) + ) + ) + val expected = DataDiscovery( BlobStorage.Folder.coerce("s3://bucket/folder/"), - List( - ShreddedType.Tabular( - ShreddedType.Info( - BlobStorage.Folder.coerce("s3://bucket/folder/"), - "com.acme", - "event-a", - 1, - LoaderMessage.SnowplowEntity.SelfDescribingEvent - ) - ) - ), + shreddedTypes, Compression.None, TypesInfo.Shredded( List( @@ -97,7 +118,8 @@ class DataDiscoverySpec extends Specification { ) ) ), - Nil + Nil, + shredModels ).asRight val result = DataDiscovery.fromLoaderMessage[Pure](None, DataDiscoverySpec.shreddingCompleteWithSameModel).value.runA @@ -161,30 +183,40 @@ class DataDiscoverySpec extends Specification { val message = DataDiscoverySpec.shreddingComplete + val s1 = ShreddedType.Json( + ShreddedType.Info( + BlobStorage.Folder.coerce("s3://bucket/folder/"), + "com.acme", + "event-a", + SchemaVer.Full(1, 0, 0), + LoaderMessage.SnowplowEntity.SelfDescribingEvent + ), + BlobStorage.Key.coerce("s3://snowplow-hosted-assets-eu-central-1/4-storage/redshift-storage/jsonpaths/com.acme/event_a_1.json") + ) + + val s2 = ShreddedType.Json( + ShreddedType.Info( + BlobStorage.Folder.coerce("s3://bucket/folder/"), + "com.acme", + "event-b", + SchemaVer.Full(1, 0, 0), + LoaderMessage.SnowplowEntity.SelfDescribingEvent + ), + BlobStorage.Key.coerce("s3://snowplow-hosted-assets-eu-central-1/4-storage/redshift-storage/jsonpaths/com.acme/event_b_1.json") + ) + + val shredModels = Map( + s1.info.getSchemaKey -> foldMapMergeRedshiftSchemas( + NonEmptyList.of(SelfDescribingSchema(SchemaMap(s1.info.getSchemaKey), Schema())) + ), + s2.info.getSchemaKey -> foldMapMergeRedshiftSchemas( + NonEmptyList.of(SelfDescribingSchema(SchemaMap(s2.info.getSchemaKey), Schema())) + ) + ) + val expected = DataDiscovery( BlobStorage.Folder.coerce("s3://bucket/folder/"), - List( - ShreddedType.Json( - ShreddedType.Info( - BlobStorage.Folder.coerce("s3://bucket/folder/"), - "com.acme", - "event-a", - 1, - LoaderMessage.SnowplowEntity.SelfDescribingEvent - ), - BlobStorage.Key.coerce("s3://snowplow-hosted-assets-eu-central-1/4-storage/redshift-storage/jsonpaths/com.acme/event_a_1.json") - ), - ShreddedType.Json( - ShreddedType.Info( - BlobStorage.Folder.coerce("s3://bucket/folder/"), - "com.acme", - "event-b", - 1, - LoaderMessage.SnowplowEntity.SelfDescribingEvent - ), - BlobStorage.Key.coerce("s3://snowplow-hosted-assets-eu-central-1/4-storage/redshift-storage/jsonpaths/com.acme/event_b_1.json") - ) - ), + List(s1, s2), Compression.Gzip, TypesInfo.Shredded( List( @@ -194,7 +226,8 @@ class DataDiscoverySpec extends Specification { .Type(SchemaKey("com.acme", "event-b", "jsonschema", Full(1, 0, 0)), JSON, LoaderMessage.SnowplowEntity.SelfDescribingEvent) ) ), - Nil + Nil, + shredModels ) val (state, result) = DataDiscovery.handle[Pure](None, message).run @@ -210,6 +243,8 @@ class DataDiscoverySpec extends Specification { LogEntry.Message( "PUT com.acme/event_b_1.json: Some(s3://snowplow-hosted-assets-eu-central-1/4-storage/redshift-storage/jsonpaths/com.acme/event_b_1.json)" ), + LogEntry.Message("Fetch iglu:com.acme/event-a/jsonschema/1-0-0"), + LogEntry.Message("Fetch iglu:com.acme/event-b/jsonschema/1-0-0"), LogEntry.Message( "New data discovery at folder with following shredded types: * iglu:com.acme/event-a/jsonschema/1-*-* (s3://snowplow-hosted-assets-eu-central-1/4-storage/redshift-storage/jsonpaths/com.acme/event_a_1.json) * iglu:com.acme/event-b/jsonschema/1-*-* (s3://snowplow-hosted-assets-eu-central-1/4-storage/redshift-storage/jsonpaths/com.acme/event_b_1.json)" ) diff --git a/modules/loader/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/discovery/TableTypeSpec.scala b/modules/loader/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/discovery/TableTypeSpec.scala index be1fd10db..ba660aa06 100644 --- a/modules/loader/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/discovery/TableTypeSpec.scala +++ b/modules/loader/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/discovery/TableTypeSpec.scala @@ -9,6 +9,7 @@ package com.snowplowanalytics.snowplow.rdbloader package discovery import cats.implicits._ +import com.snowplowanalytics.iglu.core.SchemaVer import com.snowplowanalytics.snowplow.rdbloader.cloud.JsonPathDiscovery import org.scalacheck.Gen import org.specs2.ScalaCheck @@ -34,7 +35,7 @@ class TableTypeSpec extends Specification with ScalaCheck { BlobStorage.Folder.coerce("s3://some-bucket/"), "com.acme", "entity", - 1, + SchemaVer.Full(1, 0, 0), LoaderMessage.SnowplowEntity.SelfDescribingEvent ) val discoveryAction = jsonPathDiscovery.discoverJsonPath(None, info) 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 684d68204..2861edabb 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 @@ -7,12 +7,16 @@ */ package com.snowplowanalytics.snowplow.rdbloader.loading +import cats.data.NonEmptyList + import java.time.Instant import java.sql.SQLException import scala.concurrent.duration.FiniteDuration import cats.syntax.option._ import cats.effect.Clock -import com.snowplowanalytics.iglu.core.{SchemaKey, SchemaVer} +import com.snowplowanalytics.iglu.core.{SchemaKey, SchemaMap, SchemaVer, SelfDescribingSchema} +import com.snowplowanalytics.iglu.schemaddl.jsonschema.Schema +import com.snowplowanalytics.iglu.schemaddl.redshift.{ShredModel, foldMapMergeRedshiftSchemas} import com.snowplowanalytics.snowplow.rdbloader.common.LoaderMessage import com.snowplowanalytics.snowplow.rdbloader.common.LoaderMessage.{Processor, Timestamps, TypesInfo} import com.snowplowanalytics.snowplow.rdbloader.common.config.TransformerConfig.Compression @@ -54,9 +58,16 @@ class LoadSpec extends Specification { implicit val loadAuthService: LoadAuthService[Pure] = PureLoadAuthService.interpreter val info = ShreddedType.Json( - ShreddedType.Info("s3://shredded/base/".dir, "com.acme", "json-context", 1, LoaderMessage.SnowplowEntity.SelfDescribingEvent), + ShreddedType.Info( + "s3://shredded/base/".dir, + "com.acme", + "json-context", + SchemaVer.Full(1, 0, 0), + LoaderMessage.SnowplowEntity.SelfDescribingEvent + ), "s3://assets/com.acme/json_context_1.json".key ) + val model = ShredModel.good(info.info.getSchemaKey, Schema()) val expected = List( PureTransaction.NoTransactionMessage, LogEntry.Sql(Statement.ReadyCheck), @@ -75,7 +86,7 @@ class LoadSpec extends Specification { () ) ), - LogEntry.Sql(Statement.ShreddedCopy(info, Compression.Gzip, LoadAuthMethod.NoCreds)), + LogEntry.Sql(Statement.ShreddedCopy(info, Compression.Gzip, LoadAuthMethod.NoCreds, model, model.tableName)), LogEntry.Sql(Statement.ManifestAdd(LoadSpec.dataDiscoveryWithOrigin.origin.toManifestItem)), LogEntry.Sql(Statement.ManifestGet("s3://shredded/base/".dir)), PureTransaction.CommitMessage @@ -87,7 +98,8 @@ class LoadSpec extends Specification { Pure.unit, LoadSpec.dataDiscoveryWithOrigin, (), - PureDAO.DummyTarget + PureDAO.DummyTarget, + Nil ) .runS @@ -118,7 +130,8 @@ class LoadSpec extends Specification { Pure.unit, LoadSpec.dataDiscoveryWithOrigin, (), - PureDAO.DummyTarget + PureDAO.DummyTarget, + Nil ) .runS @@ -150,7 +163,8 @@ class LoadSpec extends Specification { Pure.unit, LoadSpec.dataDiscoveryWithOrigin, (), - PureDAO.DummyTarget + PureDAO.DummyTarget, + Nil ) .runS @@ -167,9 +181,16 @@ class LoadSpec extends Specification { implicit val loadAuthService: LoadAuthService[Pure] = PureLoadAuthService.interpreter val info = ShreddedType.Json( - ShreddedType.Info("s3://shredded/base/".dir, "com.acme", "json-context", 1, LoaderMessage.SnowplowEntity.SelfDescribingEvent), + ShreddedType.Info( + "s3://shredded/base/".dir, + "com.acme", + "json-context", + SchemaVer.Full(1, 0, 0), + LoaderMessage.SnowplowEntity.SelfDescribingEvent + ), "s3://assets/com.acme/json_context_1.json".key ) + val model = ShredModel.good(info.info.getSchemaKey, Schema()) val expected = List( LogEntry.Message("TICK REALTIME"), PureTransaction.NoTransactionMessage, @@ -191,7 +212,7 @@ class LoadSpec extends Specification { () ) ), - LogEntry.Sql(Statement.ShreddedCopy(info, Compression.Gzip, LoadAuthMethod.NoCreds)), + LogEntry.Sql(Statement.ShreddedCopy(info, Compression.Gzip, LoadAuthMethod.NoCreds, model, model.tableName)), PureTransaction.RollbackMessage, LogEntry.Message("TICK REALTIME"), LogEntry.Message("SLEEP 30000000000 nanoseconds"), @@ -208,7 +229,7 @@ class LoadSpec extends Specification { () ) ), - LogEntry.Sql(Statement.ShreddedCopy(info, Compression.Gzip, LoadAuthMethod.NoCreds)), + LogEntry.Sql(Statement.ShreddedCopy(info, Compression.Gzip, LoadAuthMethod.NoCreds, model, model.tableName)), LogEntry.Sql(Statement.ManifestAdd(LoadSpec.dataDiscoveryWithOrigin.origin.toManifestItem)), LogEntry.Sql(Statement.ManifestGet("s3://shredded/base/".dir)), PureTransaction.CommitMessage @@ -219,7 +240,8 @@ class LoadSpec extends Specification { Pure.unit, LoadSpec.dataDiscoveryWithOrigin, (), - PureDAO.DummyTarget + PureDAO.DummyTarget, + Nil ) .runS @@ -259,7 +281,8 @@ class LoadSpec extends Specification { Pure.unit, LoadSpec.dataDiscoveryWithOrigin, (), - PureDAO.DummyTarget + PureDAO.DummyTarget, + Nil ) .runS @@ -269,23 +292,30 @@ class LoadSpec extends Specification { } object LoadSpec { + val shreddedType = ShreddedType.Json( + ShreddedType.Info( + BlobStorage.Folder.coerce("s3://shredded/base/"), + "com.acme", + "json-context", + SchemaVer.Full(1, 0, 0), + LoaderMessage.SnowplowEntity.SelfDescribingEvent + ), + BlobStorage.Key.coerce("s3://assets/com.acme/json_context_1.json") + ) + val shredModels = Map( + shreddedType.info.getSchemaKey -> foldMapMergeRedshiftSchemas( + NonEmptyList.of(SelfDescribingSchema(SchemaMap(shreddedType.info.getSchemaKey), Schema())) + ) + ) val dataDiscovery = DataDiscovery( BlobStorage.Folder.coerce("s3://shredded/base/"), List( - ShreddedType.Json( - ShreddedType.Info( - BlobStorage.Folder.coerce("s3://shredded/base/"), - "com.acme", - "json-context", - 1, - LoaderMessage.SnowplowEntity.SelfDescribingEvent - ), - BlobStorage.Key.coerce("s3://assets/com.acme/json_context_1.json") - ) + shreddedType ), Compression.Gzip, TypesInfo.Shredded(List.empty), - Nil + Nil, + shredModels ) val arn = "arn:aws:iam::123456789876:role/RedshiftLoadRole" diff --git a/modules/loader/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/test/PureDAO.scala b/modules/loader/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/test/PureDAO.scala index 7bb32111f..5f3156ea4 100644 --- a/modules/loader/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/test/PureDAO.scala +++ b/modules/loader/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/test/PureDAO.scala @@ -10,14 +10,9 @@ package com.snowplowanalytics.snowplow.rdbloader.test import cats.Monad import cats.data.NonEmptyList import cats.implicits._ - import doobie.{Fragment, Read} - -import com.snowplowanalytics.iglu.core.{SchemaKey, SchemaVer} -import com.snowplowanalytics.iglu.schemaddl.StringUtils -import com.snowplowanalytics.iglu.schemaddl.migrations.{FlatSchema, Migration => SchemaMigration, SchemaList} -import com.snowplowanalytics.iglu.schemaddl.redshift.generators.DdlGenerator - +import com.snowplowanalytics.iglu.core.{SchemaCriterion, SchemaKey, SchemaVer} +import com.snowplowanalytics.iglu.schemaddl.redshift.ShredModel import com.snowplowanalytics.snowplow.rdbloader.common.LoaderMessage.TypesInfo import com.snowplowanalytics.snowplow.rdbloader.LoadStatements import com.snowplowanalytics.snowplow.rdbloader.common.config.TransformerConfig.Compression @@ -89,7 +84,8 @@ object PureDAO { def getLoadStatements( discovery: DataDiscovery, eventTableColumns: EventTableColumns, - i: Unit + i: Unit, + disableMigration: List[SchemaCriterion] ): LoadStatements = NonEmptyList( loadAuthMethod => @@ -102,7 +98,14 @@ object PureDAO { loadAuthMethod, i ), - discovery.shreddedTypes.map(shredded => loadAuthMethod => Statement.ShreddedCopy(shredded, Compression.Gzip, loadAuthMethod)) + discovery.shreddedTypes.map { shredded => + val mergeResult = discovery.shredModels(shredded.info.getSchemaKey) + val shredModel = + mergeResult.recoveryModels.getOrElse(shredded.info.getSchemaKey, mergeResult.goodModel) + val isMigrationDisabled = disableMigration.contains(shredded.info.toCriterion) + val tableName = if (isMigrationDisabled) mergeResult.goodModel.tableName else shredModel.tableName + loadAuthMethod => Statement.ShreddedCopy(shredded, Compression.Gzip, loadAuthMethod, shredModel, tableName) + } ) def initQuery[F[_]: DAO: Monad]: F[Unit] = Monad[F].unit @@ -113,18 +116,18 @@ object PureDAO { override def getEventTable: Statement = Statement.CreateTable(Fragment.const0("CREATE events")) - def updateTable(migration: SchemaMigration): Migration.Block = - throw new RuntimeException("Not implemented in test suite") + def updateTable( + shredModel: ShredModel.GoodModel, + currentSchemaKey: SchemaKey + ): Migration.Block = + throw new Throwable("Not implemented in test suite") - def extendTable(info: ShreddedType.Info): Option[Block] = - throw new RuntimeException("Not implemented in test suite") + def extendTable(info: ShreddedType.Info): List[Block] = + throw new Throwable("Not implemented in test suite") - def createTable(schemas: SchemaList): Migration.Block = { - val subschemas = FlatSchema.extractProperties(schemas) - val tableName = StringUtils.getTableName(schemas.latest) - val createTable = DdlGenerator.generateTableDdl(subschemas, tableName, Some("public"), 4096, false) - val entity = Migration.Entity.Table("public", schemas.latest.schemaKey) - Block(Nil, List(Item.CreateTable(Fragment.const0(createTable.toDdl))), entity) + def createTable(shredModel: ShredModel): Migration.Block = { + val entity = Migration.Entity.Table("public", shredModel.schemaKey, shredModel.tableName) + Block(Nil, List(Item.CreateTable(Fragment.const0(shredModel.toTableSql("public")))), entity) } def requiresEventsColumns: Boolean = false diff --git a/modules/loader/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/test/PureIglu.scala b/modules/loader/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/test/PureIglu.scala index c83eabde8..612da3538 100644 --- a/modules/loader/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/test/PureIglu.scala +++ b/modules/loader/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/test/PureIglu.scala @@ -7,14 +7,13 @@ */ package com.snowplowanalytics.snowplow.rdbloader.test -import cats.data.EitherT +import cats.data.{EitherT, NonEmptyList} import cats.implicits._ import io.circe.literal._ -import com.snowplowanalytics.iglu.core.{SchemaKey, SchemaList, SchemaMap, SelfDescribingSchema} +import com.snowplowanalytics.iglu.core.{SchemaKey, SchemaMap, SelfDescribingSchema} import com.snowplowanalytics.iglu.schemaddl.IgluSchema import com.snowplowanalytics.iglu.schemaddl.jsonschema.Schema import com.snowplowanalytics.iglu.schemaddl.jsonschema.circe.implicits._ -import com.snowplowanalytics.iglu.schemaddl.migrations.{SchemaList => DSchemaList} import com.snowplowanalytics.snowplow.badrows.FailureDetails import com.snowplowanalytics.snowplow.rdbloader.LoaderError import com.snowplowanalytics.snowplow.rdbloader.common.LoaderMessage.SnowplowEntity @@ -24,16 +23,10 @@ import com.snowplowanalytics.snowplow.rdbloader.discovery.DiscoveryFailure object PureIglu { def interpreter: Iglu[Pure] = new Iglu[Pure] { - def getSchemas( - vendor: String, - name: String, - model: Int - ): Pure[Either[LoaderError, DSchemaList]] = - SchemaList - .parseStrings(List(s"iglu:$vendor/$name/jsonschema/$model-0-0")) - .map(x => DSchemaList.fromSchemaList(x, fetch).value) - .sequence[Pure, Either[String, DSchemaList]] - .map(e => e.flatten.leftMap(x => LoaderError.DiscoveryError(DiscoveryFailure.IgluError(x)))) + def getSchemasWithSameModel(schemaKey: SchemaKey): Pure[Either[LoaderError, NonEmptyList[IgluSchema]]] = + fetch(schemaKey).value + .map(_.map(NonEmptyList.one)) + .map(_.leftMap(err => LoaderError.DiscoveryError(DiscoveryFailure.IgluError(err)))) override def fieldNamesFromTypes(types: List[WideRow.Type]): EitherT[Pure, FailureDetails.LoaderIgluError, List[String]] = types.traverse { `type` => 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 f32228dbc..2b43a781d 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 @@ -8,29 +8,21 @@ package com.snowplowanalytics.snowplow.loader.redshift import java.sql.Timestamp - import cats.Monad import cats.data.NonEmptyList - +import cats.implicits._ +import com.snowplowanalytics.iglu.core.{SchemaCriterion, SchemaKey} import doobie.Fragment import doobie.implicits._ import doobie.implicits.javasql._ - import io.circe.syntax._ - -import com.snowplowanalytics.iglu.core.SchemaKey - -import com.snowplowanalytics.iglu.schemaddl.StringUtils -import com.snowplowanalytics.iglu.schemaddl.migrations.{FlatSchema, Migration, SchemaList} -import com.snowplowanalytics.iglu.schemaddl.redshift._ -import com.snowplowanalytics.iglu.schemaddl.redshift.generators.{DdlFile, DdlGenerator, MigrationGenerator} - +import com.snowplowanalytics.iglu.schemaddl.redshift.{ShredModel, ShredModelEntry} import com.snowplowanalytics.snowplow.rdbloader.LoadStatements import com.snowplowanalytics.snowplow.rdbloader.common.Common import com.snowplowanalytics.snowplow.rdbloader.common.config.TransformerConfig.Compression 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.Migration.{Block, Entity, Item} import com.snowplowanalytics.snowplow.rdbloader.db.{AtomicColumns, Manifest, Statement, Target} import com.snowplowanalytics.snowplow.rdbloader.cloud.authservice.LoadAuthService.LoadAuthMethod import com.snowplowanalytics.snowplow.rdbloader.discovery.{DataDiscovery, ShreddedType} @@ -53,39 +45,68 @@ object Redshift { override val requiresEventsColumns: Boolean = false - override def updateTable(migration: Migration): Block = { - val ddlFile = MigrationGenerator.generateMigration(migration, 4096, Some(schema)) - - val (preTransaction, inTransaction) = ddlFile.statements.foldLeft((NoPreStatements, NoStatements)) { - case ((preTransaction, inTransaction), statement) => - statement match { - case s @ AlterTable(_, _: AlterType) => - (Item.AlterColumn(Fragment.const0(s.toDdl)) :: preTransaction, inTransaction) - case s @ AlterTable(_, _) => - (preTransaction, Item.AddColumn(Fragment.const0(s.toDdl), ddlFile.warnings) :: inTransaction) - case _ => // We explicitly support only ALTER TABLE here; also drops BEGIN/END - (preTransaction, inTransaction) - } - } + override def updateTable( + goodModel: ShredModel.GoodModel, + currentSchemaKey: SchemaKey + ): Block = { + val outTransactions = goodModel.migrations.outTransaction(Some(currentSchemaKey), Some(goodModel.schemaKey)) + val inTransactions = goodModel.migrations.inTransaction(Some(currentSchemaKey), Some(goodModel.schemaKey)) + + val preTransaction = + outTransactions.map { varcharExtension => + Item.AlterColumn( + Fragment.const0( + s"""ALTER TABLE $schema.${goodModel.tableName} + | ALTER COLUMN "${varcharExtension.old.columnName}" TYPE ${varcharExtension.newEntry.columnType.show} + |""".stripMargin + ) + ) + } + + val inTransaction = + inTransactions.map { columnAddition => + Item.AddColumn( + Fragment.const0( + s"""ALTER TABLE $schema.${goodModel.tableName} + | ADD COLUMN "${columnAddition.column.columnName}" ${columnAddition.column.columnType.show} ${columnAddition.column.compressionEncoding.show} + |""".stripMargin + ), + Nil + ) + } - val target = SchemaKey(migration.vendor, migration.name, "jsonschema", migration.to) - Block(preTransaction.reverse, inTransaction.reverse, Entity.Table(schema, target)) + Block(preTransaction, inTransaction, Entity.Table(schema, goodModel.schemaKey, goodModel.tableName)) } - override def extendTable(info: ShreddedType.Info): Option[Block] = + override def extendTable(info: ShreddedType.Info): List[Block] = throw new IllegalStateException("Redshift Loader does not support loading wide row") override def getLoadStatements( discovery: DataDiscovery, eventTableColumns: EventTableColumns, - i: Unit + i: Unit, + disableMigration: List[SchemaCriterion] ): LoadStatements = { val shreddedStatements = discovery.shreddedTypes .filterNot(_.isAtomic) .groupBy(_.getLoadPath) .values .map(_.head) // So we get only one copy statement for given path - .map(shreddedType => loadAuthMethod => Statement.ShreddedCopy(shreddedType, discovery.compression, loadAuthMethod)) + .map { shreddedType => + val mergeResult = discovery.shredModels(shreddedType.info.getSchemaKey) + val shredModel = + mergeResult.recoveryModels.getOrElse(shreddedType.info.getSchemaKey, mergeResult.goodModel) + val isMigrationDisabled = disableMigration.contains(shreddedType.info.toCriterion) + val tableName = if (isMigrationDisabled) mergeResult.goodModel.tableName else shredModel.tableName + loadAuthMethod => + Statement.ShreddedCopy( + shreddedType, + discovery.compression, + loadAuthMethod, + shredModel, + tableName + ) + } .toList val atomic = { loadAuthMethod: LoadAuthMethod => @@ -104,15 +125,47 @@ object Redshift { override def initQuery[F[_]: DAO: Monad]: F[Unit] = Monad[F].unit - override def createTable(schemas: SchemaList): Block = { - val subschemas = FlatSchema.extractProperties(schemas) - val tableName = StringUtils.getTableName(schemas.latest) - val createTable = DdlGenerator.generateTableDdl(subschemas, tableName, Some(schema), 4096, false) - Block(Nil, List(Item.CreateTable(Fragment.const0(createTable.toDdl))), Entity.Table(schema, schemas.latest.schemaKey)) - } + override def createTable(shredModel: ShredModel): Block = + Block( + Nil, + List( + Item.CreateTable( + Fragment.const0( + s""" + |CREATE TABLE IF NOT EXISTS $schema.${shredModel.tableName} ( + |${shredModel.entries.show}, + | FOREIGN KEY (root_id) REFERENCES $schema.events(event_id) + |) + |DISTSTYLE KEY + |DISTKEY (root_id) + |SORTKEY (root_tstamp); + |""".stripMargin + ) + ) + ), + Entity.Table(schema, shredModel.schemaKey, shredModel.tableName) + ) override def getManifest: Statement = - Statement.CreateTable(Fragment.const0(getManifestDef(schema).render)) + Statement.CreateTable( + Fragment.const0( + s""" + |CREATE TABLE IF NOT EXISTS $schema.${Manifest.Name} ( + | "base" VARCHAR(512) ENCODE ZSTD NOT NULL PRIMARY KEY, + | "types" VARCHAR(65535) ENCODE ZSTD NOT NULL, + | "shredding_started" TIMESTAMP ENCODE ZSTD NOT NULL, + | "shredding_completed" TIMESTAMP ENCODE ZSTD NOT NULL, + | "min_collector_tstamp" TIMESTAMP ENCODE RAW NULL, + | "max_collector_tstamp" TIMESTAMP ENCODE ZSTD NULL, + | "ingestion_tstamp" TIMESTAMP ENCODE ZSTD NOT NULL, + | "compression" VARCHAR(16) ENCODE ZSTD NOT NULL, + | "processor_artifact" VARCHAR(64) ENCODE ZSTD NOT NULL, + | "processor_version" VARCHAR(32) ENCODE ZSTD NOT NULL, + | "count_good" INT ENCODE ZSTD NULL + |); + |""".stripMargin + ) + ) override def getEventTable: Statement = Statement.CreateTable( @@ -165,13 +218,15 @@ object Redshift { | ACCEPTINVCHARS | $frCompression""".stripMargin - case Statement.ShreddedCopy(shreddedType, compression, loadAuthMethod) => - val frTableName = Fragment.const0(qualify(shreddedType.info.getName)) + case Statement.ShreddedCopy(shreddedType, compression, loadAuthMethod, shredModel, tableName) => + val frTableName = Fragment.const0(qualify(tableName)) val frPath = Fragment.const0(shreddedType.getLoadPath) val frCredentials = loadAuthMethodFragment(loadAuthMethod, storage.roleArn) val frRegion = Fragment.const0(region.name) val frMaxError = Fragment.const0(maxError.toString) val frCompression = getCompressionFormat(compression) + val columns = shredModel.entries.map(_.columnName) + val frColumns = Fragment.const0((ShredModelEntry.commonColumnNames ::: columns).mkString(",")) shreddedType match { case ShreddedType.Json(_, jsonPathsFile) => @@ -186,7 +241,7 @@ object Redshift { | ACCEPTINVCHARS | $frCompression""".stripMargin case ShreddedType.Tabular(_) => - sql"""COPY $frTableName FROM '$frPath' + sql"""COPY $frTableName ($frColumns) FROM '$frPath' | DELIMITER '$EventFieldSeparator' | CREDENTIALS '$frCredentials' | REGION AS '$frRegion' @@ -224,9 +279,7 @@ object Redshift { WHERE nspname = ${schema}) AND relname = $tableName""".stripMargin case Statement.RenameTable(from, to) => - val ddl = DdlFile(List(AlterTable(qualify(from), RenameTo(to)))) - val str = ddl.render.split("\n").filterNot(l => l.startsWith("--") || l.isBlank).mkString("\n") - Fragment.const0(str) + Fragment.const0(s"ALTER TABLE ${qualify(from)} RENAME TO ${to}") case Statement.GetColumns(tableName) => sql"""SELECT column_name FROM information_schema.columns WHERE table_name = $tableName and table_schema = $schema""" @@ -257,8 +310,8 @@ object Redshift { case Statement.CreateTable(ddl) => ddl - case Statement.CommentOn(table, comment) => - Fragment.const0(CommentOn(qualify(table), comment).toDdl) + case Statement.CommentOn(tableName, comment) => + Fragment.const0(s"COMMENT ON TABLE $tableName IS '$comment'") case Statement.DdlFile(ddl) => ddl case Statement.AlterTable(ddl) => @@ -307,27 +360,4 @@ object Redshift { case _: LoadAuthMethod.TempCreds.Azure => throw new IllegalStateException("Azure temp credentials can't be used with Redshift") } - - val ManifestColumns = List( - Column("base", RedshiftVarchar(512), Set(CompressionEncoding(ZstdEncoding)), Set(Nullability(NotNull), KeyConstaint(PrimaryKey))), - Column("types", RedshiftVarchar(65535), Set(CompressionEncoding(ZstdEncoding)), Set(Nullability(NotNull))), - Column("shredding_started", RedshiftTimestamp, Set(CompressionEncoding(ZstdEncoding)), Set(Nullability(NotNull))), - Column("shredding_completed", RedshiftTimestamp, Set(CompressionEncoding(ZstdEncoding)), Set(Nullability(NotNull))), - Column("min_collector_tstamp", RedshiftTimestamp, Set(CompressionEncoding(RawEncoding)), Set(Nullability(Null))), - Column("max_collector_tstamp", RedshiftTimestamp, Set(CompressionEncoding(ZstdEncoding)), Set(Nullability(Null))), - Column("ingestion_tstamp", RedshiftTimestamp, Set(CompressionEncoding(ZstdEncoding)), Set(Nullability(NotNull))), - Column("compression", RedshiftVarchar(16), Set(CompressionEncoding(ZstdEncoding)), Set(Nullability(NotNull))), - Column("processor_artifact", RedshiftVarchar(64), Set(CompressionEncoding(ZstdEncoding)), Set(Nullability(NotNull))), - Column("processor_version", RedshiftVarchar(32), Set(CompressionEncoding(ZstdEncoding)), Set(Nullability(NotNull))), - Column("count_good", RedshiftInteger, Set(CompressionEncoding(ZstdEncoding)), Set(Nullability(Null))) - ) - - /** Add `schema` to otherwise static definition of manifest table */ - private def getManifestDef(schema: String): CreateTable = - CreateTable( - s"$schema.${Manifest.Name}", - ManifestColumns, - Set.empty, - Set(Diststyle(Key), DistKeyTable("base"), SortKeyTable(None, NonEmptyList.one("ingestion_tstamp"))) - ) } 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 aabd82757..89c7dbda3 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 @@ -7,9 +7,12 @@ */ package com.snowplowanalytics.snowplow.loader.redshift -import com.snowplowanalytics.iglu.core.{SchemaKey, SchemaVer} -import com.snowplowanalytics.iglu.schemaddl.migrations.{Migration => SchemaMigration, SchemaList} -import com.snowplowanalytics.iglu.schemaddl.redshift.{AddColumn, AlterTable, AlterType, CompressionEncoding, RedshiftVarchar, ZstdEncoding} +import cats.data.NonEmptyList +import com.snowplowanalytics.iglu.core.{SchemaKey, SchemaMap, SchemaVer, SelfDescribingSchema} +import com.snowplowanalytics.iglu.schemaddl.jsonschema.Schema +import com.snowplowanalytics.iglu.schemaddl.redshift.foldMapMergeRedshiftSchemas +import com.snowplowanalytics.snowplow.rdbloader.db.Statement +import com.snowplowanalytics.snowplow.rdbloader.test.TestState import com.snowplowanalytics.snowplow.rdbloader.db.{Migration, Target} import org.specs2.mutable.Specification import com.snowplowanalytics.snowplow.loader.redshift.db.MigrationSpec @@ -19,59 +22,96 @@ import com.snowplowanalytics.snowplow.rdbloader.common.LoaderMessage.SnowplowEnt import com.snowplowanalytics.snowplow.rdbloader.common.LoaderMessage.TypesInfo import com.snowplowanalytics.snowplow.rdbloader.common.cloud.BlobStorage.Folder import com.snowplowanalytics.snowplow.rdbloader.common.config.TransformerConfig.Compression +import com.snowplowanalytics.snowplow.rdbloader.db.Migration.Description import com.snowplowanalytics.snowplow.rdbloader.discovery.ShreddedType.{Info, Tabular} import com.snowplowanalytics.snowplow.rdbloader.discovery.DataDiscovery +import com.snowplowanalytics.snowplow.rdbloader.dsl.DAO +import com.snowplowanalytics.snowplow.rdbloader.test.Pure +import com.snowplowanalytics.snowplow.rdbloader.test.PureDAO +import com.snowplowanalytics.snowplow.rdbloader.test.PureOps class RedshiftSpec extends Specification { - import RedshiftSpec.redshift + import RedshiftSpec._ "updateTable" should { "create a Block with in-transaction migration" in { - val migration = MigrationSpec.schemaListTwo.asInstanceOf[SchemaList.Full].extractSegments.map(SchemaMigration.fromSegment).head - val result = redshift.updateTable(migration) - val alterTable = AlterTable( - "atomic.com_acme_context_1", - AddColumn("three", RedshiftVarchar(4096), None, Some(CompressionEncoding(ZstdEncoding)), None) - ) + implicit val dao: DAO[Pure] = PureDAO.interpreter(PureDAO.custom(jdbcResults)) + val description = Description.Table(foldMapMergeRedshiftSchemas(MigrationSpec.schemaListTwo)) + val (_, result) = Migration.buildBlock[Pure, Unit](description, redshift).run + + val expected = + """Fragment("ALTER TABLE atomic.com_acme_context_1 + | ADD COLUMN "three" VARCHAR(4096) ENCODE ZSTD + |")""".stripMargin result must beLike { - case Migration.Block( - Nil, - List(Migration.Item.AddColumn(fragment, Nil)), - Migration.Entity.Table("atomic", SchemaKey("com.acme", "context", "jsonschema", SchemaVer.Full(1, 0, 1))) - ) => - fragment.toString() must beEqualTo(s"""Fragment("${alterTable.toDdl}")""") - case _ => ko("Unexpected block found") + case Right(f :: Nil) => + f.preTransaction must haveSize(0) + f.inTransaction must haveSize(1) + f.inTransaction.head must beLike { + case Migration.Item.AddColumn(fragment, Nil) => fragment.toString() must beEqualTo(expected) + case i => ko(s"unexpected migration item: $i") + } + case Right(blocks) => ko(s"unexpected blocks: $blocks") + case Left(t) => ko(s"failed to build a block: $t") } } "create a Block with pre-transaction migration" in { - val migration = MigrationSpec.schemaListThree.asInstanceOf[SchemaList.Full].extractSegments.map(SchemaMigration.fromSegment).head - val result = redshift.updateTable(migration) - val alterTable = AlterTable( - "atomic.com_acme_context_2", - AlterType("one", RedshiftVarchar(64)) - ) + implicit val dao: DAO[Pure] = PureDAO.interpreter(PureDAO.custom(jdbcResults)) + val description = Description.Table(foldMapMergeRedshiftSchemas(MigrationSpec.schemaListThree)) + val (_, result) = Migration.buildBlock[Pure, Unit](description, redshift).run + + val expected = + """Fragment("ALTER TABLE atomic.com_acme_context_2 + | ALTER COLUMN "one" TYPE VARCHAR(64) + |")""".stripMargin result must beLike { - case Migration.Block( - List(Migration.Item.AlterColumn(fragment)), - List(), - Migration.Entity.Table("atomic", SchemaKey("com.acme", "context", "jsonschema", SchemaVer.Full(2, 0, 1))) - ) => - fragment.toString() must beEqualTo(s"""Fragment("${alterTable.toDdl}")""") - case _ => ko("Unexpected block found") + case Right(f :: Nil) => + f.preTransaction must haveSize(1) + f.preTransaction.head must beLike { + case Migration.Item.AlterColumn(fragment) => fragment.toString() must beEqualTo(expected) + case i => ko(s"unexpected migration item: $i") + } + f.inTransaction must haveSize(0) + case Right(blocks) => ko(s"unexpected blocks: $blocks") + case Left(t) => ko(s"failed to build a block: $t") } } "getLoadStatements should return one COPY per unique schema (vendor, name, model)" in { val shreddedTypes = List( - Info(vendor = "com.acme", name = "event", model = 2, entity = SelfDescribingEvent, base = Folder.coerce("s3://my-bucket/my-path")), - Info(vendor = "com.acme", name = "event", model = 2, entity = Context, base = Folder.coerce("s3://my-bucket/my-path")), - Info(vendor = "com.acme", name = "event", model = 3, entity = SelfDescribingEvent, base = Folder.coerce("s3://my-bucket/my-path")), - Info(vendor = "com.acme", name = "event", model = 3, entity = Context, base = Folder.coerce("s3://my-bucket/my-path")) + Info( + vendor = "com.acme", + name = "event", + version = SchemaVer.Full(2, 0, 0), + entity = SelfDescribingEvent, + base = Folder.coerce("s3://my-bucket/my-path") + ), + Info( + vendor = "com.acme", + name = "event", + version = SchemaVer.Full(2, 0, 0), + entity = Context, + base = Folder.coerce("s3://my-bucket/my-path") + ), + Info( + vendor = "com.acme", + name = "event", + version = SchemaVer.Full(3, 0, 0), + entity = SelfDescribingEvent, + base = Folder.coerce("s3://my-bucket/my-path") + ), + Info( + vendor = "com.acme", + name = "event", + version = SchemaVer.Full(3, 0, 0), + entity = Context, + base = Folder.coerce("s3://my-bucket/my-path") + ) ).map(Tabular) val discovery = DataDiscovery( @@ -79,19 +119,24 @@ class RedshiftSpec extends Specification { shreddedTypes, Compression.None, TypesInfo.Shredded(List.empty), - Nil + Nil, + shreddedTypes.map { s => + s.info.getSchemaKey -> foldMapMergeRedshiftSchemas( + NonEmptyList.of(SelfDescribingSchema(SchemaMap(s.info.getSchemaKey), Schema())) + ) + }.toMap ) val result = redshift - .getLoadStatements(discovery, List.empty, ()) + .getLoadStatements(discovery, List.empty, (), Nil) .map(f => f(LoadAuthService.LoadAuthMethod.NoCreds).title) result.size must beEqualTo(3) result.toList must containTheSameElementsAs( List( "COPY events FROM s3://my-bucket/my-path/", // atomic - "COPY com_acme_event_2 FROM s3://my-bucket/my-path/output=good/vendor=com.acme/name=event/format=tsv/model=2", - "COPY com_acme_event_3 FROM s3://my-bucket/my-path/output=good/vendor=com.acme/name=event/format=tsv/model=3" + "COPY com_acme_event_2 FROM s3://my-bucket/my-path/output=good/vendor=com.acme/name=event/format=tsv/model=2/revision=0/addition=0", + "COPY com_acme_event_3 FROM s3://my-bucket/my-path/output=good/vendor=com.acme/name=event/format=tsv/model=3/revision=0/addition=0" ) ) } @@ -99,6 +144,18 @@ class RedshiftSpec extends Specification { } object RedshiftSpec { + def jdbcResults(state: TestState)(statement: Statement): Any = { + val _ = state + statement match { + case Statement.GetVersion(_) => SchemaKey("com.acme", "context", "jsonschema", SchemaVer.Full(1, 0, 0)) + case Statement.TableExists(_) => true + case Statement.GetColumns(_) => List("some_column") + case Statement.ManifestGet(_) => List() + case Statement.ReadyCheck => 1 + case _ => throw new IllegalArgumentException(s"Unexpected statement $statement with ${state.getLog}") + } + } + val redshift: Target[Unit] = Redshift.build(validConfig).getOrElse(throw new RuntimeException("Invalid config")) } diff --git a/modules/redshift-loader/src/test/scala/com/snowplowanalytics/snowplow/loader/redshift/db/MigrationSpec.scala b/modules/redshift-loader/src/test/scala/com/snowplowanalytics/snowplow/loader/redshift/db/MigrationSpec.scala index 7baac6d87..4ca42768d 100644 --- a/modules/redshift-loader/src/test/scala/com/snowplowanalytics/snowplow/loader/redshift/db/MigrationSpec.scala +++ b/modules/redshift-loader/src/test/scala/com/snowplowanalytics/snowplow/loader/redshift/db/MigrationSpec.scala @@ -8,25 +8,18 @@ package com.snowplowanalytics.snowplow.loader.redshift.db import cats.data.NonEmptyList - import doobie.Fragment - import com.snowplowanalytics.iglu.core.{SchemaKey, SchemaMap, SchemaVer, SelfDescribingSchema} - import com.snowplowanalytics.iglu.schemaddl.jsonschema.Schema import com.snowplowanalytics.iglu.schemaddl.jsonschema.properties.{CommonProperties, ObjectProperty, StringProperty} -import com.snowplowanalytics.iglu.schemaddl.migrations.SchemaList -import com.snowplowanalytics.iglu.schemaddl.migrations.SchemaList.ModelGroupSet -import com.snowplowanalytics.iglu.schemaddl.redshift._ +import com.snowplowanalytics.iglu.schemaddl.redshift.foldMapMergeRedshiftSchemas import com.snowplowanalytics.snowplow.rdbloader.common.LoaderMessage.{SnowplowEntity, TypesInfo} import com.snowplowanalytics.snowplow.rdbloader.common.cloud.BlobStorage import com.snowplowanalytics.snowplow.rdbloader.db.{Migration, Statement} import com.snowplowanalytics.snowplow.rdbloader.discovery.{DataDiscovery, ShreddedType} import com.snowplowanalytics.snowplow.rdbloader.dsl.{DAO, Iglu, Logging, Transaction} import com.snowplowanalytics.snowplow.rdbloader.common.config.TransformerConfig.Compression - import org.specs2.mutable.Specification - import com.snowplowanalytics.snowplow.rdbloader.test.TestState.LogEntry import com.snowplowanalytics.snowplow.rdbloader.test.{Pure, PureDAO, PureIglu, PureLogging, PureTransaction} @@ -37,62 +30,77 @@ class MigrationSpec extends Specification { implicit val dao: DAO[Pure] = PureDAO.interpreter(PureDAO.init) implicit val iglu: Iglu[Pure] = PureIglu.interpreter implicit val logging: Logging[Pure] = PureLogging.interpreter() - val types = - List( - ShreddedType.Tabular( - ShreddedType.Info( - BlobStorage.Folder.coerce("s3://shredded/archive"), - "com.acme", - "some_context", - 2, - SnowplowEntity.Context - ) - ), - ShreddedType.Json( - ShreddedType.Info( - BlobStorage.Folder.coerce("s3://shredded/archive"), - "com.acme", - "some_event", - 1, - SnowplowEntity.Context - ), - BlobStorage.Key.coerce("s3://shredded/jsonpaths") - ) + val s1 = ShreddedType.Tabular( + ShreddedType.Info( + BlobStorage.Folder.coerce("s3://shredded/archive"), + "com.acme", + "some_context", + SchemaVer.Full(2, 0, 0), + SnowplowEntity.Context ) - val input = - DataDiscovery(BlobStorage.Folder.coerce("s3://shredded/archive"), types, Compression.Gzip, TypesInfo.Shredded(List.empty), Nil) - - val create = CreateTable( - "public.com_acme_some_context_2", - List( - Column("schema_vendor", RedshiftVarchar(128), Set(CompressionEncoding(ZstdEncoding)), Set(Nullability(NotNull))), - Column("schema_name", RedshiftVarchar(128), Set(CompressionEncoding(ZstdEncoding)), Set(Nullability(NotNull))), - Column("schema_format", RedshiftVarchar(128), Set(CompressionEncoding(ZstdEncoding)), Set(Nullability(NotNull))), - Column("schema_version", RedshiftVarchar(128), Set(CompressionEncoding(ZstdEncoding)), Set(Nullability(NotNull))), - Column("root_id", RedshiftChar(36), Set(CompressionEncoding(RawEncoding)), Set(Nullability(NotNull))), - Column("root_tstamp", RedshiftTimestamp, Set(CompressionEncoding(ZstdEncoding)), Set(Nullability(NotNull))), - Column("ref_root", RedshiftVarchar(255), Set(CompressionEncoding(ZstdEncoding)), Set(Nullability(NotNull))), - Column("ref_tree", RedshiftVarchar(1500), Set(CompressionEncoding(ZstdEncoding)), Set(Nullability(NotNull))), - Column("ref_parent", RedshiftVarchar(255), Set(CompressionEncoding(ZstdEncoding)), Set(Nullability(NotNull))) + ) + val s2 = ShreddedType.Json( + ShreddedType.Info( + BlobStorage.Folder.coerce("s3://shredded/archive"), + "com.acme", + "some_event", + SchemaVer.Full(1, 0, 0), + SnowplowEntity.Context ), - Set(ForeignKeyTable(NonEmptyList.one("root_id"), RefTable("public.events", Some("event_id")))), - Set(Diststyle(Key), DistKeyTable("root_id"), SortKeyTable(None, NonEmptyList.one("root_tstamp"))) + BlobStorage.Key.coerce("s3://shredded/jsonpaths") ) + val types = List(s1, s2) + val shredModels = Map( + s1.info.getSchemaKey -> foldMapMergeRedshiftSchemas( + NonEmptyList.of(SelfDescribingSchema(SchemaMap(s1.info.getSchemaKey), Schema())) + ), + s2.info.getSchemaKey -> foldMapMergeRedshiftSchemas( + NonEmptyList.of(SelfDescribingSchema(SchemaMap(s2.info.getSchemaKey), Schema())) + ) + ) + val input = + DataDiscovery( + BlobStorage.Folder.coerce("s3://shredded/archive"), + types, + Compression.Gzip, + TypesInfo.Shredded(List.empty), + Nil, + shredModels + ) + + val createToDdl = + """CREATE TABLE IF NOT EXISTS public.com_acme_some_context_2 ( + | "schema_vendor" VARCHAR(128) ENCODE ZSTD NOT NULL, + | "schema_name" VARCHAR(128) ENCODE ZSTD NOT NULL, + | "schema_format" VARCHAR(128) ENCODE ZSTD NOT NULL, + | "schema_version" VARCHAR(128) ENCODE ZSTD NOT NULL, + | "root_id" CHAR(36) ENCODE RAW NOT NULL, + | "root_tstamp" TIMESTAMP ENCODE ZSTD NOT NULL, + | "ref_root" VARCHAR(255) ENCODE ZSTD NOT NULL, + | "ref_tree" VARCHAR(1500) ENCODE ZSTD NOT NULL, + | "ref_parent" VARCHAR(255) ENCODE ZSTD NOT NULL, + | FOREIGN KEY (root_id) REFERENCES public.events(event_id) + |) + |DISTSTYLE KEY + |DISTKEY (root_id) + |SORTKEY (root_tstamp); + | + |COMMENT ON TABLE public.com_acme_some_context_2 IS 'iglu:com.acme/some_context/jsonschema/2-0-0'; + |""".stripMargin val expected = List( PureTransaction.NoTransactionMessage, - LogEntry.Message("Fetch iglu:com.acme/some_context/jsonschema/2-0-0"), LogEntry.Sql(Statement.TableExists("com_acme_some_context_2")) ) val expectedMigration = List( LogEntry.Message("Creating public.com_acme_some_context_2 table for iglu:com.acme/some_context/jsonschema/2-0-0"), - LogEntry.Sql(Statement.CreateTable(Fragment.const0(create.toDdl))), + LogEntry.Sql(Statement.CreateTable(Fragment.const0(createToDdl))), LogEntry.Sql(Statement.CommentOn("public.com_acme_some_context_2", "iglu:com.acme/some_context/jsonschema/2-0-0")), LogEntry.Message("Table created") ) - val (state, value) = Migration.build[Pure, Pure, Unit](input, PureDAO.DummyTarget).run + val (state, value) = Migration.build[Pure, Pure, Unit](input, PureDAO.DummyTarget, Nil).run state.getLog must beEqualTo(expected) value must beRight.like { case Migration(preTransaction, inTransaction) => @@ -107,61 +115,79 @@ class MigrationSpec extends Specification { implicit val iglu: Iglu[Pure] = PureIglu.interpreter implicit val logging: Logging[Pure] = PureLogging.interpreter() - val types = - List( - ShreddedType.Tabular( - ShreddedType.Info( - BlobStorage.Folder.coerce("s3://shredded/archive"), - "com.snowplowanalytics.snowplow", - "atomic", - 1, - SnowplowEntity.Context - ) - ), - ShreddedType.Tabular( - ShreddedType.Info( - BlobStorage.Folder.coerce("s3://shredded/archive"), - "com.acme", - "some_event", - 1, - SnowplowEntity.Context - ) - ) + val s1 = ShreddedType.Tabular( + ShreddedType.Info( + BlobStorage.Folder.coerce("s3://shredded/archive"), + "com.snowplowanalytics.snowplow", + "atomic", + SchemaVer.Full(1, 0, 0), + SnowplowEntity.Context ) - val input = - DataDiscovery(BlobStorage.Folder.coerce("s3://shredded/archive"), types, Compression.Gzip, TypesInfo.Shredded(List.empty), Nil) - - val create = CreateTable( - "public.com_acme_some_event_1", - List( - Column("schema_vendor", RedshiftVarchar(128), Set(CompressionEncoding(ZstdEncoding)), Set(Nullability(NotNull))), - Column("schema_name", RedshiftVarchar(128), Set(CompressionEncoding(ZstdEncoding)), Set(Nullability(NotNull))), - Column("schema_format", RedshiftVarchar(128), Set(CompressionEncoding(ZstdEncoding)), Set(Nullability(NotNull))), - Column("schema_version", RedshiftVarchar(128), Set(CompressionEncoding(ZstdEncoding)), Set(Nullability(NotNull))), - Column("root_id", RedshiftChar(36), Set(CompressionEncoding(RawEncoding)), Set(Nullability(NotNull))), - Column("root_tstamp", RedshiftTimestamp, Set(CompressionEncoding(ZstdEncoding)), Set(Nullability(NotNull))), - Column("ref_root", RedshiftVarchar(255), Set(CompressionEncoding(ZstdEncoding)), Set(Nullability(NotNull))), - Column("ref_tree", RedshiftVarchar(1500), Set(CompressionEncoding(ZstdEncoding)), Set(Nullability(NotNull))), - Column("ref_parent", RedshiftVarchar(255), Set(CompressionEncoding(ZstdEncoding)), Set(Nullability(NotNull))) + ) + + val s2 = ShreddedType.Tabular( + ShreddedType.Info( + BlobStorage.Folder.coerce("s3://shredded/archive"), + "com.acme", + "some_event", + SchemaVer.Full(1, 0, 0), + SnowplowEntity.Context + ) + ) + + val shredModels = Map( + s1.info.getSchemaKey -> foldMapMergeRedshiftSchemas( + NonEmptyList.of(SelfDescribingSchema(SchemaMap(s1.info.getSchemaKey), Schema())) ), - Set(ForeignKeyTable(NonEmptyList.one("root_id"), RefTable("public.events", Some("event_id")))), - Set(Diststyle(Key), DistKeyTable("root_id"), SortKeyTable(None, NonEmptyList.one("root_tstamp"))) + s2.info.getSchemaKey -> foldMapMergeRedshiftSchemas( + NonEmptyList.of(SelfDescribingSchema(SchemaMap(s2.info.getSchemaKey), Schema())) + ) ) + val types = List(s1, s2) + val input = + DataDiscovery( + BlobStorage.Folder.coerce("s3://shredded/archive"), + types, + Compression.Gzip, + TypesInfo.Shredded(List.empty), + Nil, + shredModels + ) + + val createToDdl = + """CREATE TABLE IF NOT EXISTS public.com_acme_some_event_1 ( + | "schema_vendor" VARCHAR(128) ENCODE ZSTD NOT NULL, + | "schema_name" VARCHAR(128) ENCODE ZSTD NOT NULL, + | "schema_format" VARCHAR(128) ENCODE ZSTD NOT NULL, + | "schema_version" VARCHAR(128) ENCODE ZSTD NOT NULL, + | "root_id" CHAR(36) ENCODE RAW NOT NULL, + | "root_tstamp" TIMESTAMP ENCODE ZSTD NOT NULL, + | "ref_root" VARCHAR(255) ENCODE ZSTD NOT NULL, + | "ref_tree" VARCHAR(1500) ENCODE ZSTD NOT NULL, + | "ref_parent" VARCHAR(255) ENCODE ZSTD NOT NULL, + | FOREIGN KEY (root_id) REFERENCES public.events(event_id) + |) + |DISTSTYLE KEY + |DISTKEY (root_id) + |SORTKEY (root_tstamp); + | + |COMMENT ON TABLE public.com_acme_some_event_1 IS 'iglu:com.acme/some_event/jsonschema/1-0-0'; + |""".stripMargin + val expected = List( PureTransaction.NoTransactionMessage, - LogEntry.Message("Fetch iglu:com.acme/some_event/jsonschema/1-0-0"), LogEntry.Sql(Statement.TableExists("com_acme_some_event_1")) ) val expectedMigration = List( LogEntry.Message("Creating public.com_acme_some_event_1 table for iglu:com.acme/some_event/jsonschema/1-0-0"), - LogEntry.Sql(Statement.CreateTable(Fragment.const0(create.toDdl))), + LogEntry.Sql(Statement.CreateTable(Fragment.const0(createToDdl))), LogEntry.Sql(Statement.CommentOn("public.com_acme_some_event_1", "iglu:com.acme/some_event/jsonschema/1-0-0")), LogEntry.Message("Table created") ) - val (state, value) = Migration.build[Pure, Pure, Unit](input, PureDAO.DummyTarget).run + val (state, value) = Migration.build[Pure, Pure, Unit](input, PureDAO.DummyTarget, Nil).run state.getLog must beEqualTo(expected) value must beRight.like { case Migration(preTransaction, inTransaction) => preTransaction must beEmpty @@ -174,8 +200,9 @@ class MigrationSpec extends Specification { object MigrationSpec { val schema100 = SelfDescribingSchema( SchemaMap(SchemaKey("com.acme", "context", "jsonschema", SchemaVer.Full(1, 0, 0))), - Schema(properties = - Some( + Schema( + `type` = Some(CommonProperties.Type.Object), + properties = Some( ObjectProperty.Properties( Map( "one" -> Schema(), @@ -187,8 +214,9 @@ object MigrationSpec { ) val schema101 = SelfDescribingSchema( SchemaMap(SchemaKey("com.acme", "context", "jsonschema", SchemaVer.Full(1, 0, 1))), - Schema(properties = - Some( + Schema( + `type` = Some(CommonProperties.Type.Object), + properties = Some( ObjectProperty.Properties( Map( "one" -> Schema(), @@ -200,12 +228,8 @@ object MigrationSpec { ) ) - val schemaListSingle = SchemaList - .unsafeBuildWithReorder(ModelGroupSet.groupSchemas(NonEmptyList.of(schema100)).head) - .getOrElse(throw new RuntimeException("Cannot create SchemaList")) - val schemaListTwo = SchemaList - .unsafeBuildWithReorder(ModelGroupSet.groupSchemas(NonEmptyList.of(schema100, schema101)).head) - .getOrElse(throw new RuntimeException("Cannot create SchemaList")) + val schemaListSingle = NonEmptyList.of(schema100) + val schemaListTwo = NonEmptyList.of(schema100, schema101) val schema200 = SelfDescribingSchema( SchemaMap(SchemaKey("com.acme", "context", "jsonschema", SchemaVer.Full(2, 0, 0))), @@ -232,7 +256,5 @@ object MigrationSpec { ) ) - val schemaListThree = SchemaList - .unsafeBuildWithReorder(ModelGroupSet.groupSchemas(NonEmptyList.of(schema200, schema201)).head) - .getOrElse(throw new RuntimeException("Cannot create SchemaList")) + val schemaListThree = NonEmptyList.of(schema200, schema201) } 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 9b64d64a5..bdbaeabf5 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 @@ -10,23 +10,16 @@ package com.snowplowanalytics.snowplow.loader.snowflake import cats.implicits._ import cats.Monad import cats.data.NonEmptyList - import doobie.Fragment import doobie.implicits._ - import io.circe.syntax._ import io.circe.parser.parse - -import com.snowplowanalytics.iglu.core.SchemaKey - -import com.snowplowanalytics.iglu.schemaddl.migrations.{Migration, SchemaList} - +import com.snowplowanalytics.iglu.core.{SchemaCriterion, SchemaKey} +import com.snowplowanalytics.iglu.schemaddl.redshift.ShredModel import com.snowplowanalytics.snowplow.analytics.scalasdk.SnowplowEvent - import com.snowplowanalytics.snowplow.loader.snowflake.ast.SnowflakeDatatype import com.snowplowanalytics.snowplow.loader.snowflake.ast.Statements.AddColumn import com.snowplowanalytics.snowplow.loader.snowflake.db.SnowflakeManifest - import com.snowplowanalytics.snowplow.rdbloader.common.LoaderMessage.TypesInfo.WideRow.WideRowFormat.{JSON, PARQUET} import com.snowplowanalytics.snowplow.rdbloader.common.LoaderMessage.{SnowplowEntity, TypesInfo} import com.snowplowanalytics.snowplow.rdbloader.common.cloud.BlobStorage @@ -56,25 +49,25 @@ object Snowflake { override val requiresEventsColumns: Boolean = false - override def updateTable(migration: Migration): Block = { - val target = SchemaKey(migration.vendor, migration.name, "jsonschema", migration.to) - val entity = Entity.Table(schema, target) + def updateTable(shredModel: ShredModel.GoodModel, currentSchemaKey: SchemaKey): Block = { + val entity = Entity.Table(schema, currentSchemaKey, shredModel.tableName) Block(Nil, Nil, entity) } - override def extendTable(info: ShreddedType.Info): Option[Block] = { + override def extendTable(info: ShreddedType.Info): List[Block] = { val isContext = info.entity == SnowplowEntity.Context val columnType = if (isContext) SnowflakeDatatype.JsonArray else SnowflakeDatatype.JsonObject val columnName = info.getNameFull val addColumnSql = AddColumn(schema, EventsTable.MainName, columnName, columnType) val addColumn = Item.AddColumn(addColumnSql.toFragment, Nil) - Some(Block(List(addColumn), Nil, Entity.Column(info))) + List(Block(List(addColumn), Nil, Entity.Column(info))) } override def getLoadStatements( discovery: DataDiscovery, eventTableColumns: EventTableColumns, - initQueryResult: InitQueryResult + initQueryResult: InitQueryResult, + disableMigration: List[SchemaCriterion] ): LoadStatements = { val columnsToCopy = columnsToCopyFromDiscoveredData(discovery) @@ -118,10 +111,8 @@ object Snowflake { } yield InitQueryResult(transformedStagePath, folderMonitoringStagePath) // Technically, Snowflake Loader cannot create new tables - override def createTable(schemas: SchemaList): Block = { - val entity = Entity.Table(schema, schemas.latest.schemaKey) - Block(Nil, Nil, entity) - } + override def createTable(shredModel: ShredModel): Block = + throw new IllegalStateException("createTable should never be called for Snowflake") override def getManifest: Statement = Statement.CreateTable(SnowflakeManifest.getManifestDef(schema).toFragment) @@ -378,7 +369,7 @@ object Snowflake { private def getShredTypeColumn(shreddedType: ShreddedType): ColumnName = { val shredProperty = shreddedType.getSnowplowEntity.toSdkProperty val info = shreddedType.info - ColumnName(SnowplowEvent.transformSchema(shredProperty, info.vendor, info.name, info.model)) + ColumnName(SnowplowEvent.transformSchema(shredProperty, info.vendor, info.name, info.version.model)) } private def findPathAfterStage( diff --git a/modules/transformer-batch/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/batch/Transformer.scala b/modules/transformer-batch/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/batch/Transformer.scala index 08b0519aa..150d15443 100644 --- a/modules/transformer-batch/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/batch/Transformer.scala +++ b/modules/transformer-batch/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/batch/Transformer.scala @@ -14,7 +14,7 @@ import com.snowplowanalytics.iglu.client.resolver.registries.JavaNetRegistryLook import com.snowplowanalytics.iglu.schemaddl.parquet.FieldValue import com.snowplowanalytics.snowplow.rdbloader.common.transformation.parquet.ParquetTransformer import com.snowplowanalytics.snowplow.rdbloader.common.transformation.parquet.fields.AllFields -import com.snowplowanalytics.snowplow.rdbloader.transformer.batch.spark.singleton.{IgluSingleton, PropertiesCacheSingleton} +import com.snowplowanalytics.snowplow.rdbloader.transformer.batch.spark.singleton.{IgluSingleton, ShredModelCacheSingleton} import org.apache.spark.sql.Row import org.apache.spark.sql.types.StructType @@ -86,7 +86,7 @@ object Transformer { Transformed .shredEvent[Id]( IgluSingleton.get(resolverConfig), - PropertiesCacheSingleton.get(resolverConfig), + ShredModelCacheSingleton.get(resolverConfig), isTabular, ShredJob.BadRowsProcessor )(event) @@ -101,10 +101,10 @@ object Transformer { } def badTransform(badRow: BadRow, badEventsCounter: LongAccumulator): Transformed = { - val SchemaKey(vendor, name, _, SchemaVer.Full(model, _, _)) = badRow.schemaKey + val SchemaKey(vendor, name, _, SchemaVer.Full(model, revision, addition)) = badRow.schemaKey val data = Transformed.Data.DString(badRow.compact) badEventsCounter.add(1L) - Transformed.Shredded.Json(false, vendor, name, model, data) + Transformed.Shredded.Json(false, vendor, name, model, revision, addition, data) } def typesInfo(skipSchemas: List[SchemaCriterion]): TypesInfo = { @@ -214,7 +214,7 @@ object Transformer { } type WideRowTuple = (String, String) - type ShreddedTuple = (String, String, String, String, Int, String) + type ShreddedTuple = (String, String, String, String, Int, Int, Int, String) private implicit class TransformedOps(t: Transformed) { def wideRow: Option[WideRowTuple] = t match { @@ -227,7 +227,7 @@ object Transformer { def shredded: Option[ShreddedTuple] = t match { case p: Transformed.Shredded => val outputType = if (p.isGood) "good" else "bad" - (outputType, p.vendor, p.name, p.format.path, p.model, p.data.value).some + (outputType, p.vendor, p.name, p.format.path, p.model, p.revision, p.addition, p.data.value).some case _ => None } diff --git a/modules/transformer-batch/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/batch/spark/Sink.scala b/modules/transformer-batch/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/batch/spark/Sink.scala index cb688afa1..fbb16da29 100644 --- a/modules/transformer-batch/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/batch/spark/Sink.scala +++ b/modules/transformer-batch/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/batch/spark/Sink.scala @@ -17,15 +17,15 @@ object Sink { def writeShredded( spark: SparkSession, compression: Compression, - data: RDD[(String, String, String, String, Int, String)], + data: RDD[(String, String, String, String, Int, Int, Int, String)], outFolder: String ): Unit = { import spark.implicits._ data - .toDF("output", "vendor", "name", "format", "model", "data") + .toDF("output", "vendor", "name", "format", "model", "revision", "addition", "data") .write .withCompression(compression) - .partitionBy("output", "vendor", "name", "format", "model") + .partitionBy("output", "vendor", "name", "format", "model", "revision", "addition") .mode(SaveMode.Append) .text(outFolder) } diff --git a/modules/transformer-batch/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/batch/spark/singleton.scala b/modules/transformer-batch/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/batch/spark/singleton.scala index 6335b5592..74102dfd2 100644 --- a/modules/transformer-batch/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/batch/spark/singleton.scala +++ b/modules/transformer-batch/src/main/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/batch/spark/singleton.scala @@ -14,13 +14,13 @@ import cats.syntax.show._ import com.snowplowanalytics.iglu.client.Resolver import com.snowplowanalytics.iglu.client.resolver.Resolver.ResolverConfig import com.snowplowanalytics.iglu.client.resolver.registries.JavaNetRegistryLookup.idLookupInstance -import com.snowplowanalytics.iglu.schemaddl.Properties +import com.snowplowanalytics.iglu.schemaddl.redshift.ShredModel import com.snowplowanalytics.lrumap.CreateLruMap import com.snowplowanalytics.snowplow.analytics.scalasdk.Event import com.snowplowanalytics.snowplow.eventsmanifest.{EventsManifest, EventsManifestConfig} import com.snowplowanalytics.snowplow.rdbloader.common._ import com.snowplowanalytics.snowplow.rdbloader.common.transformation.EventUtils.EventParser -import com.snowplowanalytics.snowplow.rdbloader.common.transformation.{EventUtils, PropertiesCache, PropertiesKey} +import com.snowplowanalytics.snowplow.rdbloader.common.transformation.{EventUtils, ShredModelCache, ShredModelCacheKey} import com.snowplowanalytics.snowplow.rdbloader.transformer.batch.Config import com.snowplowanalytics.snowplow.rdbloader.transformer.batch.Config.Output.BadSink import com.snowplowanalytics.snowplow.rdbloader.transformer.batch.badrows.{BadrowSink, KinesisSink, WiderowFileSink} @@ -93,14 +93,14 @@ object singleton { } } - object PropertiesCacheSingleton { - @volatile private var instance: PropertiesCache[Id] = _ + object ShredModelCacheSingleton { + @volatile private var instance: ShredModelCache[Id] = _ - def get(resolverConfig: ResolverConfig): PropertiesCache[Id] = { + def get(resolverConfig: ResolverConfig): ShredModelCache[Id] = { if (instance == null) { synchronized { if (instance == null) { - instance = CreateLruMap[Id, PropertiesKey, Properties].create(resolverConfig.cacheSize) + instance = CreateLruMap[Id, ShredModelCacheKey, ShredModel].create(resolverConfig.cacheSize) } } } diff --git a/modules/transformer-batch/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/batch/ShredJobSpec.scala b/modules/transformer-batch/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/batch/ShredJobSpec.scala index 3077cad3b..b5005eee1 100644 --- a/modules/transformer-batch/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/batch/ShredJobSpec.scala +++ b/modules/transformer-batch/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/batch/ShredJobSpec.scala @@ -61,7 +61,7 @@ object ShredJobSpec { val Name = "snowplow-transformer-batch" val Version = BuildInfo.version - val AtomicFolder = "vendor=com.snowplowanalytics.snowplow/name=atomic/format=tsv/model=1" + val AtomicFolder = "vendor=com.snowplowanalytics.snowplow/name=atomic/format=tsv/model=1/revision=0/addition=0" sealed trait Events diff --git a/modules/transformer-batch/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/batch/good/CrossBatchDeduplicationSpec.scala b/modules/transformer-batch/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/batch/good/CrossBatchDeduplicationSpec.scala index 18fc06e27..eddf6e1e4 100644 --- a/modules/transformer-batch/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/batch/good/CrossBatchDeduplicationSpec.scala +++ b/modules/transformer-batch/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/batch/good/CrossBatchDeduplicationSpec.scala @@ -52,7 +52,7 @@ object CrossBatchDeduplicationSpec { ) object expected { - val additionalContextPath = "vendor=com.snowplowanalytics.snowplow/name=ua_parser_context/format=json/model=1" + val additionalContextPath = "vendor=com.snowplowanalytics.snowplow/name=ua_parser_context/format=json/model=1/revision=0/addition=0" val additionalContextContents1 = s""" |{ diff --git a/modules/transformer-batch/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/batch/good/DerivedContextsSpec.scala b/modules/transformer-batch/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/batch/good/DerivedContextsSpec.scala index a58fd1ab9..fd8577a8e 100644 --- a/modules/transformer-batch/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/batch/good/DerivedContextsSpec.scala +++ b/modules/transformer-batch/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/batch/good/DerivedContextsSpec.scala @@ -18,7 +18,7 @@ object DerivedContextsSpec { ) object expected { - val path = s"vendor=org.schema/name=WebPage/format=json/model=1" + val path = s"vendor=org.schema/name=WebPage/format=json/model=1/revision=0/addition=0" val contents = s"""|{ |"schema":{ diff --git a/modules/transformer-batch/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/batch/good/EmptySchemaSpec.scala b/modules/transformer-batch/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/batch/good/EmptySchemaSpec.scala index b582afe4e..193d8adaf 100644 --- a/modules/transformer-batch/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/batch/good/EmptySchemaSpec.scala +++ b/modules/transformer-batch/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/batch/good/EmptySchemaSpec.scala @@ -65,8 +65,8 @@ object EmptySchemaSpec { val contexBContents = "com.snowplowanalytics.iglu\tanything-b\tjsonschema\t1-0-0\t2b1b25a4-c0df-4859-8201-cf21492ad61b\t2014-05-29 18:16:35.000\tevents\t[\"events\",\"anything-b\"]\tevents" - val contextAPath = "vendor=com.snowplowanalytics.iglu/name=anything-a/format=tsv/model=1" - val contextBPath = "vendor=com.snowplowanalytics.iglu/name=anything-b/format=tsv/model=1" + val contextAPath = "vendor=com.snowplowanalytics.iglu/name=anything-a/format=tsv/model=1/revision=0/addition=0" + val contextBPath = "vendor=com.snowplowanalytics.iglu/name=anything-b/format=tsv/model=1/revision=0/addition=0" // Removed three JSON columns and added 7 columns at the end val event = diff --git a/modules/transformer-batch/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/batch/good/EventDeduplicationSpec.scala b/modules/transformer-batch/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/batch/good/EventDeduplicationSpec.scala index 66f663460..cef073d68 100644 --- a/modules/transformer-batch/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/batch/good/EventDeduplicationSpec.scala +++ b/modules/transformer-batch/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/batch/good/EventDeduplicationSpec.scala @@ -65,7 +65,7 @@ class EventDeduplicationSpec extends Specification with ShredJobSpec { "A job which is provided with a two events with same event_id and both natural and synthetic deduplication are enabled" should { object expected { - val path = "vendor=com.snowplowanalytics.snowplow/name=duplicate/format=json/model=1" + val path = "vendor=com.snowplowanalytics.snowplow/name=duplicate/format=json/model=1/revision=0/addition=0" val contents = s"""|{ |"schema":{ @@ -86,7 +86,7 @@ class EventDeduplicationSpec extends Specification with ShredJobSpec { |} |}""".stripMargin.replaceAll("[\n\r]", "") - val additionalContextPath = "vendor=com.snowplowanalytics.snowplow/name=ua_parser_context/format=json/model=1" + val additionalContextPath = "vendor=com.snowplowanalytics.snowplow/name=ua_parser_context/format=json/model=1/revision=0/addition=0" val additionalContextContents = s""" |{ @@ -180,7 +180,7 @@ class EventDeduplicationSpec extends Specification with ShredJobSpec { "A job which is provided with a two events with same event_id and both natural and synthetic deduplication are disabled" should { object expected { - val additionalContextPath = "vendor=com.snowplowanalytics.snowplow/name=ua_parser_context/format=json/model=1" + val additionalContextPath = "vendor=com.snowplowanalytics.snowplow/name=ua_parser_context/format=json/model=1/revision=0/addition=0" val additionalContextContents = s""" |{ diff --git a/modules/transformer-batch/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/batch/good/ForwardCompatibleContextSpec.scala b/modules/transformer-batch/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/batch/good/ForwardCompatibleContextSpec.scala index c45c42002..d8eef5834 100644 --- a/modules/transformer-batch/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/batch/good/ForwardCompatibleContextSpec.scala +++ b/modules/transformer-batch/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/batch/good/ForwardCompatibleContextSpec.scala @@ -18,7 +18,7 @@ object ForwardCompatibleContextSpec { ) object expected { - val path = "vendor=org.schema/name=WebPage/format=json/model=1" + val path = "vendor=org.schema/name=WebPage/format=json/model=1/revision=0/addition=0" val contents = s"""|{ |"schema":{ diff --git a/modules/transformer-batch/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/batch/good/LinkClickEventSpec.scala b/modules/transformer-batch/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/batch/good/LinkClickEventSpec.scala index 840e085c0..2cdc581b5 100644 --- a/modules/transformer-batch/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/batch/good/LinkClickEventSpec.scala +++ b/modules/transformer-batch/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/batch/good/LinkClickEventSpec.scala @@ -18,7 +18,7 @@ object LinkClickEventSpec { ) object expected { - val path = "vendor=com.snowplowanalytics.snowplow/name=link_click/format=json/model=1" + val path = "vendor=com.snowplowanalytics.snowplow/name=link_click/format=json/model=1/revision=0/addition=0" val contents = s"""|{ |"schema":{ diff --git a/modules/transformer-batch/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/batch/good/WebsitePageContextSpec.scala b/modules/transformer-batch/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/batch/good/WebsitePageContextSpec.scala index b39c6f974..c6c8a9d2c 100644 --- a/modules/transformer-batch/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/batch/good/WebsitePageContextSpec.scala +++ b/modules/transformer-batch/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/batch/good/WebsitePageContextSpec.scala @@ -17,7 +17,7 @@ object WebsitePageContextSpec { ) object expected { - val path = "vendor=org.schema/name=WebPage/format=json/model=1" + val path = "vendor=org.schema/name=WebPage/format=json/model=1/revision=0/addition=0" val contents = s"""|{ |"schema":{ diff --git a/modules/transformer-batch/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/batch/good/tabular/NewlineSpec.scala b/modules/transformer-batch/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/batch/good/tabular/NewlineSpec.scala index 497774c6d..2dd167304 100644 --- a/modules/transformer-batch/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/batch/good/tabular/NewlineSpec.scala +++ b/modules/transformer-batch/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/batch/good/tabular/NewlineSpec.scala @@ -63,7 +63,7 @@ object NewlineSpec { ) object expected { - val contextPath = s"vendor=com.snowplowanalytics.snowplow/name=change_form/format=tsv/model=1" + val contextPath = s"vendor=com.snowplowanalytics.snowplow/name=change_form/format=tsv/model=1/revision=0/addition=0" val contextContents = "com.snowplowanalytics.snowplow\tchange_form\tjsonschema\t1-0-0\tdeadbeef-dead-beef-dead-0000beefdead\t1970-01-01 00:00:00.000\tevents\t[\"events\",\"change_form\"]\tevents\tb\ta\tTEXTAREA\t\\N\t\\N\tline 1 line2 column2" diff --git a/modules/transformer-batch/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/batch/good/tabular/TabularOutputSpec.scala b/modules/transformer-batch/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/batch/good/tabular/TabularOutputSpec.scala index e785fbbf6..b2810e683 100644 --- a/modules/transformer-batch/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/batch/good/tabular/TabularOutputSpec.scala +++ b/modules/transformer-batch/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/batch/good/tabular/TabularOutputSpec.scala @@ -58,7 +58,7 @@ object TabularOutputSpec { "stackTrace": null, "isFatal": true, "className": "AbstractSingletonFactoryBean", - "causeStackTrace": "this column should be last" + "causeStackTrace": "trace" } } }""".noSpaces @@ -68,15 +68,15 @@ object TabularOutputSpec { ) object expected { - val contextPath = s"vendor=org.schema/name=WebPage/format=tsv/model=1" + val contextPath = s"vendor=org.schema/name=WebPage/format=tsv/model=1/revision=0/addition=0" val contextContents = "org.schema\tWebPage\tjsonschema\t1-0-0\t2b1b25a4-c0df-4859-8201-cf21492ad61b\t2014-05-29 18:16:35.000\tevents\t[\"events\",\"WebPage\"]\tevents\tJonathan Almeida\t[\"blog\",\"releases\"]\t\\N\t\\N\t2014-07-23T00:00:00Z\tblog\ten-US\t[\"snowplow\",\"analytics\",\"java\",\"jvm\",\"tracker\"]" - val eventPath = s"vendor=com.snowplowanalytics.snowplow/name=application_error/format=tsv/model=1" + val eventPath = s"vendor=com.snowplowanalytics.snowplow/name=application_error/format=tsv/model=1/revision=0/addition=2" val eventContents = - "com.snowplowanalytics.snowplow\tapplication_error\tjsonschema\t1-0-2\t2b1b25a4-c0df-4859-8201-cf21492ad61b\t2014-05-29 18:16:35.000\tevents\t[\"events\",\"application_error\"]\tevents\tundefined is not a function\tJAVASCRIPT\tAbstractSingletonFactoryBean\t\\N\t1\t\\N\t\\N\t14\t\\N\t\\N\t\\N\tthis column should be last" + "com.snowplowanalytics.snowplow\tapplication_error\tjsonschema\t1-0-2\t2b1b25a4-c0df-4859-8201-cf21492ad61b\t2014-05-29 18:16:35.000\tevents\t[\"events\",\"application_error\"]\tevents\tundefined is not a function\tJAVASCRIPT\ttrace\tAbstractSingletonFactoryBean\t\\N\t\\N\t1\t\\N\t\\N\t\\N\t14\t\\N" // Removed three JSON columns and added 7 columns at the end val event = diff --git a/modules/transformer-batch/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/batch/iterator/GoodShredIteratorSpec.scala b/modules/transformer-batch/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/batch/iterator/GoodShredIteratorSpec.scala index 3023052b1..ba281cf80 100644 --- a/modules/transformer-batch/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/batch/iterator/GoodShredIteratorSpec.scala +++ b/modules/transformer-batch/src/test/scala/com/snowplowanalytics/snowplow/rdbloader/transformer/batch/iterator/GoodShredIteratorSpec.scala @@ -13,6 +13,6 @@ import com.snowplowanalytics.snowplow.rdbloader.transformer.batch.iterator.OnlyG class GoodShredIteratorSpec extends GoodDataIteratorSpec { - override val goodGenerator: GoodGenerator = () => Shredded.Tabular("vendor", "name", 1, DString("TEST GOOD DATA")) - override val badGenerator: BadGenerator = () => Shredded.Json(isGood = false, "vendor", "name", 1, DString("TEST BAD DATA")) + override val goodGenerator: GoodGenerator = () => Shredded.Tabular("vendor", "name", 1, 0, 0, DString("TEST GOOD DATA")) + override val badGenerator: BadGenerator = () => Shredded.Json(isGood = false, "vendor", "name", 1, 0, 0, DString("TEST BAD DATA")) } diff --git a/project/Dependencies.scala b/project/Dependencies.scala index 414343e3d..8c344b430 100644 --- a/project/Dependencies.scala +++ b/project/Dependencies.scala @@ -42,7 +42,7 @@ object Dependencies { val spark = "3.3.1" val eventsManifest = "0.4.0" - val schemaDdl = "0.18.2" + val schemaDdl = "0.20.0" val jacksonModule = "2.14.2" // Override incompatible version in spark runtime val jacksonDatabind = "2.14.2" val jacksonMapper = "1.9.14-atlassian-6" // Fix CVE