diff --git a/Jenkinsfile b/Jenkinsfile index 16df1e9..69f2049 100644 --- a/Jenkinsfile +++ b/Jenkinsfile @@ -1,6 +1,4 @@ pipeline { - agent { label 'm3d' } - options { ansiColor('xterm') disableConcurrentBuilds() @@ -10,10 +8,6 @@ pipeline { buildDiscarder(logRotator(daysToKeepStr: '32', numToKeepStr: '16')) } - environment { - GIT_CREDENTIALS = "9654c627-4650-4079-be03-2d0336fe724f" - } - stages { stage('cleanup workspace') { steps { diff --git a/README.md b/README.md index 3009da3..f8a7ad1 100644 --- a/README.md +++ b/README.md @@ -86,7 +86,7 @@ The parameter file for the full load algorithm for example has the following con "has_header": false, "partition_column": "date_column_name", "partition_column_format": "yyyyMMdd", - "partition_columns": [ + "target_partitions": [ "year", "month" ], @@ -103,7 +103,7 @@ The parameter file for the full load algorithm for example has the following con * `has_header` flag defining whether the input files have a header * `partition_column` column that contains the partitioning information * `partition_column_format` format of the partitioning column in the case of of time/date columns -* `partition_columns` partitioning columns +* `target_partitions` partitioning columns in the target * `target_table` target table where the data will be available for querying after loading ### License and Software Information diff --git a/src/main/scala/com/adidas/analytics/AlgorithmFactory.scala b/src/main/scala/com/adidas/analytics/AlgorithmFactory.scala index 40e34b1..07eb338 100644 --- a/src/main/scala/com/adidas/analytics/AlgorithmFactory.scala +++ b/src/main/scala/com/adidas/analytics/AlgorithmFactory.scala @@ -69,6 +69,7 @@ object AlgorithmFactory { case "PartitionRangeMaterialization" => PartitionMaterialization.newRangeMaterialization(spark, dfs, configLocation) case "PartitionQueryMaterialization" => PartitionMaterialization.newQueryMaterialization(spark, dfs, configLocation) case "FixedSizeStringExtractor" => FixedSizeStringExtractor(spark, dfs, configLocation) + case "NestedFlattener" => NestedFlattener(spark, dfs, configLocation) case _ => throw new RuntimeException(s"Unable to find algorithm corresponding to $className") } } diff --git a/src/main/scala/com/adidas/analytics/algo/AppendLoad.scala b/src/main/scala/com/adidas/analytics/algo/AppendLoad.scala index e9bf054..f7d02ba 100644 --- a/src/main/scala/com/adidas/analytics/algo/AppendLoad.scala +++ b/src/main/scala/com/adidas/analytics/algo/AppendLoad.scala @@ -27,11 +27,11 @@ final class AppendLoad protected(val spark: SparkSession, val dfs: DFSWrapper, v } override protected def transform(dataFrames: Vector[DataFrame]): Vector[DataFrame] = { - dataFrames.map(df => df.transform(addPartitionColumns(columnToRegexPairs, targetSchema))) + dataFrames.map(df => df.transform(addtargetPartitions(columnToRegexPairs, targetSchema))) } override protected def write(dataFrames: Vector[DataFrame]): Unit = { - writeHeaders(dataFrames, partitionColumns, headerDir, dfs) + writeHeaders(dataFrames, targetPartitions, headerDir, dfs) super.write(dataFrames) if (computeTableStatistics && dataType == STRUCTURED) computeStatisticsForTable(targetTable) @@ -47,7 +47,7 @@ final class AppendLoad protected(val spark: SparkSession, val dfs: DFSWrapper, v } private def listSources(inputDirPath: Path, headerDirPath: Path, fs: FileSystem, targetSchema: StructType): Seq[Source] = { - val targetSchemaWithoutPartitionColumns = getSchemaWithoutPartitionColumns(targetSchema, partitionColumns.toSet) + val targetSchemaWithouttargetPartitions = getSchemaWithouttargetPartitions(targetSchema, targetPartitions.toSet) logger.info(s"Looking for input files in $inputDirPath") val groupedHeaderPathAndSourcePaths = fs.ls(inputDirPath, recursive = true).groupBy { inputPath => @@ -56,7 +56,7 @@ final class AppendLoad protected(val spark: SparkSession, val dfs: DFSWrapper, v def getMapSchemaStructToPath = { val mapSchemaStructToPath = groupedHeaderPathAndSourcePaths.toSeq.map { case (headerPath, sourcePaths) => - getSchemaFromHeaderOrSource(fs, headerPath, sourcePaths, targetSchemaWithoutPartitionColumns) + getSchemaFromHeaderOrSource(fs, headerPath, sourcePaths, targetSchemaWithouttargetPartitions) }.groupBy(_._1).map { case (k, v) => (k, v.flatMap(_._2)) } val filteredMapSchemaStructToPath = mapSchemaStructToPath.filter(schemaFromInputData => matchingSchemas_?(schemaFromInputData._1, targetSchema, schemaFromInputData._2)) @@ -73,7 +73,7 @@ final class AppendLoad protected(val spark: SparkSession, val dfs: DFSWrapper, v val schemaAndSourcePath = if(!verifySchema) { groupedHeaderPathAndSourcePaths.flatMap { case (headerPath, sourcePaths) => - val schema = if (fs.exists(headerPath)) loadHeader(headerPath, fs) else targetSchemaWithoutPartitionColumns + val schema = if (fs.exists(headerPath)) loadHeader(headerPath, fs) else targetSchemaWithouttargetPartitions sourcePaths.map { sourcePath => Source(schema, sourcePath.toString) } @@ -84,7 +84,7 @@ final class AppendLoad protected(val spark: SparkSession, val dfs: DFSWrapper, v schemaAndSourcePath.toSeq } - private def getSchemaFromHeaderOrSource(fs: FileSystem, headerPath: Path, sourcePaths: Seq[Path], targetSchemaWithoutPartitionColumns: StructType): (StructType, Seq[Path]) ={ + private def getSchemaFromHeaderOrSource(fs: FileSystem, headerPath: Path, sourcePaths: Seq[Path], targetSchemaWithouttargetPartitions: StructType): (StructType, Seq[Path]) ={ val schema = if (fs.exists(headerPath)){ loadHeader(headerPath, fs) } else { @@ -143,8 +143,8 @@ object AppendLoad { path.replaceFirst("\\w+\\d*://.+?/", "") } - private def getSchemaWithoutPartitionColumns(targetSchema: StructType, partitionColumns: Set[String]): StructType = { - StructType(targetSchema.fields.filterNot(field => partitionColumns.contains(field.name))) + private def getSchemaWithouttargetPartitions(targetSchema: StructType, targetPartitions: Set[String]): StructType = { + StructType(targetSchema.fields.filterNot(field => targetPartitions.contains(field.name))) } private def groupSourcesBySchema(sources: Seq[Source]): Map[StructType, Seq[String]] = { @@ -153,7 +153,7 @@ object AppendLoad { } } - private def addPartitionColumns(columnNameToRegexPairs: Seq[(String, String)], schema: StructType)(inputDf: DataFrame): DataFrame = { + private def addtargetPartitions(columnNameToRegexPairs: Seq[(String, String)], schema: StructType)(inputDf: DataFrame): DataFrame = { def getInputFileName: Column = { udf((path: String) => extractPathWithoutServerAndProtocol(path)).apply(input_file_name) } @@ -185,13 +185,13 @@ object AppendLoad { DataType.fromJson(fs.readFile(headerPath)).asInstanceOf[StructType] } - protected def writeHeaders(dataFrames: Seq[DataFrame], partitionColumns: Seq[String], headerDir: String, dfs: DFSWrapper): Unit = { + protected def writeHeaders(dataFrames: Seq[DataFrame], targetPartitions: Seq[String], headerDir: String, dfs: DFSWrapper): Unit = { logger.info(s"Writing header files to $headerDir") val headerDirPath = new Path(headerDir) val fs = dfs.getFileSystem(headerDirPath) dataFrames.foreach { df => - val schemaJson = getSchemaWithoutPartitionColumns(df.schema, partitionColumns.toSet).prettyJson - df.collectPartitions(partitionColumns).foreach { partitionCriteria => + val schemaJson = getSchemaWithouttargetPartitions(df.schema, targetPartitions.toSet).prettyJson + df.collectPartitions(targetPartitions).foreach { partitionCriteria => val subdirectories = DataFrameUtils.mapPartitionsToDirectories(partitionCriteria) val headerPath = new Path(headerDirPath.join(subdirectories), headerFileName) if (!fs.exists(headerPath)) { diff --git a/src/main/scala/com/adidas/analytics/algo/DeltaLoad.scala b/src/main/scala/com/adidas/analytics/algo/DeltaLoad.scala index 4f54bb0..48cc30e 100644 --- a/src/main/scala/com/adidas/analytics/algo/DeltaLoad.scala +++ b/src/main/scala/com/adidas/analytics/algo/DeltaLoad.scala @@ -19,12 +19,12 @@ final class DeltaLoad protected(val spark: SparkSession, val dfs: DFSWrapper, va extends Algorithm with PartitionedDeltaLoadConfiguration with DateComponentDerivation { override protected def transform(dataFrames: Vector[DataFrame]): Vector[DataFrame] = { - val dataFramesWithPartitionColumnsAdded = withDatePartitions(spark, dfs, dataFrames.take(1)) - val deltaRecords = dataFramesWithPartitionColumnsAdded(0).persist(StorageLevel.MEMORY_AND_DISK) + val dataFramesWithTargetPartitionsAdded = withDatePartitions(spark, dfs, dataFrames.take(1)) + val deltaRecords = dataFramesWithTargetPartitionsAdded(0).persist(StorageLevel.MEMORY_AND_DISK) val activeRecords = dataFrames(1) - val partitions = deltaRecords.collectPartitions(partitionColumns) + val partitions = deltaRecords.collectPartitions(targetPartitions) val isRequiredPartition = DataFrameUtils.buildPartitionsCriteriaMatcherFunc(partitions, activeRecords.schema) // Create DataFrame containing full content of partitions that need to be touched @@ -70,8 +70,8 @@ final class DeltaLoad protected(val spark: SparkSession, val dfs: DFSWrapper, va logger.info("Adding partitioning information if needed") try { dataFrames.map { df => - if (df.columns.toSeq.intersect(partitionColumns) != partitionColumns){ - df.transform(withDateComponents(partitionSourceColumn, partitionSourceColumnFormat, partitionColumns)) + if (df.columns.toSeq.intersect(targetPartitions) != targetPartitions){ + df.transform(withDateComponents(partitionSourceColumn, partitionSourceColumnFormat, targetPartitions)) } else df } diff --git a/src/main/scala/com/adidas/analytics/algo/FixedSizeStringExtractor.scala b/src/main/scala/com/adidas/analytics/algo/FixedSizeStringExtractor.scala index a852249..37cfd1e 100644 --- a/src/main/scala/com/adidas/analytics/algo/FixedSizeStringExtractor.scala +++ b/src/main/scala/com/adidas/analytics/algo/FixedSizeStringExtractor.scala @@ -24,7 +24,7 @@ final class FixedSizeStringExtractor protected(val spark: SparkSession, val dfs: } def extractFields(df: DataFrame, targetSchema: StructType): DataFrame = { - val nonPartitionFields = targetSchema.fields.filter(field => !partitionColumnsSet.contains(field.name)) + val nonPartitionFields = targetSchema.fields.filter(field => !targetPartitionsSet.contains(field.name)) if (substringPositions.length != nonPartitionFields.length) { throw new RuntimeException("Field positions do not correspond to the target schema") } diff --git a/src/main/scala/com/adidas/analytics/algo/FullLoad.scala b/src/main/scala/com/adidas/analytics/algo/FullLoad.scala index 28b0af2..0827eb4 100644 --- a/src/main/scala/com/adidas/analytics/algo/FullLoad.scala +++ b/src/main/scala/com/adidas/analytics/algo/FullLoad.scala @@ -1,10 +1,10 @@ package com.adidas.analytics.algo +import com.adidas.analytics.config.FullLoadConfiguration import com.adidas.analytics.algo.FullLoad._ import com.adidas.analytics.algo.core.Algorithm import com.adidas.analytics.algo.core.Algorithm.{ComputeTableStatisticsOperation, WriteOperation} import com.adidas.analytics.algo.shared.DateComponentDerivation -import com.adidas.analytics.config.FullLoadConfiguration import com.adidas.analytics.util.DFSWrapper._ import com.adidas.analytics.util.DataFormat.{DSVFormat, ParquetFormat} import com.adidas.analytics.util._ @@ -12,10 +12,14 @@ import org.apache.hadoop.fs.Path import org.apache.spark.sql.{DataFrame, SparkSession} import org.slf4j.{Logger, LoggerFactory} +import scala.util.{Failure, Success, Try} + final class FullLoad protected(val spark: SparkSession, val dfs: DFSWrapper, val configLocation: String) extends Algorithm with WriteOperation with FullLoadConfiguration with DateComponentDerivation with ComputeTableStatisticsOperation{ + val currentHdfsDir: String = HiveTableAttributeReader(targetTable, spark).getTableLocation + override protected def read(): Vector[DataFrame] = { createBackupTable() @@ -33,21 +37,32 @@ final class FullLoad protected(val spark: SparkSession, val dfs: DFSWrapper, val } override protected def write(dataFrames: Vector[DataFrame]): Unit = { - super.write(dataFrames) - restoreTable() - if (computeTableStatistics && dataType == STRUCTURED) - computeStatisticsForTable(Some(targetTable)) + Try{ + super.write(dataFrames) + } match { + case Failure(exception) => + logger.error(s"Handled Exception: ${exception.getMessage}. " + + s"Start Rolling Back the Full Load of table: ${targetTable}!") + recoverFailedWrite() + cleanupDirectory(backupDir) + throw new RuntimeException(exception.getMessage) + case Success(_) => + restoreTable() + if (computeTableStatistics && dataType == STRUCTURED) + computeStatisticsForTable(Option(targetTable)) + } + } private def createBackupTable(): Unit = { createDirectory(backupDir) // backup the data from the current dir because currently data directory for full load is varying - val currentDir = HiveTableAttributeReader(targetTable, spark).getTableLocation - backupDataDirectory(currentDir, backupDir) + + backupDataDirectory(currentHdfsDir, backupDir) try { - dropAndRecreateTableInNewLocation(targetTable, backupDir, partitionColumns) + dropAndRecreateTableInNewLocation(targetTable, backupDir, targetPartitions) } catch { case e: Throwable => logger.error("Data backup failed", e) @@ -87,7 +102,7 @@ final class FullLoad protected(val spark: SparkSession, val dfs: DFSWrapper, val DistCpLoadHelper.backupDirectoryContent(dfs, sourceDir, destinationDir) } - private def dropAndRecreateTableInNewLocation(table: String, destinationDir: String, partitionColumns: Seq[String]): Unit = { + private def dropAndRecreateTableInNewLocation(table: String, destinationDir: String, targetPartitions: Seq[String]): Unit = { val tempTable: String = s"${table}_temp" val tempTableDummyLocation: String = s"/tmp/$table" @@ -97,7 +112,7 @@ final class FullLoad protected(val spark: SparkSession, val dfs: DFSWrapper, val //create the target table like the temp table with data in the new directory createTable(tempTable, table, destinationDir) - if (partitionColumns.nonEmpty) { + if (targetPartitions.nonEmpty) { spark.catalog.recoverPartitions(table) } } @@ -111,8 +126,8 @@ final class FullLoad protected(val spark: SparkSession, val dfs: DFSWrapper, val private def withDatePartitions(dataFrames: Vector[DataFrame]): Vector[DataFrame] ={ logger.info("Adding partitioning information if needed") try { - if (partitionColumns.nonEmpty) { - dataFrames.map(df => df.transform(withDateComponents(partitionSourceColumn, partitionSourceColumnFormat, partitionColumns))) + if (targetPartitions.nonEmpty) { + dataFrames.map(df => df.transform(withDateComponents(partitionSourceColumn, partitionSourceColumnFormat, targetPartitions))) } else { dataFrames } @@ -128,7 +143,7 @@ final class FullLoad protected(val spark: SparkSession, val dfs: DFSWrapper, val private def restoreTable(): Unit ={ try { - dropAndRecreateTableInNewLocation(targetTable, currentDir, partitionColumns) + dropAndRecreateTableInNewLocation(targetTable, currentDir, targetPartitions) } catch { case e: Throwable => logger.error("Data writing failed", e) @@ -151,23 +166,24 @@ final class FullLoad protected(val spark: SparkSession, val dfs: DFSWrapper, val spark.sql(s"DROP TABLE IF EXISTS $tempTable") } - if (partitionColumns.nonEmpty) { + if (targetPartitions.nonEmpty) { spark.catalog.recoverPartitions(targetTable) } } private def recoverFailedRead(): Unit = { - dropAndRecreateTableInNewLocation(targetTable, currentDir, partitionColumns) + dropAndRecreateTableInNewLocation(targetTable, currentDir, targetPartitions) } private def recoverFailedWrite(): Unit = { restoreDirectoryContent(currentDir, backupDir) - dropAndRecreateTableInNewLocation(targetTable, currentDir, partitionColumns) + dropAndRecreateTableInNewLocation(targetTable, currentDir, targetPartitions) } private def restoreDirectoryContent(sourceDir: String, backupDir: String): Unit = { DistCpLoadHelper.restoreDirectoryContent(dfs, sourceDir, backupDir) } + } diff --git a/src/main/scala/com/adidas/analytics/algo/NestedFlattener.scala b/src/main/scala/com/adidas/analytics/algo/NestedFlattener.scala new file mode 100644 index 0000000..933c2d3 --- /dev/null +++ b/src/main/scala/com/adidas/analytics/algo/NestedFlattener.scala @@ -0,0 +1,141 @@ +package com.adidas.analytics.algo + +import com.adidas.analytics.algo.core.Algorithm +import com.adidas.analytics.config.NestedFlattenerConfiguration +import com.adidas.analytics.util.DFSWrapper +import org.apache.spark.sql.functions.{col, explode_outer} +import org.apache.spark.sql.types.{ArrayType, StructField, StructType} +import org.apache.spark.sql.{DataFrame, SparkSession} + +/** + * An algorithm for flattening semi-structured JSON data in a configurable way, i.e., giving the user the ability to choose + * which struct fields should be flattened or array fields should be exploded by the algorithm. + * + * @param spark spark session + * @param dfs distributed file system + * @param configLocation location of the configuration file for the algorithm + */ +final class NestedFlattener protected(val spark: SparkSession, val dfs: DFSWrapper, val configLocation: String) extends Algorithm with NestedFlattenerConfiguration { + + override protected def transform(dataFrames: Vector[DataFrame]): Vector[DataFrame] = { + val inputDf = dataFrames(0) + val replacedDf = NestedFlattener.replaceCharsInColumns(spark, inputDf, charsToReplace, replacementChar) + val flattenedDf = NestedFlattener.flatDataFrame(spark, replacedDf, fieldsToFlatten, columnMapping) + Vector(flattenedDf) + } + +} + +object NestedFlattener { + + def apply(spark: SparkSession, dfs: DFSWrapper, configLocation: String): NestedFlattener = { + new NestedFlattener(spark, dfs, configLocation) + } + + /** + * Replaces problematic characters present in the semi-structured file format (e.g., JSON), such as "." which makes spark think that the field is a struct. + * Note: needs to run before any flattening attempt, that's why the transform function of this algorithm executes this step first. + * Moreover don't forget to consider the charsToReplace in the name mapping in the acon file, because these chars will be replaced by the replacementChar. + * + * @param spark spark session + * @param df dataframe to process + * @param charsToReplace problematic column name characters to be replaced + * @param replacementChar char that replaces the charsToReplace + * @return a dataframe with the column names cleansed of problematic characters + */ + def replaceCharsInColumns(spark: SparkSession, df: DataFrame, charsToReplace: String, replacementChar: String): DataFrame = { + + def changeSchemaNames(f: StructField): StructField = { + val cleansedName = f.name.replaceAll(charsToReplace, replacementChar) + f.dataType match { + case st: StructType => + val children = st.fields.map(f => changeSchemaNames(f)) + StructField(cleansedName, StructType(children), f.nullable, f.metadata) + case at: ArrayType => + val childrenDataType = changeSchemaNames(StructField("element", at.elementType)).dataType + StructField(cleansedName, ArrayType(childrenDataType, at.containsNull), f.nullable, f.metadata) + case _ => + StructField(cleansedName, f.dataType, f.nullable, f.metadata) + } + } + + val schema = StructType(df.schema.fields.map(f => changeSchemaNames(f))) + spark.createDataFrame(df.rdd, schema) + } + + /** + * Flattens a given DataFrame according to the attributes (arrays or structs) to process. + * Important Note: the chars separating parent and child fieldnames in the flattened attributes is two underscores, + * so make sure you consider this in the name mapping config in the acon file. + * + * @param spark spark session + * @param df dataframe to be processed + * @param fieldsToFlatten fields to include for the flattening process. Note: you should specify not only top-level attributes but sub-levels as well + * if you want them included. + * @param columnMapping columns to include in the final dataframe and with new (more friendly) names. Note: columns not in the columnMapping will be excluded + * @return flattened DataFrame according to the configuration of the algorithm + */ + def flatDataFrame(spark: SparkSession, df: DataFrame, fieldsToFlatten: Seq[String], columnMapping: Map[String, String]): DataFrame = { + + def dropFieldIfNotForFlattening(df: DataFrame, fieldName: String, callback: () => DataFrame): DataFrame = { + if (fieldsToFlatten.contains(fieldName)) + callback() + else + df.drop(fieldName) + } + + @scala.annotation.tailrec + def flatDataFrameAux(df: DataFrame): DataFrame = { + + var auxDf = df + var continueFlat = false + + auxDf.schema.fields.foreach(f => { + f.dataType match { + case _: ArrayType => + auxDf = dropFieldIfNotForFlattening(auxDf, f.name, () => { + val columnsWithoutArray = auxDf.schema.fieldNames + .filter(_ != f.name) + .map(n => col(n)) + val columnsWithExplode = columnsWithoutArray ++ Array(explode_outer(col(f.name)).as(f.name)) + continueFlat = true + auxDf.select(columnsWithExplode: _*) + }) + + case st: StructType => + auxDf = dropFieldIfNotForFlattening(auxDf, f.name, () => { + // renames all struct fields to have full names and removes original struct root + val fullPathNames = st.fieldNames.map(n => f.name + "." + n) + val columnNamesWithoutStruct = auxDf.schema.fieldNames.filter(_ != f.name) ++ fullPathNames + val renamedColumns = columnNamesWithoutStruct.map(n => col(n).as(n.replace(".", "__"))) + continueFlat = true + auxDf.select(renamedColumns: _*) + }) + + case _ => // do nothing + } + }) + + if (continueFlat) + flatDataFrameAux(auxDf) + else + auxDf + } + + // Rename fields according to columnMapping and drop columns that are not mapped + var flattenedDf = flatDataFrameAux(df) + flattenedDf.schema.foreach(f => { + if (columnMapping.contains(f.name)) + flattenedDf = flattenedDf.withColumnRenamed(f.name, columnMapping(f.name)) + else + flattenedDf = flattenedDf.drop(f.name) + }) + + flattenedDf + } + +} + + + + diff --git a/src/main/scala/com/adidas/analytics/algo/core/Algorithm.scala b/src/main/scala/com/adidas/analytics/algo/core/Algorithm.scala index 993bb31..1304336 100644 --- a/src/main/scala/com/adidas/analytics/algo/core/Algorithm.scala +++ b/src/main/scala/com/adidas/analytics/algo/core/Algorithm.scala @@ -12,6 +12,7 @@ import org.slf4j.{Logger, LoggerFactory} trait Algorithm extends JobRunner with Serializable with BaseReadOperation with BaseWriteOperation { protected def spark: SparkSession + protected def dfs: DFSWrapper /** @@ -63,7 +64,7 @@ object Algorithm { * * @return number of output partitions */ - protected def outputFilesNum: Option[Int] = None // TODO: make it configurable for all algorithms + protected def outputFilesNum: Option[Int] = None // TODO: make it configurable for all algorithms /** * Writes the DataFrame using logic defined in the inheritor class @@ -141,9 +142,11 @@ object Algorithm { protected def spark: SparkSession - protected def computeStatisticsForTable(tableName: Option[String]): Unit = { - if (tableName.isDefined) - spark.sql(s"ANALYZE TABLE ${tableName.get} COMPUTE STATISTICS") - } + protected def computeStatisticsForTable(tableName: Option[String]): Unit = + tableName match { + case Some(table) => spark.sql(s"ANALYZE TABLE ${table} COMPUTE STATISTICS") + case None => Unit + } } -} + +} \ No newline at end of file diff --git a/src/main/scala/com/adidas/analytics/algo/core/Metadata.scala b/src/main/scala/com/adidas/analytics/algo/core/Metadata.scala new file mode 100644 index 0000000..216d1b9 --- /dev/null +++ b/src/main/scala/com/adidas/analytics/algo/core/Metadata.scala @@ -0,0 +1,17 @@ +package com.adidas.analytics.algo.core + +import org.apache.spark.sql.DataFrame + + +trait Metadata { + + protected val tableName: String + protected val targetPartitions: Seq[String] + + def recoverPartitions(outputDataFrame: DataFrame): Unit + + def refreshTable(outputDataFrame: DataFrame): Unit = + outputDataFrame.sparkSession.catalog.refreshTable(tableName) + +} + diff --git a/src/main/scala/com/adidas/analytics/algo/shared/CustomDateFormatters.scala b/src/main/scala/com/adidas/analytics/algo/shared/CustomDateFormatters.scala new file mode 100644 index 0000000..967f51a --- /dev/null +++ b/src/main/scala/com/adidas/analytics/algo/shared/CustomDateFormatters.scala @@ -0,0 +1,28 @@ +package com.adidas.analytics.algo.shared + +import java.time.format.{DateTimeFormatter, DateTimeFormatterBuilder} +import java.time.temporal.ChronoField + +object CustomDateFormatters { + /* + Singletons of Custom Date Formatters + */ + val YEAR_WEEK: DateTimeFormatter = new DateTimeFormatterBuilder() + .appendValue(ChronoField.YEAR, 4) + .appendValue(ChronoField.ALIGNED_WEEK_OF_YEAR, 2) + .parseDefaulting(ChronoField.DAY_OF_WEEK, 1) + .toFormatter() + + val YEAR_WEEK_DAY: DateTimeFormatter = new DateTimeFormatterBuilder() + .appendValue(ChronoField.YEAR, 4) + .appendValue(ChronoField.ALIGNED_WEEK_OF_YEAR, 2) + .appendValue(ChronoField.DAY_OF_WEEK, 1) + .toFormatter() + + val YEAR_MONTH: DateTimeFormatter = new DateTimeFormatterBuilder() + .appendValue(ChronoField.YEAR, 4) + .appendValue(ChronoField.MONTH_OF_YEAR, 2) + .parseDefaulting(ChronoField.DAY_OF_MONTH, 1) + .toFormatter() + +} diff --git a/src/main/scala/com/adidas/analytics/algo/shared/DateComponentDerivation.scala b/src/main/scala/com/adidas/analytics/algo/shared/DateComponentDerivation.scala index 92145c5..bf61963 100644 --- a/src/main/scala/com/adidas/analytics/algo/shared/DateComponentDerivation.scala +++ b/src/main/scala/com/adidas/analytics/algo/shared/DateComponentDerivation.scala @@ -1,28 +1,117 @@ package com.adidas.analytics.algo.shared -import org.apache.spark.sql.functions._ -import org.apache.spark.sql.types.StringType -import org.apache.spark.sql.{Column, DataFrame} +import java.time.LocalDate +import java.time.format.{DateTimeFormatter, DateTimeFormatterBuilder} +import java.time.temporal.ChronoField + +import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.expressions.UserDefinedFunction +import org.apache.spark.sql.functions.{udf, _} +import org.apache.spark.sql.types.{IntegerType, StringType} + +import scala.util.{Failure, Success, Try} trait DateComponentDerivation { - private val TempTimeStampColumnName: String = "__temp_timestamp_column__" + protected val tempFormatterColumnName: String = "temp_formatter_column" protected def withDateComponents(sourceDateColumnName: String, sourceDateFormat: String, targetDateComponentColumnNames: Seq[String])(inputDf: DataFrame): DataFrame = { - targetDateComponentColumnNames.foldLeft(inputDf.withColumn(TempTimeStampColumnName, to_date(col(sourceDateColumnName).cast(StringType), sourceDateFormat))) { + targetDateComponentColumnNames.foldLeft(inputDf.withColumn(tempFormatterColumnName, lit(sourceDateFormat))) { (df, colName) => colName match { - case "year" => withDateComponent(df, "year", 9999, year) - case "month" => withDateComponent(df, "month", 99, month) - case "day" => withDateComponent(df, "day", 99, dayofmonth) - case "week" => withDateComponent(df, "week", 99, weekofyear) - case everythingElse => throw new RuntimeException(s"Unable to infer a partitioning column for: $everythingElse.") + case "year" => + withDateComponent(df, sourceDateColumnName, colName, 9999, customYear) + case "month" => + withDateComponent(df, sourceDateColumnName, colName, 99, customMonth) + case "day" => + withDateComponent(df, sourceDateColumnName, colName, 99, customDay) + case "week" => + withDateComponent(df, sourceDateColumnName, colName, 99, customWeekOfYear) + case everythingElse => + throw new RuntimeException(s"Unable to infer a partitioning column for: $everythingElse.") } - }.drop(TempTimeStampColumnName) + }.drop(tempFormatterColumnName) } - private def withDateComponent(inputDf: DataFrame, targetColumnName: String, defaultValue: Int, derivationFunction: Column => Column): DataFrame = { - inputDf.withColumn(targetColumnName, when(derivationFunction(col(TempTimeStampColumnName)).isNotNull, derivationFunction(col(TempTimeStampColumnName))).otherwise(lit(defaultValue))) + private def withDateComponent(inputDf: DataFrame, + sourceDateColumnName: String, + targetColumnName: String, + defaultValue: Int, + derivationFunction: UserDefinedFunction): DataFrame = { + + inputDf + .withColumn(targetColumnName, + when( + derivationFunction(col(sourceDateColumnName).cast(StringType), col(tempFormatterColumnName)).isNotNull, + derivationFunction(col(sourceDateColumnName).cast(StringType), col(tempFormatterColumnName))) + .otherwise(lit(defaultValue)) + ) } + + private val customWeekOfYear = udf((ts: String, formatter: String) => { + Try { + getCustomFormatter(formatter) match { + case Some(customFormatter) => + LocalDate.parse(ts, customFormatter).get(ChronoField.ALIGNED_WEEK_OF_YEAR) + case None => None + } + } match { + case Failure(_) => None + case Success(value) => value + } + }, IntegerType) + + private val customYear = udf((ts: String, formatter: String) => { + Try { + getCustomFormatter(formatter) match { + case Some(customFormatter) => + LocalDate.parse(ts, customFormatter).get(ChronoField.YEAR) + case None => + LocalDate.parse(ts, DateTimeFormatter.ofPattern(formatter)).getYear + } + } match { + case Failure(_) => None + case Success(value) => value + } + }, IntegerType) + + private val customDay = udf((ts: String, formatter: String) => { + Try { + getCustomFormatter(formatter) match { + // note: this logic must be updated if we have + // customFormatters with dayOfMonth + case Some(customFormatter) => + LocalDate.parse(ts, customFormatter).get(ChronoField.DAY_OF_WEEK) + case None => + LocalDate.parse(ts, DateTimeFormatter.ofPattern(formatter)).getDayOfMonth + } + } match { + case Failure(_) => None + case Success(value) => value + } + }, IntegerType) + + private val customMonth = udf((ts: String, formatter: String) => { + Try { + getCustomFormatter(formatter) match { + case Some(customFormatter) => + LocalDate.parse(ts, customFormatter).getMonthValue + case None => + LocalDate.parse(ts, DateTimeFormatter.ofPattern(formatter)).getMonthValue + } + } match { + case Failure(_) => None + case Success(value) => value + } + }, IntegerType) + + private def getCustomFormatter(dateFormatter: String): Option[DateTimeFormatter] = + dateFormatter match { + case "yyyyww" => Option(CustomDateFormatters.YEAR_WEEK) + case "yyyywwe" => Option(CustomDateFormatters.YEAR_WEEK_DAY) + case "yyyyMM" => Option(CustomDateFormatters.YEAR_MONTH) + case _ => None + } + } diff --git a/src/main/scala/com/adidas/analytics/config/AlgorithmTemplateConfiguration.scala b/src/main/scala/com/adidas/analytics/config/AlgorithmTemplateConfiguration.scala index 1fd8de1..746d843 100644 --- a/src/main/scala/com/adidas/analytics/config/AlgorithmTemplateConfiguration.scala +++ b/src/main/scala/com/adidas/analytics/config/AlgorithmTemplateConfiguration.scala @@ -2,7 +2,7 @@ package com.adidas.analytics.config import com.adidas.analytics.algo.core.Algorithm.{ReadOperation, SafeWriteOperation} import com.adidas.analytics.config.AlgorithmTemplateConfiguration.ruleToLocalDate -import com.adidas.analytics.config.shared.ConfigurationContext +import com.adidas.analytics.config.shared.{ConfigurationContext, MetadataUpdateStrategy} import com.adidas.analytics.util.DataFormat.ParquetFormat import com.adidas.analytics.util.{InputReader, LoadMode, OutputWriter} import org.apache.spark.sql.SparkSession @@ -11,7 +11,10 @@ import org.joda.time.format.DateTimeFormat import org.joda.time.{Days, LocalDate} -trait AlgorithmTemplateConfiguration extends ConfigurationContext with ReadOperation with SafeWriteOperation { +trait AlgorithmTemplateConfiguration extends ConfigurationContext + with ReadOperation + with SafeWriteOperation + with MetadataUpdateStrategy { protected def spark: SparkSession @@ -63,7 +66,8 @@ trait AlgorithmTemplateConfiguration extends ConfigurationContext with ReadOpera OutputWriter.newTableLocationWriter ( table = targetTable, format = ParquetFormat (Some (targetSchema) ), - partitionColumns = Seq ("", "", ""), //If partitions are required, this would look like, e.g., Seq("year", "month") + metadataConfiguration = getMetaDataUpdateStrategy(targetTable, Seq ("", "", "")), + targetPartitions = Seq ("", "", ""), //If partitions are required, this would look like, e.g., Seq("year", "month") loadMode = LoadMode.OverwritePartitionsWithAddedColumns ) } diff --git a/src/main/scala/com/adidas/analytics/config/AppendLoadConfiguration.scala b/src/main/scala/com/adidas/analytics/config/AppendLoadConfiguration.scala index 589cee9..feb6c45 100644 --- a/src/main/scala/com/adidas/analytics/config/AppendLoadConfiguration.scala +++ b/src/main/scala/com/adidas/analytics/config/AppendLoadConfiguration.scala @@ -1,16 +1,20 @@ package com.adidas.analytics.config import com.adidas.analytics.algo.core.Algorithm.SafeWriteOperation -import com.adidas.analytics.config.shared.{ConfigurationContext, LoadConfiguration} +import com.adidas.analytics.config.shared.{ConfigurationContext, LoadConfiguration, MetadataUpdateStrategy} import com.adidas.analytics.util.DataFormat.ParquetFormat import com.adidas.analytics.util.{LoadMode, OutputWriter} import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.util.DropMalformedMode import org.apache.spark.sql.types.{DataType, StructType} import scala.util.parsing.json.JSONObject -trait AppendLoadConfiguration extends ConfigurationContext with LoadConfiguration with SafeWriteOperation { +trait AppendLoadConfiguration extends ConfigurationContext + with LoadConfiguration + with SafeWriteOperation + with MetadataUpdateStrategy { protected def spark: SparkSession @@ -20,6 +24,7 @@ trait AppendLoadConfiguration extends ConfigurationContext with LoadConfiguratio protected val targetTable: Option[String] = configReader.getAsOption[String]("target_table") + // This option is used to specify whether the input data schema must be the same as target schema specified in the configuration file // Note: if it is set to True, it will cause input data to be read more than once private val verifySchemaOption: Option[Boolean] = configReader.getAsOption[Boolean]("verify_schema") @@ -29,7 +34,7 @@ trait AppendLoadConfiguration extends ConfigurationContext with LoadConfiguratio case _ => false } - protected val columnToRegexPairs: Seq[(String, String)] = partitionColumns zip regexFilename + protected val columnToRegexPairs: Seq[(String, String)] = targetPartitions zip regexFilename private val jsonSchemaOption: Option[JSONObject] = configReader.getAsOption[JSONObject]("schema") @@ -41,13 +46,14 @@ trait AppendLoadConfiguration extends ConfigurationContext with LoadConfiguratio case STRUCTURED if targetTable.isDefined => OutputWriter.newTableLocationWriter( table = targetTable.get, format = ParquetFormat(Some(targetSchema)), - partitionColumns = partitionColumns, - loadMode = LoadMode.OverwritePartitionsWithAddedColumns + targetPartitions = targetPartitions, + loadMode = LoadMode.OverwritePartitionsWithAddedColumns, + metadataConfiguration = getMetaDataUpdateStrategy(targetTable.get,targetPartitions) ) case SEMISTRUCTURED if targetDir.isDefined => OutputWriter.newFileSystemWriter( location = targetDir.get, format = ParquetFormat(Some(targetSchema)), - partitionColumns = partitionColumns, + targetPartitions = targetPartitions, loadMode = LoadMode.OverwritePartitions ) case anotherDataType => throw new RuntimeException(s"Unsupported data type: $anotherDataType in AppendLoad or the configuration file is malformed.") @@ -67,4 +73,6 @@ trait AppendLoadConfiguration extends ConfigurationContext with LoadConfiguratio case anotherDataType => throw new RuntimeException(s"Unsupported data type: $anotherDataType in AppendLoad or the configuration file is malformed.") } } + + override def loadMode: String = readerModeSetter(DropMalformedMode.name) } diff --git a/src/main/scala/com/adidas/analytics/config/DeltaLoadConfiguration.scala b/src/main/scala/com/adidas/analytics/config/DeltaLoadConfiguration.scala index abf250a..02b7317 100644 --- a/src/main/scala/com/adidas/analytics/config/DeltaLoadConfiguration.scala +++ b/src/main/scala/com/adidas/analytics/config/DeltaLoadConfiguration.scala @@ -1,14 +1,14 @@ package com.adidas.analytics.config import com.adidas.analytics.algo.core.Algorithm.{ReadOperation, SafeWriteOperation} -import com.adidas.analytics.config.shared.{ConfigurationContext, DateComponentDerivationConfiguration} +import com.adidas.analytics.config.shared.{ConfigurationContext, DateComponentDerivationConfiguration, MetadataUpdateStrategy} import com.adidas.analytics.util.DataFormat.ParquetFormat import com.adidas.analytics.util.{DataFormat, InputReader, LoadMode, OutputWriter} import org.apache.spark.sql.types.StructType import org.apache.spark.sql.{Row, SparkSession} -trait DeltaLoadConfiguration extends ConfigurationContext { +trait DeltaLoadConfiguration extends ConfigurationContext with MetadataUpdateStrategy { protected val activeRecordsTable: String = configReader.getAs[String]("active_records_table_lake") protected val deltaRecordsTable: Option[String] = configReader.getAsOption[String]("delta_records_table_lake") @@ -34,7 +34,7 @@ object DeltaLoadConfiguration { protected def spark: SparkSession - override protected val partitionColumns: Seq[String] = configReader.getAsSeq[String]("partition_columns") + override protected val targetPartitions: Seq[String] = configReader.getAsSeq[String]("target_partitions") override protected val partitionSourceColumn: String = configReader.getAs[String]("partition_column") override protected val partitionSourceColumnFormat: String = configReader.getAs[String]("partition_column_format") @@ -48,7 +48,8 @@ object DeltaLoadConfiguration { override protected val writer: OutputWriter.AtomicWriter = OutputWriter.newTableLocationWriter( table = activeRecordsTable, format = ParquetFormat(Some(targetSchema)), - partitionColumns = partitionColumns, + targetPartitions = targetPartitions, + metadataConfiguration = getMetaDataUpdateStrategy(activeRecordsTable, targetPartitions), loadMode = LoadMode.OverwritePartitionsWithAddedColumns ) } @@ -61,6 +62,7 @@ object DeltaLoadConfiguration { location => InputReader.newFileSystemReader(s"$location*.parquet", DataFormat.ParquetFormat()) } } + deltaRecordsTable.fold(createInputReaderByPath)(tableName => InputReader.newTableReader(tableName)) } } diff --git a/src/main/scala/com/adidas/analytics/config/FixedSizeStringExtractorConfiguration.scala b/src/main/scala/com/adidas/analytics/config/FixedSizeStringExtractorConfiguration.scala index 8693b80..6a7b0f9 100644 --- a/src/main/scala/com/adidas/analytics/config/FixedSizeStringExtractorConfiguration.scala +++ b/src/main/scala/com/adidas/analytics/config/FixedSizeStringExtractorConfiguration.scala @@ -2,7 +2,7 @@ package com.adidas.analytics.config import com.adidas.analytics.algo.core.Algorithm.{ReadOperation, SafeWriteOperation} import com.adidas.analytics.config.FixedSizeStringExtractorConfiguration._ -import com.adidas.analytics.config.shared.ConfigurationContext +import com.adidas.analytics.config.shared.{ConfigurationContext, MetadataUpdateStrategy} import com.adidas.analytics.util.DataFormat.ParquetFormat import com.adidas.analytics.util.DataFrameUtils.PartitionCriteria import com.adidas.analytics.util.{InputReader, LoadMode, OutputWriter} @@ -12,7 +12,10 @@ import org.joda.time._ import org.slf4j.{Logger, LoggerFactory} -trait FixedSizeStringExtractorConfiguration extends ConfigurationContext with ReadOperation with SafeWriteOperation { +trait FixedSizeStringExtractorConfiguration extends ConfigurationContext + with ReadOperation + with SafeWriteOperation + with MetadataUpdateStrategy { private val logger: Logger = LoggerFactory.getLogger(getClass) @@ -22,20 +25,20 @@ trait FixedSizeStringExtractorConfiguration extends ConfigurationContext with Re private val targetTable: String = configReader.getAs[String]("target_table").trim protected val sourceField: String = configReader.getAs[String]("source_field").trim - protected val partitionColumnsOrdered: Seq[String] = configReader.getAsSeq[String]("partition_columns") - protected val partitionColumnsSet: Set[String] = partitionColumnsOrdered.toSet + protected val targetPartitionsOrdered: Seq[String] = configReader.getAsSeq[String]("target_partitions") + protected val targetPartitionsSet: Set[String] = targetPartitionsOrdered.toSet protected val partitionsCriteria: PartitionCriteria = { if (configReader.contains("select_conditions")) { - if (partitionColumnsOrdered.nonEmpty) { + if (targetPartitionsOrdered.nonEmpty) { parseConditions(configReader.getAsSeq[String]("select_conditions")) } else { logger.warn("Select conditions can be applied to partitioned tables only. Ignoring.") Seq.empty } } else if (configReader.contains("select_rules")) { - if (partitionColumnsOrdered.nonEmpty) { - parseRules(configReader.getAsSeq[String]("select_rules"), partitionColumnsOrdered, partitionColumnsSet) + if (targetPartitionsOrdered.nonEmpty) { + parseRules(configReader.getAsSeq[String]("select_rules"), targetPartitionsOrdered, targetPartitionsSet) } else { logger.warn("Select rules can be applied to partitioned tables only. Ignoring.") Seq.empty @@ -59,8 +62,9 @@ trait FixedSizeStringExtractorConfiguration extends ConfigurationContext with Re override protected val writer: OutputWriter.AtomicWriter = OutputWriter.newTableLocationWriter( table = targetTable, format = ParquetFormat(Some(targetSchema)), - partitionColumns = partitionColumnsOrdered, - loadMode = if (partitionColumnsOrdered.nonEmpty) LoadMode.OverwritePartitionsWithAddedColumns else LoadMode.OverwriteTable + metadataConfiguration = getMetaDataUpdateStrategy(targetTable, targetPartitionsOrdered), + targetPartitions = targetPartitionsOrdered, + loadMode = if (targetPartitionsOrdered.nonEmpty) LoadMode.OverwritePartitionsWithAddedColumns else LoadMode.OverwriteTable ) } @@ -83,37 +87,37 @@ object FixedSizeStringExtractorConfiguration { } } - private def parseRules(rules: Seq[String], partitionColumnsOrdered: Seq[String], partitionColumnsSet: Set[String]): PartitionCriteria = { + private def parseRules(rules: Seq[String], targetPartitionsOrdered: Seq[String], targetPartitionsSet: Set[String]): PartitionCriteria = { if (rules.nonEmpty) { val selectDate = rules.foldLeft(LocalDate.now()) { case (date, RulePattern(period, "-", value)) => - if (!partitionColumnsSet.contains(period)) { + if (!targetPartitionsSet.contains(period)) { throw new RuntimeException(s"Unsupported period: $period") } date.minus(createPeriodByNameAndValue(period, value.toInt)) case (date, RulePattern(period, "+", value)) => - if (!partitionColumnsSet.contains(period)) { + if (!targetPartitionsSet.contains(period)) { throw new RuntimeException(s"Unsupported period: $period") } date.plus(createPeriodByNameAndValue(period, value.toInt)) case rule => throw new IllegalArgumentException(s"Wrong select rule: $rule") } - createCriteriaForDate(selectDate, partitionColumnsOrdered) + createCriteriaForDate(selectDate, targetPartitionsOrdered) } else { Seq.empty } } - private def createCriteriaForDate(date: LocalDate, partitionColumns: Seq[String]): PartitionCriteria = { - partitionColumns match { + private def createCriteriaForDate(date: LocalDate, targetPartitions: Seq[String]): PartitionCriteria = { + targetPartitions match { case Year :: Month :: Day :: Nil => Seq(Year -> date.getYear.toString, Month -> date.getMonthOfYear.toString, Day -> date.getDayOfMonth.toString) case Year :: Month :: Nil => Seq(Year -> date.getYear.toString, Month -> date.getMonthOfYear.toString) case Year :: Week :: Nil => Seq(Year -> date.getYear.toString, Week -> date.getWeekOfWeekyear.toString) - case _ => throw new RuntimeException(s"Unsupported partitioning schema: $partitionColumns") + case _ => throw new RuntimeException(s"Unsupported partitioning schema: $targetPartitions") } } diff --git a/src/main/scala/com/adidas/analytics/config/FullLoadConfiguration.scala b/src/main/scala/com/adidas/analytics/config/FullLoadConfiguration.scala index 7b27f11..fb9414f 100644 --- a/src/main/scala/com/adidas/analytics/config/FullLoadConfiguration.scala +++ b/src/main/scala/com/adidas/analytics/config/FullLoadConfiguration.scala @@ -24,7 +24,7 @@ trait FullLoadConfiguration extends ConfigurationContext with LoadConfiguration case STRUCTURED => OutputWriter.newFileSystemWriter( location = currentDir, format = ParquetFormat(Some(targetSchema)), - partitionColumns = partitionColumns, + targetPartitions = targetPartitions, loadMode = LoadMode.OverwriteTable ) case anotherDataType => throw new RuntimeException(s"Unsupported data type: $anotherDataType for FullLoad.") @@ -34,10 +34,9 @@ trait FullLoadConfiguration extends ConfigurationContext with LoadConfiguration override protected val partitionSourceColumn: String = configReader.getAs[String]("partition_column") override protected val partitionSourceColumnFormat: String = configReader.getAs[String]("partition_column_format") - override protected def loadMode: String = PermissiveMode.name - override protected def readNullValue: Option[String] = { - // all views built on top of BI full loads expect to have empty strings instead of null values, so we have to effectively disable the empty string to null conversion here per default (BDE-2256) super.readNullValue.orElse(Some("XXNULLXXX")) } + + override def loadMode: String = readerModeSetter(PermissiveMode.name) } diff --git a/src/main/scala/com/adidas/analytics/config/NestedFlattenerConfiguration.scala b/src/main/scala/com/adidas/analytics/config/NestedFlattenerConfiguration.scala new file mode 100644 index 0000000..2664da1 --- /dev/null +++ b/src/main/scala/com/adidas/analytics/config/NestedFlattenerConfiguration.scala @@ -0,0 +1,52 @@ +package com.adidas.analytics.config + +import com.adidas.analytics.config.shared.{ConfigurationContext, MetadataUpdateStrategy} +import com.adidas.analytics.algo.core.Algorithm.{ReadOperation, SafeWriteOperation} +import com.adidas.analytics.util.DataFormat.ParquetFormat +import com.adidas.analytics.util.{InputReader, LoadMode, OutputWriter} +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.types.StructType + + +trait NestedFlattenerConfiguration extends ConfigurationContext with ReadOperation with SafeWriteOperation with MetadataUpdateStrategy { + + protected def spark: SparkSession + + private val sourceLocation: String = configReader.getAs[String]("source_location") + private val targetTable: String = configReader.getAs[String]("target_table") + protected val targetPartitions: Option[Seq[String]] = configReader.getAsOptionSeq[String]("target_partitions") + protected val targetSchema: StructType = spark.table(targetTable).schema + protected val charsToReplace: String = configReader.getAs[String]("chars_to_replace") + protected val replacementChar: String = configReader.getAs[String]("replacement_char") + + /* + * Be aware of the naming pattern after flattening, because you also need to include sub-level structs or arrays if you want them. + * Example: events__data if you want to flatten an array called "data" inside a struct called "events" + */ + protected val fieldsToFlatten: Seq[String] = configReader.getAsSeq[String]("fields_to_flatten") + + /* + * columnMapping provides the columns (with user-friendly names) to include in the final DataFrame. + * Columns not in the nameMapping will be excluded + */ + protected val columnMapping: Map[String, String] = configReader.getAsMap("column_mapping") + + override protected val readers: Vector[InputReader] = Vector( + InputReader.newFileSystemReader(sourceLocation, ParquetFormat()) + ) + + override protected val writer: OutputWriter.AtomicWriter = { + var loadMode: LoadMode = LoadMode.OverwritePartitions + if (targetPartitions.isEmpty) + loadMode = LoadMode.OverwriteTable + + OutputWriter.newTableLocationWriter( + targetTable, + ParquetFormat(Some(targetSchema)), + targetPartitions.getOrElse(Seq.empty), + loadMode = loadMode, + metadataConfiguration = getMetaDataUpdateStrategy(targetTable, targetPartitions.getOrElse(Seq.empty)) + ) + } + +} diff --git a/src/main/scala/com/adidas/analytics/config/PartitionMaterializationConfiguration.scala b/src/main/scala/com/adidas/analytics/config/PartitionMaterializationConfiguration.scala index 6b22ba2..f9b1efe 100644 --- a/src/main/scala/com/adidas/analytics/config/PartitionMaterializationConfiguration.scala +++ b/src/main/scala/com/adidas/analytics/config/PartitionMaterializationConfiguration.scala @@ -1,6 +1,7 @@ package com.adidas.analytics.config import com.adidas.analytics.algo.core.Algorithm.{ReadOperation, SafeWriteOperation} +import com.adidas.analytics.config.shared.MetadataUpdateStrategy import com.adidas.analytics.util.DataFormat.ParquetFormat import com.adidas.analytics.util.DataFrameUtils.PartitionCriteria import com.adidas.analytics.util.{ConfigReader, InputReader, LoadMode, OutputWriter} @@ -10,18 +11,23 @@ import org.joda.time._ import org.joda.time.format.DateTimeFormat -trait PartitionMaterializationConfiguration extends ReadOperation with SafeWriteOperation { +trait PartitionMaterializationConfiguration extends ReadOperation + with SafeWriteOperation + with MetadataUpdateStrategy { protected def configReader: ConfigReader + protected def spark: SparkSession + protected def loadMode: LoadMode + protected def partitionsCriteria: Seq[PartitionCriteria] private val sourceTable: String = configReader.getAs[String]("source_table") private val targetTable: String = configReader.getAs[String]("target_table") private val targetSchema: StructType = spark.table(targetTable).schema - protected val partitionColumns: Seq[String] = configReader.getAsSeq[String]("target_partitions").toList + protected val targetPartitions: Seq[String] = configReader.getAsSeq[String]("target_partitions").toList override protected val readers: Vector[InputReader.TableReader] = Vector( InputReader.newTableReader(table = sourceTable) @@ -30,8 +36,9 @@ trait PartitionMaterializationConfiguration extends ReadOperation with SafeWrite override protected val writer: OutputWriter.AtomicWriter = OutputWriter.newTableLocationWriter( table = targetTable, format = ParquetFormat(Some(targetSchema)), - partitionColumns = partitionColumns, - loadMode = loadMode + targetPartitions = targetPartitions, + loadMode = loadMode, + metadataConfiguration = getMetaDataUpdateStrategy(targetTable, targetPartitions) ) override protected def outputFilesNum: Option[Int] = configReader.getAsOption[Int]("number_output_partitions") @@ -79,10 +86,11 @@ object PartitionMaterializationConfiguration { private val fromDateString = configReader.getAs[String]("date_from") private val toDateString = configReader.getAs[String]("date_to") - protected def partitionColumns: Seq[String] + protected def targetPartitions: Seq[String] + protected def configReader: ConfigReader - protected val partitionsCriteria: Seq[PartitionCriteria] = partitionColumns match { + protected val partitionsCriteria: Seq[PartitionCriteria] = targetPartitions match { case Year :: Month :: Day :: Nil => getDatesRange(FormatYearMonthDay, Days.ONE).map { date => Seq(Year -> date.getYear.toString, Month -> date.getMonthOfYear.toString, Day -> date.getDayOfMonth.toString) @@ -95,7 +103,7 @@ object PartitionMaterializationConfiguration { getDatesRange(FormatYearWeek, Weeks.ONE).map { date => Seq(Year -> date.getYear.toString, Week -> date.getWeekOfWeekyear.toString) }.toSeq - case _ => throw new RuntimeException(s"Unable to run materialization by date range: unsupported partitioning schema: $partitionColumns") + case _ => throw new RuntimeException(s"Unable to run materialization by date range: unsupported partitioning schema: $targetPartitions") } private def getDatesRange(pattern: String, period: ReadablePeriod): Iterator[LocalDate] = { @@ -108,4 +116,5 @@ object PartitionMaterializationConfiguration { Iterator.iterate(startDate)(_.plus(period)).takeWhile(!_.isAfter(endDate)) } } + } diff --git a/src/main/scala/com/adidas/analytics/config/shared/ConfigurationContext.scala b/src/main/scala/com/adidas/analytics/config/shared/ConfigurationContext.scala index 6a3fb45..4f4231f 100644 --- a/src/main/scala/com/adidas/analytics/config/shared/ConfigurationContext.scala +++ b/src/main/scala/com/adidas/analytics/config/shared/ConfigurationContext.scala @@ -8,6 +8,7 @@ import org.apache.hadoop.fs.Path trait ConfigurationContext extends Serializable { protected def dfs: DFSWrapper + protected def configLocation: String protected lazy val configReader: ConfigReader = { @@ -15,4 +16,5 @@ trait ConfigurationContext extends Serializable { val jsonContent = dfs.getFileSystem(configFilePath).readFile(configFilePath) ConfigReader(jsonContent) } + } diff --git a/src/main/scala/com/adidas/analytics/config/shared/DateComponentDerivationConfiguration.scala b/src/main/scala/com/adidas/analytics/config/shared/DateComponentDerivationConfiguration.scala index 89ca417..205fe69 100644 --- a/src/main/scala/com/adidas/analytics/config/shared/DateComponentDerivationConfiguration.scala +++ b/src/main/scala/com/adidas/analytics/config/shared/DateComponentDerivationConfiguration.scala @@ -7,6 +7,6 @@ trait DateComponentDerivationConfiguration { protected def partitionSourceColumnFormat: String - protected def partitionColumns: Seq[String] + protected def targetPartitions: Seq[String] } diff --git a/src/main/scala/com/adidas/analytics/config/shared/LoadConfiguration.scala b/src/main/scala/com/adidas/analytics/config/shared/LoadConfiguration.scala index c4e3654..623acdc 100644 --- a/src/main/scala/com/adidas/analytics/config/shared/LoadConfiguration.scala +++ b/src/main/scala/com/adidas/analytics/config/shared/LoadConfiguration.scala @@ -1,7 +1,7 @@ package com.adidas.analytics.config.shared import com.adidas.analytics.util.ConfigReader -import org.apache.spark.sql.catalyst.util.{DropMalformedMode} +import org.apache.spark.sql.catalyst.util.{DropMalformedMode, FailFastMode, PermissiveMode} trait LoadConfiguration { val STRUCTURED = "structured" @@ -26,16 +26,35 @@ trait LoadConfiguration { } - protected val partitionColumns: Seq[String] = configReader.getAsSeq[String]("partition_columns") + protected val targetPartitions: Seq[String] = configReader.getAsSeq[String]("target_partitions") protected val inputDir: String = configReader.getAs[String]("source_dir") protected val fileFormat: String = configReader.getAs[String]("file_format") protected val dataType: String = configReader.getAsOption[String]("data_type").getOrElse(STRUCTURED) + protected val sparkReaderOptions: Map[String, String] = requiredSparkOptions ++ optionalSparkOptions protected def configReader: ConfigReader - protected def loadMode: String = DropMalformedMode.name + + protected def loadMode: String + protected def readNullValue: Option[String] = configReader.getAsOption[String]("null_value") + protected def readQuoteValue: Option[String] = configReader.getAsOption[String]("quote_character") + protected def computeTableStatistics: Boolean = configReader.getAsOption[Boolean]("compute_table_statistics").getOrElse(false) + + protected def readerModeSetter(defaultMode: String): String = { + configReader.getAsOption[String]("reader_mode") match { + case Some(mode) => { + mode.toUpperCase match { + case PermissiveMode.name => PermissiveMode.name + case FailFastMode.name => FailFastMode.name + case DropMalformedMode.name => DropMalformedMode.name + case _ => throw new RuntimeException(s"Invalid reader mode: $mode provided") + } + } + case None => defaultMode + } + } } diff --git a/src/main/scala/com/adidas/analytics/config/shared/MetadataUpdateStrategy.scala b/src/main/scala/com/adidas/analytics/config/shared/MetadataUpdateStrategy.scala new file mode 100644 index 0000000..6498d1c --- /dev/null +++ b/src/main/scala/com/adidas/analytics/config/shared/MetadataUpdateStrategy.scala @@ -0,0 +1,17 @@ +package com.adidas.analytics.config.shared + +import com.adidas.analytics.algo.core.Metadata +import com.adidas.analytics.util.{SparkRecoverPartitionsCustom, SparkRecoverPartitionsNative} + +trait MetadataUpdateStrategy extends ConfigurationContext { + + protected def getMetaDataUpdateStrategy(targetTable: String, + partitionColumns: Seq[String]): Metadata = + configReader.getAsOption[String]("metadata_update_strategy") match { + case Some("SparkRecoverPartitionsNative") => SparkRecoverPartitionsNative(targetTable, partitionColumns) + case Some("SparkRecoverPartitionsCustom") => SparkRecoverPartitionsCustom(targetTable, partitionColumns) + case Some(invalidConfig) => throw new Exception(s"Invalid metadata update strategy ${invalidConfig}") + case None => SparkRecoverPartitionsNative(targetTable, partitionColumns) + } + +} diff --git a/src/main/scala/com/adidas/analytics/util/ConfigReader.scala b/src/main/scala/com/adidas/analytics/util/ConfigReader.scala index 884ab2b..46ddd2a 100644 --- a/src/main/scala/com/adidas/analytics/util/ConfigReader.scala +++ b/src/main/scala/com/adidas/analytics/util/ConfigReader.scala @@ -29,6 +29,13 @@ class ConfigReader(jsonContent: String) extends Serializable { } } + def getAsMap[K, V](propertyName: String): Map[K,V] = { + config.get(propertyName) match { + case Some(JSONObject(obj)) => obj.asInstanceOf[Map[K,V]] + case _ => throw new IllegalArgumentException(s"Unable to find configuration property $propertyName") + } + } + def getAs[T](propertyName: String): T = { config.get(propertyName) match { case Some(property) => property.asInstanceOf[T] @@ -40,6 +47,10 @@ class ConfigReader(jsonContent: String) extends Serializable { config.get(propertyName).map(property => property.asInstanceOf[T]) } + def getAsOptionSeq[T](propertyName: String): Option[Seq[T]] = { + config.get(propertyName).map(_ => getAsSeq(propertyName)) + } + def contains(propertyName: String): Boolean = { config.contains(propertyName) } diff --git a/src/main/scala/com/adidas/analytics/util/DataFrameUtils.scala b/src/main/scala/com/adidas/analytics/util/DataFrameUtils.scala index c0d4708..e45d522 100644 --- a/src/main/scala/com/adidas/analytics/util/DataFrameUtils.scala +++ b/src/main/scala/com/adidas/analytics/util/DataFrameUtils.scala @@ -20,9 +20,9 @@ object DataFrameUtils { } def buildPartitionsCriteriaMatcherFunc(multiplePartitionsCriteria: Seq[PartitionCriteria], schema: StructType): FilterFunction = { - val partitionColumns = multiplePartitionsCriteria.flatten.map(_._1).toSet + val targetPartitions = multiplePartitionsCriteria.flatten.map(_._1).toSet val fieldNameToMatchFunctionMapping = schema.fields.filter { - case StructField(name, _, _, _) => partitionColumns.contains(name) + case StructField(name, _, _, _) => targetPartitions.contains(name) }.map { case StructField(name, _: ByteType, _, _) => name -> ((r: Row, value: String) => r.getAs[Byte](name) == value.toByte) case StructField(name, _: ShortType, _, _) => name -> ((r: Row, value: String) => r.getAs[Short](name) == value.toShort) @@ -53,12 +53,12 @@ object DataFrameUtils { implicit class DataFrameHelper(df: DataFrame) { - def collectPartitions(partitionColumns: Seq[String]): Seq[PartitionCriteria] = { - logger.info(s"Collecting unique partitions for partitions columns (${partitionColumns.mkString(", ")})") - val partitions = df.selectExpr(partitionColumns: _*).distinct().collect() + def collectPartitions(targetPartitions: Seq[String]): Seq[PartitionCriteria] = { + logger.info(s"Collecting unique partitions for partitions columns (${targetPartitions.mkString(", ")})") + val partitions = df.selectExpr(targetPartitions: _*).distinct().collect() partitions.map { row => - partitionColumns.map { columnName => + targetPartitions.map { columnName => Option(row.getAs[Any](columnName)) match { case Some(columnValue) => columnName -> columnValue.toString case None => throw new RuntimeException(s"Partition column '$columnName' contains null value") diff --git a/src/main/scala/com/adidas/analytics/util/HadoopLoadHelper.scala b/src/main/scala/com/adidas/analytics/util/HadoopLoadHelper.scala index d09fd3c..b8abf25 100644 --- a/src/main/scala/com/adidas/analytics/util/HadoopLoadHelper.scala +++ b/src/main/scala/com/adidas/analytics/util/HadoopLoadHelper.scala @@ -36,6 +36,9 @@ object HadoopLoadHelper { def backupDirectoryContent(fs: FileSystem, sourceDir: Path, backupDir: Path): Unit = { logger.info(s"Creating backup $sourceDir -> $backupDir") try { + if (fs.exists(backupDir) || !fs.mkdirs(backupDir)) { + throw new IOException(s"Unable to create target directory ${backupDir}") + } moveChildren(fs, sourceDir, backupDir) logger.info("Backup successfully created") } catch { diff --git a/src/main/scala/com/adidas/analytics/util/OutputWriter.scala b/src/main/scala/com/adidas/analytics/util/OutputWriter.scala index 0f8dcb2..87d3fcf 100644 --- a/src/main/scala/com/adidas/analytics/util/OutputWriter.scala +++ b/src/main/scala/com/adidas/analytics/util/OutputWriter.scala @@ -1,5 +1,6 @@ package com.adidas.analytics.util +import com.adidas.analytics.algo.core.Metadata import com.adidas.analytics.util.DataFrameUtils._ import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.spark.sql.{DataFrameWriter, _} @@ -12,15 +13,15 @@ sealed abstract class OutputWriter { val logger: Logger = LoggerFactory.getLogger(getClass) - def partitionColumns: Seq[String] + def targetPartitions: Seq[String] def options: Map[String, String] def write(dfs: DFSWrapper, df: DataFrame): Unit protected def getWriter(df: DataFrame): DataFrameWriter[Row] = { - if (partitionColumns.nonEmpty) { - df.write.partitionBy(partitionColumns: _*) + if (targetPartitions.nonEmpty) { + df.write.partitionBy(targetPartitions: _*) } else { df.write } @@ -34,14 +35,14 @@ object OutputWriter { * Factory method which creates TableWriter * * @param table target table - * @param partitionColumns specifies how data should be partitioned + * @param targetPartitions specifies how data should be partitioned * @param options options which are provided to Spark DataFrameWriter * @param loadMode specifies LoadMode for the writer (see more details for SaveMode in Spark documentation) * @return TableWriter */ - def newTableWriter(table: String, partitionColumns: Seq[String] = Seq.empty, options: Map[String, String] = Map.empty, + def newTableWriter(table: String, targetPartitions: Seq[String] = Seq.empty, options: Map[String, String] = Map.empty, loadMode: LoadMode = LoadMode.OverwritePartitionsWithAddedColumns): TableWriter = { - TableWriter(table, partitionColumns, options, loadMode) + TableWriter(table, targetPartitions, options, loadMode) } /** @@ -49,14 +50,15 @@ object OutputWriter { * * @param table target table which location is used for writing data to * @param format format of result data - * @param partitionColumns specifies how data should be partitioned + * @param targetPartitions specifies how data should be partitioned * @param options options which are provided to Spark DataFrameWriter * @param loadMode specifies LoadMode for the writer (see more details for SaveMode in Spark documentation) * @return TableLocationWriter */ - def newTableLocationWriter(table: String, format: DataFormat, partitionColumns: Seq[String] = Seq.empty, - options: Map[String, String] = Map.empty, loadMode: LoadMode = LoadMode.OverwritePartitionsWithAddedColumns): TableLocationWriter = { - TableLocationWriter(table, format, partitionColumns, options, loadMode) + def newTableLocationWriter(table: String, format: DataFormat, targetPartitions: Seq[String] = Seq.empty, + options: Map[String, String] = Map.empty, loadMode: LoadMode = LoadMode.OverwritePartitionsWithAddedColumns, + metadataConfiguration: Metadata): TableLocationWriter = { + TableLocationWriter(table, format, targetPartitions, options, loadMode, metadataConfiguration) } /** @@ -64,14 +66,14 @@ object OutputWriter { * * @param location output location on the filesystem * @param format format of result data - * @param partitionColumns specifies how data should be partitioned + * @param targetPartitions specifies how data should be partitioned * @param options options which are provided to Spark DataFrameWriter * @param loadMode specifies LoadMode for the writer (see more details for SaveMode in Spark documentation) * @return FileSystemWriter */ - def newFileSystemWriter(location: String, format: DataFormat, partitionColumns: Seq[String] = Seq.empty, + def newFileSystemWriter(location: String, format: DataFormat, targetPartitions: Seq[String] = Seq.empty, options: Map[String, String] = Map.empty, loadMode: LoadMode = LoadMode.OverwritePartitionsWithAddedColumns): FileSystemWriter = { - FileSystemWriter(location, format, partitionColumns, options, loadMode) + FileSystemWriter(location, format, targetPartitions, options, loadMode) } /** @@ -97,7 +99,7 @@ object OutputWriter { } protected def writeSafe(dfs: DFSWrapper, df: DataFrame, finalLocation: String, loadMode: LoadMode): Unit = { - lazy val partitionsCriteria = df.collectPartitions(partitionColumns) + lazy val partitionsCriteria = df.collectPartitions(targetPartitions) val finalPath = new Path(finalLocation) val fs = dfs.getFileSystem(finalPath) @@ -107,8 +109,6 @@ object OutputWriter { val tempBackupPath = new Path(tempPath, "backup") fs.delete(tempPath, true) - fs.mkdirs(tempDataPath) - fs.mkdirs(tempBackupPath) loadMode match { case LoadMode.OverwriteTable => @@ -183,12 +183,12 @@ object OutputWriter { throw new RuntimeException(s"Unable to load data to $finalPath", e) } } else { - logger.warn(s"Unable to load data, output data has no partitions for partition columns $partitionColumns") + logger.warn(s"Unable to load data, output data has no partitions for partition columns $targetPartitions") } } } - case class TableWriter(table: String, partitionColumns: Seq[String], options: Map[String, String], + case class TableWriter(table: String, targetPartitions: Seq[String], options: Map[String, String], loadMode: LoadMode) extends OutputWriter { override def write(dfs: DFSWrapper, df: DataFrame): Unit = { @@ -201,7 +201,7 @@ object OutputWriter { } } - case class FileSystemWriter(location: String, format: DataFormat, partitionColumns: Seq[String], + case class FileSystemWriter(location: String, format: DataFormat, targetPartitions: Seq[String], options: Map[String, String], loadMode: LoadMode) extends AtomicWriter { override def write(dfs: DFSWrapper, df: DataFrame): Unit = { @@ -213,17 +213,18 @@ object OutputWriter { } } - case class TableLocationWriter(table: String, format: DataFormat, partitionColumns: Seq[String], - options: Map[String, String], loadMode: LoadMode) extends AtomicWriter { + case class TableLocationWriter(table: String, format: DataFormat, targetPartitions: Seq[String], + options: Map[String, String], loadMode: LoadMode, + metadataConfiguration: Metadata) extends AtomicWriter { override def write(dfs: DFSWrapper, df: DataFrame): Unit = { val spark = df.sparkSession val location = getTableLocation(spark) writeUnsafe(dfs, df, location, loadMode) - if (partitionColumns.nonEmpty){ - spark.catalog.recoverPartitions(table) + if (targetPartitions.nonEmpty){ + metadataConfiguration.recoverPartitions(df) } else { - spark.catalog.refreshTable(table) + metadataConfiguration.refreshTable(df) } } @@ -231,10 +232,10 @@ object OutputWriter { val spark = df.sparkSession val location = getTableLocation(spark) writeSafe(dfs, df, location, loadMode) - if (partitionColumns.nonEmpty){ - spark.catalog.recoverPartitions(table) + if (targetPartitions.nonEmpty){ + metadataConfiguration.recoverPartitions(df) } else { - spark.catalog.refreshTable(table) + metadataConfiguration.refreshTable(df) } } diff --git a/src/main/scala/com/adidas/analytics/util/SparkRecoverPartitionsCustom.scala b/src/main/scala/com/adidas/analytics/util/SparkRecoverPartitionsCustom.scala new file mode 100644 index 0000000..8c57fcf --- /dev/null +++ b/src/main/scala/com/adidas/analytics/util/SparkRecoverPartitionsCustom.scala @@ -0,0 +1,49 @@ +package com.adidas.analytics.util + +import com.adidas.analytics.algo.core.Metadata +import org.apache.spark.sql._ +import org.apache.spark.sql.functions.col + +import scala.collection.JavaConversions._ + +case class SparkRecoverPartitionsCustom(override val tableName: String, + override val targetPartitions: Seq[String]) extends Metadata { + + override def recoverPartitions(outputDataFrame: DataFrame): Unit = { + + val spark: SparkSession = outputDataFrame.sparkSession + + val targetPartitionsColumns: Seq[Column] = targetPartitions.map(partitionString => col(partitionString)) + + val distinctPartitions: DataFrame = outputDataFrame.select(targetPartitionsColumns: _*).distinct + + val sqlStatements: Dataset[String] = generateAddPartitionStatements(distinctPartitions) + + sqlStatements.collectAsList().foreach((statement: String) => spark.sql(statement)) + } + + private def generateAddPartitionStatements(partitionsDataset: DataFrame): Dataset[String] = { + partitionsDataset.map(row => { + val partitionStatementValues: Seq[String] = targetPartitions + .map(partitionString => s"${partitionString}=${getParameterValue(row, partitionString)}") + + s"ALTER TABLE ${tableName} ADD IF NOT EXISTS PARTITION(${partitionStatementValues.mkString(",")})" + })(Encoders.STRING) + } + + private def getParameterValue(row: Row, partitionString: String): String = + createParameterValue(row.get(row.fieldIndex(partitionString))) + + private def createParameterValue(partitionRawValue: Any): String = { + partitionRawValue match { + case value: java.lang.Short => value.toString + case value: java.lang.Integer => value.toString + case value: scala.Predef.String => "'" + value + "'" + case null => throw new Exception("Partition Value is null. No support for null partitions!") + case value => throw new Exception("Unsupported partition DataType: " + value.getClass) + + } + + } + +} diff --git a/src/main/scala/com/adidas/analytics/util/SparkRecoverPartitionsNative.scala b/src/main/scala/com/adidas/analytics/util/SparkRecoverPartitionsNative.scala new file mode 100644 index 0000000..c79c8c4 --- /dev/null +++ b/src/main/scala/com/adidas/analytics/util/SparkRecoverPartitionsNative.scala @@ -0,0 +1,11 @@ +package com.adidas.analytics.util + +import com.adidas.analytics.algo.core.Metadata +import org.apache.spark.sql.DataFrame + +case class SparkRecoverPartitionsNative(override val tableName: String, + override val targetPartitions: Seq[String]) extends Metadata { + + override def recoverPartitions(outputDataFrame: DataFrame): Unit = outputDataFrame.sparkSession.catalog.recoverPartitions(tableName) + +} diff --git a/src/test/resources/AlgorithmTemplateTest/algorithm_template_params.json b/src/test/resources/AlgorithmTemplateTest/algorithm_template_params.json index 3c8f42a..d80538d 100644 --- a/src/test/resources/AlgorithmTemplateTest/algorithm_template_params.json +++ b/src/test/resources/AlgorithmTemplateTest/algorithm_template_params.json @@ -6,7 +6,7 @@ "has_header": false, "partition_column": "", "partition_column_format": "", - "partition_columns": [], + "target_partitions": [], "source_dir": "/tmp/tests/test_landing/test/test_table/data", "target_table": "test_lake.test_table" } \ No newline at end of file diff --git a/src/test/resources/AppendLoadTest/different_schemas/params.json b/src/test/resources/AppendLoadTest/different_schemas/params.json index b45f1b2..cd69cc2 100644 --- a/src/test/resources/AppendLoadTest/different_schemas/params.json +++ b/src/test/resources/AppendLoadTest/different_schemas/params.json @@ -10,7 +10,7 @@ "(?<=[0-9]{4})([0-9]{2})(?=[0-9]{2})", "(?<=[0-9]{6})([0-9]{2})" ], - "partition_columns": [ + "target_partitions": [ "year", "month", "day" diff --git a/src/test/resources/AppendLoadTest/duplicate_values/params.json b/src/test/resources/AppendLoadTest/duplicate_values/params.json index b45f1b2..cd69cc2 100644 --- a/src/test/resources/AppendLoadTest/duplicate_values/params.json +++ b/src/test/resources/AppendLoadTest/duplicate_values/params.json @@ -10,7 +10,7 @@ "(?<=[0-9]{4})([0-9]{2})(?=[0-9]{2})", "(?<=[0-9]{6})([0-9]{2})" ], - "partition_columns": [ + "target_partitions": [ "year", "month", "day" diff --git a/src/test/resources/AppendLoadTest/hierarchical_load/params.json b/src/test/resources/AppendLoadTest/hierarchical_load/params.json index b45f1b2..cd69cc2 100644 --- a/src/test/resources/AppendLoadTest/hierarchical_load/params.json +++ b/src/test/resources/AppendLoadTest/hierarchical_load/params.json @@ -10,7 +10,7 @@ "(?<=[0-9]{4})([0-9]{2})(?=[0-9]{2})", "(?<=[0-9]{6})([0-9]{2})" ], - "partition_columns": [ + "target_partitions": [ "year", "month", "day" diff --git a/src/test/resources/AppendLoadTest/main_test/params.json b/src/test/resources/AppendLoadTest/main_test/params.json index dd5fdd5..03ccf8c 100644 --- a/src/test/resources/AppendLoadTest/main_test/params.json +++ b/src/test/resources/AppendLoadTest/main_test/params.json @@ -10,7 +10,7 @@ "(?<=[0-9]{4})([0-9]{2})(?=[0-9]{2})", "(?<=[0-9]{6})([0-9]{2})" ], - "partition_columns": [ + "target_partitions": [ "year", "month", "day" diff --git a/src/test/resources/AppendLoadTest/missing_columns/params.json b/src/test/resources/AppendLoadTest/missing_columns/params.json index 6f5d5c7..dcc3fa3 100644 --- a/src/test/resources/AppendLoadTest/missing_columns/params.json +++ b/src/test/resources/AppendLoadTest/missing_columns/params.json @@ -10,7 +10,7 @@ "(?<=[0-9]{4})([0-9]{2})(?=[0-9]{2})", "(?<=[0-9]{6})([0-9]{2})" ], - "partition_columns": [ + "target_partitions": [ "year", "month", "day" diff --git a/src/test/resources/AppendLoadTest/multiple_source_files/params.json b/src/test/resources/AppendLoadTest/multiple_source_files/params.json index b45f1b2..cd69cc2 100644 --- a/src/test/resources/AppendLoadTest/multiple_source_files/params.json +++ b/src/test/resources/AppendLoadTest/multiple_source_files/params.json @@ -10,7 +10,7 @@ "(?<=[0-9]{4})([0-9]{2})(?=[0-9]{2})", "(?<=[0-9]{6})([0-9]{2})" ], - "partition_columns": [ + "target_partitions": [ "year", "month", "day" diff --git a/src/test/resources/AppendLoadTest/parquet_test/params.json b/src/test/resources/AppendLoadTest/parquet_test/params.json index 6f09606..ba1a8a4 100644 --- a/src/test/resources/AppendLoadTest/parquet_test/params.json +++ b/src/test/resources/AppendLoadTest/parquet_test/params.json @@ -10,7 +10,7 @@ "(?<=/[a-zA-Z]{0,20}[-_]{0,1}[0-9]{4})([0-9]{2})(?=[0-9]{2})", "(?<=/[a-zA-Z]{0,20}[-_]{0,1}[0-9]{4}[0-9]{2})([0-9]{2})(?=.*\\.[a-zA-Z]{3})" ], - "partition_columns": [ + "target_partitions": [ "year", "month", "day" diff --git a/src/test/resources/AppendLoadTest/partition_from_full_path/params.json b/src/test/resources/AppendLoadTest/partition_from_full_path/params.json index 6741e97..fab82a6 100644 --- a/src/test/resources/AppendLoadTest/partition_from_full_path/params.json +++ b/src/test/resources/AppendLoadTest/partition_from_full_path/params.json @@ -10,7 +10,7 @@ "(?<=year=[0-9]{4})/month=([0-9]{2})/(?=day=[0-9]{2})", "(?<=year=[0-9]{4}/month=[0-9]{2})/day=([0-9]{2})" ], - "partition_columns": [ + "target_partitions": [ "year", "month", "day" diff --git a/src/test/resources/AppendLoadTest/reader_mode_specification/data_20180422-00001.psv b/src/test/resources/AppendLoadTest/reader_mode_specification/data_20180422-00001.psv new file mode 100644 index 0000000..a0a23d6 --- /dev/null +++ b/src/test/resources/AppendLoadTest/reader_mode_specification/data_20180422-00001.psv @@ -0,0 +1,8 @@ +salesorder|item|recordmode|date|customer +7|2|N|2018-04-20|customer5 +7|3|N|2018-04-20|customer5 +7|1|N|2018-04-20|customer5 +8|1|N|2018-04-20|customer5 +8|2|N|2018-04-20|customer5 +8|3|N|2018-04-20|customer5 +9|3|N|2018-04-18|customer5 \ No newline at end of file diff --git a/src/test/resources/AppendLoadTest/reader_mode_specification/lake_data_post.psv b/src/test/resources/AppendLoadTest/reader_mode_specification/lake_data_post.psv new file mode 100644 index 0000000..e4660be --- /dev/null +++ b/src/test/resources/AppendLoadTest/reader_mode_specification/lake_data_post.psv @@ -0,0 +1,26 @@ +1|1||2016-06-01|customer1|2016|6|1 +1|2||2016-06-01|customer1|2016|6|1 +1|3||2016-06-01|customer1|2016|6|1 +2|1||2017-02-15|customer2|2017|2|15 +2|2||2017-02-15|customer2|2017|2|15 +2|3||2017-02-15|customer2|2017|2|15 +3|1||2017-02-15|customer1|2017|2|15 +3|2||2017-02-15|customer1|2017|2|15 +3|3||2017-02-15|customer1|2017|2|15 +4|1||2017-04-30|customer3|2017|4|30 +4|2||2017-04-30|customer3|2017|4|30 +4|3||2017-04-30|customer3|2017|4|30 +4|4||2017-04-30|customer3|2017|4|30 +5|1||2017-05-10|customer4|2017|5|10 +5|2||2017-05-10|customer4|2017|5|10 +5|3||2017-05-10|customer4|2017|5|10 +6|1||2018-06-01|customer2|2018|6|1 +6|2||2018-06-01|customer2|2018|6|1 +6|3||2018-06-01|customer2|2018|6|1 +7|1|N|2018-04-20|customer5|2018|04|22 +7|2|N|2018-04-20|customer5|2018|04|22 +7|3|N|2018-04-20|customer5|2018|04|22 +8|1|N|2018-04-20|customer5|2018|04|22 +8|2|N|2018-04-20|customer5|2018|04|22 +8|3|N|2018-04-20|customer5|2018|04|22 +9|3|N|2018-04-18|customer5|2018|04|22 diff --git a/src/test/resources/AppendLoadTest/reader_mode_specification/lake_data_pre.psv b/src/test/resources/AppendLoadTest/reader_mode_specification/lake_data_pre.psv new file mode 100644 index 0000000..dbf804f --- /dev/null +++ b/src/test/resources/AppendLoadTest/reader_mode_specification/lake_data_pre.psv @@ -0,0 +1,19 @@ +1|1||2016-06-01|customer1|2016|6|1 +1|2||2016-06-01|customer1|2016|6|1 +1|3||2016-06-01|customer1|2016|6|1 +2|1||2017-02-15|customer2|2017|2|15 +2|2||2017-02-15|customer2|2017|2|15 +2|3||2017-02-15|customer2|2017|2|15 +3|1||2017-02-15|customer1|2017|2|15 +3|2||2017-02-15|customer1|2017|2|15 +3|3||2017-02-15|customer1|2017|2|15 +4|1||2017-04-30|customer3|2017|4|30 +4|2||2017-04-30|customer3|2017|4|30 +4|3||2017-04-30|customer3|2017|4|30 +4|4||2017-04-30|customer3|2017|4|30 +5|1||2017-05-10|customer4|2017|5|10 +5|2||2017-05-10|customer4|2017|5|10 +5|3||2017-05-10|customer4|2017|5|10 +6|1||2018-06-01|customer2|2018|6|1 +6|2||2018-06-01|customer2|2018|6|1 +6|3||2018-06-01|customer2|2018|6|1 \ No newline at end of file diff --git a/src/test/resources/AppendLoadTest/reader_mode_specification/params.json b/src/test/resources/AppendLoadTest/reader_mode_specification/params.json new file mode 100644 index 0000000..bff8c3d --- /dev/null +++ b/src/test/resources/AppendLoadTest/reader_mode_specification/params.json @@ -0,0 +1,19 @@ +{ + "target_table": "test_lake.test_table", + "source_dir": "/tmp/tests/test_landing/test_table/data", + "header_dir": "/tmp/tests/test_landing/test_table/header", + "file_format": "dsv", + "delimiter": "|", + "has_header": true, + "regex_filename": [ + "([0-9]{4})(?=[0-9]{2})(?=[0-9]{2})", + "(?<=[0-9]{4})([0-9]{2})(?=[0-9]{2})", + "(?<=[0-9]{6})([0-9]{2})" + ], + "target_partitions": [ + "year", + "month", + "day" + ], + "reader_mode": "dropmalformed" +} \ No newline at end of file diff --git a/src/test/resources/AppendLoadTest/reader_mode_specification/params_failfast_mode.json b/src/test/resources/AppendLoadTest/reader_mode_specification/params_failfast_mode.json new file mode 100644 index 0000000..60f93c3 --- /dev/null +++ b/src/test/resources/AppendLoadTest/reader_mode_specification/params_failfast_mode.json @@ -0,0 +1,19 @@ +{ + "target_table": "test_lake.test_table", + "source_dir": "/tmp/tests/test_landing/test_table/data", + "header_dir": "/tmp/tests/test_landing/test_table/header", + "file_format": "dsv", + "delimiter": "|", + "has_header": true, + "regex_filename": [ + "([0-9]{4})(?=[0-9]{2})(?=[0-9]{2})", + "(?<=[0-9]{4})([0-9]{2})(?=[0-9]{2})", + "(?<=[0-9]{6})([0-9]{2})" + ], + "target_partitions": [ + "year", + "month", + "day" + ], + "reader_mode": "failfast" +} \ No newline at end of file diff --git a/src/test/resources/AppendLoadTest/reader_mode_specification/params_invalid_reader_mode.json b/src/test/resources/AppendLoadTest/reader_mode_specification/params_invalid_reader_mode.json new file mode 100644 index 0000000..a314524 --- /dev/null +++ b/src/test/resources/AppendLoadTest/reader_mode_specification/params_invalid_reader_mode.json @@ -0,0 +1,19 @@ +{ + "target_table": "test_lake.test_table", + "source_dir": "/tmp/tests/test_landing/test_table/data", + "header_dir": "/tmp/tests/test_landing/test_table/header", + "file_format": "dsv", + "delimiter": "|", + "has_header": true, + "regex_filename": [ + "([0-9]{4})(?=[0-9]{2})(?=[0-9]{2})", + "(?<=[0-9]{4})([0-9]{2})(?=[0-9]{2})", + "(?<=[0-9]{6})([0-9]{2})" + ], + "target_partitions": [ + "year", + "month", + "day" + ], + "reader_mode": "invalid_mode" +} \ No newline at end of file diff --git a/src/test/resources/AppendLoadTest/reader_mode_specification/params_no_reader_mode.json b/src/test/resources/AppendLoadTest/reader_mode_specification/params_no_reader_mode.json new file mode 100644 index 0000000..03ccf8c --- /dev/null +++ b/src/test/resources/AppendLoadTest/reader_mode_specification/params_no_reader_mode.json @@ -0,0 +1,18 @@ +{ + "target_table": "test_lake.test_table", + "source_dir": "/tmp/tests/test_landing/test_table/data", + "header_dir": "/tmp/tests/test_landing/test_table/header", + "file_format": "dsv", + "delimiter": "|", + "has_header": true, + "regex_filename": [ + "([0-9]{4})(?=[0-9]{2})(?=[0-9]{2})", + "(?<=[0-9]{4})([0-9]{2})(?=[0-9]{2})", + "(?<=[0-9]{6})([0-9]{2})" + ], + "target_partitions": [ + "year", + "month", + "day" + ] +} \ No newline at end of file diff --git a/src/test/resources/AppendLoadTest/reader_mode_specification/params_permissive_mode.json b/src/test/resources/AppendLoadTest/reader_mode_specification/params_permissive_mode.json new file mode 100644 index 0000000..aad0739 --- /dev/null +++ b/src/test/resources/AppendLoadTest/reader_mode_specification/params_permissive_mode.json @@ -0,0 +1,19 @@ +{ + "target_table": "test_lake.test_table", + "source_dir": "/tmp/tests/test_landing/test_table/data", + "header_dir": "/tmp/tests/test_landing/test_table/header", + "file_format": "dsv", + "delimiter": "|", + "has_header": true, + "regex_filename": [ + "([0-9]{4})(?=[0-9]{2})(?=[0-9]{2})", + "(?<=[0-9]{4})([0-9]{2})(?=[0-9]{2})", + "(?<=[0-9]{6})([0-9]{2})" + ], + "target_partitions": [ + "year", + "month", + "day" + ], + "reader_mode": "permissive" +} \ No newline at end of file diff --git a/src/test/resources/AppendLoadTest/reader_mode_specification/target_schema.json b/src/test/resources/AppendLoadTest/reader_mode_specification/target_schema.json new file mode 100644 index 0000000..e608d85 --- /dev/null +++ b/src/test/resources/AppendLoadTest/reader_mode_specification/target_schema.json @@ -0,0 +1,44 @@ +{ + "type" : "struct", + "fields" : [ { + "name" : "salesorder", + "type" : "integer", + "nullable" : true, + "metadata" : { } + }, { + "name" : "item", + "type" : "integer", + "nullable" : true, + "metadata" : { } + }, { + "name" : "recordmode", + "type" : "string", + "nullable" : true, + "metadata" : { } + }, { + "name" : "date", + "type" : "string", + "nullable" : true, + "metadata" : { } + }, { + "name" : "customer", + "type" : "string", + "nullable" : true, + "metadata" : { } + }, { + "name" : "year", + "type" : "short", + "nullable" : true, + "metadata" : { } + }, { + "name" : "month", + "type" : "short", + "nullable" : true, + "metadata" : { } + }, { + "name" : "day", + "type" : "short", + "nullable" : true, + "metadata" : { } + } ] +} diff --git a/src/test/resources/AppendLoadTest/reader_mode_specification/wrong_data_20180422-00001.psv b/src/test/resources/AppendLoadTest/reader_mode_specification/wrong_data_20180422-00001.psv new file mode 100644 index 0000000..de9d848 --- /dev/null +++ b/src/test/resources/AppendLoadTest/reader_mode_specification/wrong_data_20180422-00001.psv @@ -0,0 +1,9 @@ +salesorder|item|recordmode|date|customer +7|2|N|2018-04-20|customer4|some|wrong|columns +7|2|N|2018-04-20|customer5|very|wrong +7|3|N|2018-04-20|customer5 +7|1|N|2018-04-20|customer5 +8|1|N|2018-04-20|customer5 +8|2|N|2018-04-20|customer5 +8|3|N|2018-04-20|customer5 +9|3|N|2018-04-18|customer5 \ No newline at end of file diff --git a/src/test/resources/AppendLoadTest/similar_schemas/params.json b/src/test/resources/AppendLoadTest/similar_schemas/params.json index b45f1b2..cd69cc2 100644 --- a/src/test/resources/AppendLoadTest/similar_schemas/params.json +++ b/src/test/resources/AppendLoadTest/similar_schemas/params.json @@ -10,7 +10,7 @@ "(?<=[0-9]{4})([0-9]{2})(?=[0-9]{2})", "(?<=[0-9]{6})([0-9]{2})" ], - "partition_columns": [ + "target_partitions": [ "year", "month", "day" diff --git a/src/test/resources/DeltaLoadTest/csv_test/params.json b/src/test/resources/DeltaLoadTest/csv_test/params.json index 7f6499c..a72de54 100644 --- a/src/test/resources/DeltaLoadTest/csv_test/params.json +++ b/src/test/resources/DeltaLoadTest/csv_test/params.json @@ -3,7 +3,7 @@ "business_key": ["salesorder", "item"], "delta_records_table_lake": "test_lake.delta_load_delta_data", "technical_key": ["sort_timestamp", "datapakid", "partno", "record"], - "partition_columns": ["year", "month", "day"], + "target_partitions": ["year", "month", "day"], "partition_column": "date", "partition_column_format": "yyyyMMdd" } \ No newline at end of file diff --git a/src/test/resources/DeltaLoadTest/csv_test/params_part.json b/src/test/resources/DeltaLoadTest/csv_test/params_part.json index f0e5842..e450f98 100644 --- a/src/test/resources/DeltaLoadTest/csv_test/params_part.json +++ b/src/test/resources/DeltaLoadTest/csv_test/params_part.json @@ -3,7 +3,7 @@ "business_key": ["salesorder", "item"], "delta_records_table_lake": "test_lake.delta_load_delta_data", "technical_key": ["sort_timestamp", "datapakid", "partno", "record"], - "partition_columns": ["customer"], + "target_partitions": ["customer"], "partition_column": "", "partition_column_format": "" } \ No newline at end of file diff --git a/src/test/resources/DeltaLoadTest/parquet_test_delta_init/params.json b/src/test/resources/DeltaLoadTest/parquet_test_delta_init/params.json index ac3ecc3..fdbabc2 100644 --- a/src/test/resources/DeltaLoadTest/parquet_test_delta_init/params.json +++ b/src/test/resources/DeltaLoadTest/parquet_test_delta_init/params.json @@ -3,7 +3,7 @@ "business_key": ["salesorder", "item"], "delta_records_file_path": "/tmp/tests/", "technical_key": ["m3d_timestamp", "datapakid", "partno", "record"], - "partition_columns": ["year", "month", "day"], + "target_partitions": ["year", "month", "day"], "partition_column": "date", "partition_column_format": "yyyyMMdd" } \ No newline at end of file diff --git a/src/test/resources/DeltaLoadTest/parquet_test_delta_merge_additional_columns/params.json b/src/test/resources/DeltaLoadTest/parquet_test_delta_merge_additional_columns/params.json index 1342901..fb9b4f7 100644 --- a/src/test/resources/DeltaLoadTest/parquet_test_delta_merge_additional_columns/params.json +++ b/src/test/resources/DeltaLoadTest/parquet_test_delta_merge_additional_columns/params.json @@ -3,7 +3,7 @@ "business_key": ["salesorder", "item"], "delta_records_file_path": "hdfs:///tmp/tests/", "technical_key": ["m3d_timestamp", "datapakid", "partno", "record"], - "partition_columns": ["year", "month", "day"], + "target_partitions": ["year", "month", "day"], "partition_column": "date", "partition_column_format": "yyyyMMdd" } \ No newline at end of file diff --git a/src/test/resources/DeltaLoadTest/parquet_test_delta_merge_missing_columns/params.json b/src/test/resources/DeltaLoadTest/parquet_test_delta_merge_missing_columns/params.json index 1342901..fb9b4f7 100644 --- a/src/test/resources/DeltaLoadTest/parquet_test_delta_merge_missing_columns/params.json +++ b/src/test/resources/DeltaLoadTest/parquet_test_delta_merge_missing_columns/params.json @@ -3,7 +3,7 @@ "business_key": ["salesorder", "item"], "delta_records_file_path": "hdfs:///tmp/tests/", "technical_key": ["m3d_timestamp", "datapakid", "partno", "record"], - "partition_columns": ["year", "month", "day"], + "target_partitions": ["year", "month", "day"], "partition_column": "date", "partition_column_format": "yyyyMMdd" } \ No newline at end of file diff --git a/src/test/resources/DeltaLoadTest/parquet_test_delta_merge_partitioned/params.json b/src/test/resources/DeltaLoadTest/parquet_test_delta_merge_partitioned/params.json index ac3ecc3..fdbabc2 100644 --- a/src/test/resources/DeltaLoadTest/parquet_test_delta_merge_partitioned/params.json +++ b/src/test/resources/DeltaLoadTest/parquet_test_delta_merge_partitioned/params.json @@ -3,7 +3,7 @@ "business_key": ["salesorder", "item"], "delta_records_file_path": "/tmp/tests/", "technical_key": ["m3d_timestamp", "datapakid", "partno", "record"], - "partition_columns": ["year", "month", "day"], + "target_partitions": ["year", "month", "day"], "partition_column": "date", "partition_column_format": "yyyyMMdd" } \ No newline at end of file diff --git a/src/test/resources/DeltaLoadTest/parquet_test_delta_merge_unpartitioned/params.json b/src/test/resources/DeltaLoadTest/parquet_test_delta_merge_unpartitioned/params.json index 1342901..fb9b4f7 100644 --- a/src/test/resources/DeltaLoadTest/parquet_test_delta_merge_unpartitioned/params.json +++ b/src/test/resources/DeltaLoadTest/parquet_test_delta_merge_unpartitioned/params.json @@ -3,7 +3,7 @@ "business_key": ["salesorder", "item"], "delta_records_file_path": "hdfs:///tmp/tests/", "technical_key": ["m3d_timestamp", "datapakid", "partno", "record"], - "partition_columns": ["year", "month", "day"], + "target_partitions": ["year", "month", "day"], "partition_column": "date", "partition_column_format": "yyyyMMdd" } \ No newline at end of file diff --git a/src/test/resources/FailFastIntegrationTest/landing/new_data.psv b/src/test/resources/FailFastIntegrationTest/landing/new_data.psv new file mode 100644 index 0000000..b93ab27 --- /dev/null +++ b/src/test/resources/FailFastIntegrationTest/landing/new_data.psv @@ -0,0 +1,25 @@ +1|1||20160601|customer1|article1|150 +1|2||20160601|customer1|article2|200 +1|3||20160601|customer1|article3|50 +2|1||20170215|customer2|article4|10 +2|2||20170215|customer2|article5|50 +2|3||20170215|customer2|article1|30 +3|1||20170215|customer1|article6|200 +3|2||20170215|customer1|article2|120 +3|3||20170215|customer1|article4|90 +4|1||20170430|customer3|article3|80 +4|2||20170430|customer3|article7|70 +4|3||20170430|customer3|article1|30 +4|4||20170430|customer3|article2|50 +5|1||20170510|customer4|article5|150 +5|2||20170510|customer4|article3|100 +5|3||20170510|customer4|article6|80 +6|1||20180601|customer2|article4|100 +6|2||20180601|customer2|article1|50 +6|3||20180601|customer2|article2|90 +7|1|N|20180110|customer5|article2|120 +7|2|N|20180110|customer5|article4|180 +7|3|N|20180110|customer5|article1|220 +8|1|N|20180110|customer5|article2|200 +8|2|N|20180110|customer5|article4|80 +8|3|N|20180110|customer5|article1|20 \ No newline at end of file diff --git a/src/test/resources/FailFastIntegrationTest/landing/new_data_wrong_format.psv b/src/test/resources/FailFastIntegrationTest/landing/new_data_wrong_format.psv new file mode 100644 index 0000000..f201ad8 --- /dev/null +++ b/src/test/resources/FailFastIntegrationTest/landing/new_data_wrong_format.psv @@ -0,0 +1,5 @@ +1|1||20160601|customer1|article1|150 +1|2||20160601|customer1|article2|200 +1|3||20160601|customer1|article3|50 +2|1||20170215|customer2|article4|10 +2|2||20170215|custo|mer2|article5|50 diff --git a/src/test/resources/FailFastIntegrationTest/partitioned/expected_partitions.txt b/src/test/resources/FailFastIntegrationTest/partitioned/expected_partitions.txt new file mode 100644 index 0000000..a165d68 --- /dev/null +++ b/src/test/resources/FailFastIntegrationTest/partitioned/expected_partitions.txt @@ -0,0 +1,6 @@ +year=2016/month=6/day=1 +year=2017/month=2/day=15 +year=2017/month=4/day=30 +year=2017/month=5/day=10 +year=2018/month=1/day=10 +year=2018/month=6/day=1 \ No newline at end of file diff --git a/src/test/resources/FailFastIntegrationTest/partitioned/expected_partitions_schema.json b/src/test/resources/FailFastIntegrationTest/partitioned/expected_partitions_schema.json new file mode 100644 index 0000000..0f4cce7 --- /dev/null +++ b/src/test/resources/FailFastIntegrationTest/partitioned/expected_partitions_schema.json @@ -0,0 +1,10 @@ +{ + "type" : "struct", + "fields" : [ { + "name" : "partition", + "type" : "string", + "nullable" : true, + "metadata" : { } + } + ] +} diff --git a/src/test/resources/FailFastIntegrationTest/partitioned/lake_data_post.psv b/src/test/resources/FailFastIntegrationTest/partitioned/lake_data_post.psv new file mode 100644 index 0000000..2d66c8f --- /dev/null +++ b/src/test/resources/FailFastIntegrationTest/partitioned/lake_data_post.psv @@ -0,0 +1,25 @@ +1|1||20160601|customer1|article1|150|2016|6|1 +1|2||20160601|customer1|article2|200|2016|6|1 +1|3||20160601|customer1|article3|50|2016|6|1 +2|1||20170215|customer2|article4|10|2017|2|15 +2|2||20170215|customer2|article5|50|2017|2|15 +2|3||20170215|customer2|article1|30|2017|2|15 +3|1||20170215|customer1|article6|200|2017|2|15 +3|2||20170215|customer1|article2|120|2017|2|15 +3|3||20170215|customer1|article4|90|2017|2|15 +4|1||20170430|customer3|article3|80|2017|4|30 +4|2||20170430|customer3|article7|70|2017|4|30 +4|3||20170430|customer3|article1|30|2017|4|30 +4|4||20170430|customer3|article2|50|2017|4|30 +5|1||20170510|customer4|article5|150|2017|5|10 +5|2||20170510|customer4|article3|100|2017|5|10 +5|3||20170510|customer4|article6|80|2017|5|10 +6|1||20180601|customer2|article4|100|2018|6|1 +6|2||20180601|customer2|article1|50|2018|6|1 +6|3||20180601|customer2|article2|90|2018|6|1 +7|1|N|20180110|customer5|article2|120|2018|1|10 +7|2|N|20180110|customer5|article4|180|2018|1|10 +7|3|N|20180110|customer5|article1|220|2018|1|10 +8|1|N|20180110|customer5|article2|200|2018|1|10 +8|2|N|20180110|customer5|article4|80|2018|1|10 +8|3|N|20180110|customer5|article1|20|2018|1|10 \ No newline at end of file diff --git a/src/test/resources/FailFastIntegrationTest/partitioned/params.json b/src/test/resources/FailFastIntegrationTest/partitioned/params.json new file mode 100644 index 0000000..8f09d5f --- /dev/null +++ b/src/test/resources/FailFastIntegrationTest/partitioned/params.json @@ -0,0 +1,17 @@ +{ + "backup_dir": "/tmp/tests/test_lake/test/test_table/data_backup/", + "current_dir": "/tmp/tests/test_lake/test/test_table/data/", + "delimiter": "|", + "file_format": "dsv", + "has_header": false, + "partition_column": "date", + "partition_column_format": "yyyyMMdd", + "target_partitions": [ + "year", + "month", + "day" + ], + "source_dir": "/tmp/tests/test_landing/test/test_table/data", + "target_table": "test_lake.test_table", + "reader_mode": "failfast" +} \ No newline at end of file diff --git a/src/test/resources/FailFastIntegrationTest/partitioned/target_schema.json b/src/test/resources/FailFastIntegrationTest/partitioned/target_schema.json new file mode 100644 index 0000000..a36ffa7 --- /dev/null +++ b/src/test/resources/FailFastIntegrationTest/partitioned/target_schema.json @@ -0,0 +1,65 @@ +{ + "type": "struct", + "fields": [ + { + "name": "salesorder", + "type": "integer", + "nullable": true, + "metadata": {} + }, + { + "name": "item", + "type": "integer", + "nullable": true, + "metadata": {} + }, + { + "name": "recordmode", + "type": "string", + "nullable": true, + "metadata": {} + }, + { + "name": "date", + "type": "integer", + "nullable": true, + "metadata": {} + }, + { + "name": "customer", + "type": "string", + "nullable": true, + "metadata": {} + }, + { + "name": "article", + "type": "string", + "nullable": true, + "metadata": {} + }, + { + "name": "amount", + "type": "integer", + "nullable": true, + "metadata": {} + }, + { + "name": "year", + "type": "short", + "nullable": true, + "metadata": {} + }, + { + "name": "month", + "type": "short", + "nullable": true, + "metadata": {} + }, + { + "name": "day", + "type": "short", + "nullable": true, + "metadata": {} + } + ] +} diff --git a/src/test/resources/FixedSizeStringExtractorTest/matched_schema/params.json b/src/test/resources/FixedSizeStringExtractorTest/matched_schema/params.json index cddcb2a..9174e93 100644 --- a/src/test/resources/FixedSizeStringExtractorTest/matched_schema/params.json +++ b/src/test/resources/FixedSizeStringExtractorTest/matched_schema/params.json @@ -1,7 +1,7 @@ { "source_table": "test_lake.source_table", "target_table": "test_lake.target_table", - "partition_columns": [], + "target_partitions": [], "select_conditions": [], "source_field": "value", "substring_positions": ["1,11", "13,20", "22,30", "32,37", "39,40", "42,71", "73,75", "77,79", "81,85"] diff --git a/src/test/resources/FixedSizeStringExtractorTest/matched_schema_partitioned/params.json b/src/test/resources/FixedSizeStringExtractorTest/matched_schema_partitioned/params.json index a6d90f1..2ee4fde 100644 --- a/src/test/resources/FixedSizeStringExtractorTest/matched_schema_partitioned/params.json +++ b/src/test/resources/FixedSizeStringExtractorTest/matched_schema_partitioned/params.json @@ -1,7 +1,7 @@ { "source_table": "test_lake.source_table", "target_table": "test_lake.target_table", - "partition_columns": ["year", "month"], + "target_partitions": ["year", "month"], "select_conditions": ["year=2019", "month=2"], "source_field": "value", "substring_positions": ["1,11", "13,20", "22,30", "32,37", "39,40", "42,71", "73,75", "77,79", "81,85"] diff --git a/src/test/resources/FixedSizeStringExtractorTest/non_matched_schema1/params.json b/src/test/resources/FixedSizeStringExtractorTest/non_matched_schema1/params.json index acffd7f..b650ced 100644 --- a/src/test/resources/FixedSizeStringExtractorTest/non_matched_schema1/params.json +++ b/src/test/resources/FixedSizeStringExtractorTest/non_matched_schema1/params.json @@ -1,7 +1,7 @@ { "source_table": "test_lake.source_table", "target_table": "test_lake.target_table", - "partition_columns": [], + "target_partitions": [], "select_conditions": [], "source_field": "value", "substring_positions": ["1,11", "13,20", "22,30", "32,37", "39,40", "42,71", "73,75", "77,79"] diff --git a/src/test/resources/FixedSizeStringExtractorTest/non_matched_schema2/params.json b/src/test/resources/FixedSizeStringExtractorTest/non_matched_schema2/params.json index 9c95494..1607e7e 100644 --- a/src/test/resources/FixedSizeStringExtractorTest/non_matched_schema2/params.json +++ b/src/test/resources/FixedSizeStringExtractorTest/non_matched_schema2/params.json @@ -1,7 +1,7 @@ { "source_table": "test_lake.source_table", "target_table": "test_lake.target_table", - "partition_columns": [], + "target_partitions": [], "select_conditions": [], "source_field": "value", "substring_positions": ["1,11", "13,20", "22,30", "32,37", "39,40", "42,71", "73,75", "77,79", "81,85", "87,89"] diff --git a/src/test/resources/FullLoadTest/failfast_option/lake_data_post.psv b/src/test/resources/FullLoadTest/failfast_option/lake_data_post.psv new file mode 100644 index 0000000..b93ab27 --- /dev/null +++ b/src/test/resources/FullLoadTest/failfast_option/lake_data_post.psv @@ -0,0 +1,25 @@ +1|1||20160601|customer1|article1|150 +1|2||20160601|customer1|article2|200 +1|3||20160601|customer1|article3|50 +2|1||20170215|customer2|article4|10 +2|2||20170215|customer2|article5|50 +2|3||20170215|customer2|article1|30 +3|1||20170215|customer1|article6|200 +3|2||20170215|customer1|article2|120 +3|3||20170215|customer1|article4|90 +4|1||20170430|customer3|article3|80 +4|2||20170430|customer3|article7|70 +4|3||20170430|customer3|article1|30 +4|4||20170430|customer3|article2|50 +5|1||20170510|customer4|article5|150 +5|2||20170510|customer4|article3|100 +5|3||20170510|customer4|article6|80 +6|1||20180601|customer2|article4|100 +6|2||20180601|customer2|article1|50 +6|3||20180601|customer2|article2|90 +7|1|N|20180110|customer5|article2|120 +7|2|N|20180110|customer5|article4|180 +7|3|N|20180110|customer5|article1|220 +8|1|N|20180110|customer5|article2|200 +8|2|N|20180110|customer5|article4|80 +8|3|N|20180110|customer5|article1|20 \ No newline at end of file diff --git a/src/test/resources/FullLoadTest/failfast_option/lake_data_pre.psv b/src/test/resources/FullLoadTest/failfast_option/lake_data_pre.psv new file mode 100644 index 0000000..2ad9429 --- /dev/null +++ b/src/test/resources/FullLoadTest/failfast_option/lake_data_pre.psv @@ -0,0 +1,19 @@ +1|1||20160601|customer1|article1|150 +1|2||20160601|customer1|article2|200 +1|3||20160601|customer1|article3|50 +2|1||20170215|customer2|article4|10 +2|2||20170215|customer2|article5|50 +2|3||20170215|customer2|article1|30 +3|1||20170215|customer1|article6|200 +3|2||20170215|customer1|article2|120 +3|3||20170215|customer1|article4|90 +4|1||20170430|customer3|article3|80 +4|2||20170430|customer3|article7|70 +4|3||20170430|customer3|article1|30 +4|4||20170430|customer3|article2|50 +5|1||20170510|customer4|article5|150 +5|2||20170510|customer4|article3|100 +5|3||20170510|customer4|article6|80 +6|1||20180601|customer2|article4|100 +6|2||20180601|customer2|article1|50 +6|3||20180601|customer2|article2|90 \ No newline at end of file diff --git a/src/test/resources/FullLoadTest/failfast_option/new_data_wrong.psv b/src/test/resources/FullLoadTest/failfast_option/new_data_wrong.psv new file mode 100644 index 0000000..1ae2b09 --- /dev/null +++ b/src/test/resources/FullLoadTest/failfast_option/new_data_wrong.psv @@ -0,0 +1,25 @@ +1|120160601customer1||||article1|150 +1|2|wrongdate|customer1|article2|200 +1|3||20160601|customer1|article3|50 +2|1||20170215|customer2|article4|10 +2|2||20170215|customer2|article5|50 +2|3||20170215|customer2|article1|30 +3|1||20170215|customer1|article6|200 +3|2||20170215|customer1|article2|120 +3|3||20170215|customer1|article4|90 +4|1||20170430|customer3|article3|80 +4|2||20170430|customer3|article7|70 +4|3||20170430|customer3|article1|30 +4|4||20170430|customer3|article2|50 +5|1||20170510|customer4|article5|150 +5|2||20170510|customer4|article3|100 +5|3||20170510|customer4|article6|80 +6|1||20180601|customer2|article4|100 +6|2||20180601|customer2|article1|50 +6|3||20180601|customer2|article2|90 +7|1|N|20180110|customer5|article2|120 +7|2|N|20180110|customer5|article4|180 +7|3|N|20180110|customer5|article1|220 +8|1|N|20180110|customer5|article2|200 +8|2|N|20180110|customer5|article4|80 +8|3|N|20180110|customer5|article1|20 \ No newline at end of file diff --git a/src/test/resources/FullLoadTest/failfast_option/params.json b/src/test/resources/FullLoadTest/failfast_option/params.json new file mode 100644 index 0000000..017db61 --- /dev/null +++ b/src/test/resources/FullLoadTest/failfast_option/params.json @@ -0,0 +1,13 @@ +{ + "backup_dir": "/tmp/tests/test_lake/test/test_table/data_backup/", + "current_dir": "/tmp/tests/test_lake/test/test_table/data/", + "delimiter": "|", + "file_format": "dsv", + "has_header": false, + "partition_column": "", + "partition_column_format": "", + "target_partitions": [], + "source_dir": "/tmp/tests/test_landing/test/test_table/data", + "target_table": "test_lake.test_table", + "reader_mode": "FAILFAST" +} \ No newline at end of file diff --git a/src/test/resources/FullLoadTest/failfast_option/params_dropmalformed_mode.json b/src/test/resources/FullLoadTest/failfast_option/params_dropmalformed_mode.json new file mode 100644 index 0000000..1ae3f4f --- /dev/null +++ b/src/test/resources/FullLoadTest/failfast_option/params_dropmalformed_mode.json @@ -0,0 +1,13 @@ +{ + "backup_dir": "/tmp/tests/test_lake/test/test_table/data_backup/", + "current_dir": "/tmp/tests/test_lake/test/test_table/data/", + "delimiter": "|", + "file_format": "dsv", + "has_header": false, + "partition_column": "", + "partition_column_format": "", + "target_partitions": [], + "source_dir": "/tmp/tests/test_landing/test/test_table/data", + "target_table": "test_lake.test_table", + "reader_mode": "DROPMALFORMED" +} \ No newline at end of file diff --git a/src/test/resources/FullLoadTest/failfast_option/params_invalid_reader_mode.json b/src/test/resources/FullLoadTest/failfast_option/params_invalid_reader_mode.json new file mode 100644 index 0000000..b2d3186 --- /dev/null +++ b/src/test/resources/FullLoadTest/failfast_option/params_invalid_reader_mode.json @@ -0,0 +1,13 @@ +{ + "backup_dir": "/tmp/tests/test_lake/test/test_table/data_backup/", + "current_dir": "/tmp/tests/test_lake/test/test_table/data/", + "delimiter": "|", + "file_format": "dsv", + "has_header": false, + "partition_column": "", + "partition_column_format": "", + "target_partitions": [], + "source_dir": "/tmp/tests/test_landing/test/test_table/data", + "target_table": "test_lake.test_table", + "reader_mode": "invalid_mode" +} \ No newline at end of file diff --git a/src/test/resources/FullLoadTest/failfast_option/params_no_reader_mode_set.json b/src/test/resources/FullLoadTest/failfast_option/params_no_reader_mode_set.json new file mode 100644 index 0000000..d80538d --- /dev/null +++ b/src/test/resources/FullLoadTest/failfast_option/params_no_reader_mode_set.json @@ -0,0 +1,12 @@ +{ + "backup_dir": "/tmp/tests/test_lake/test/test_table/data_backup/", + "current_dir": "/tmp/tests/test_lake/test/test_table/data/", + "delimiter": "|", + "file_format": "dsv", + "has_header": false, + "partition_column": "", + "partition_column_format": "", + "target_partitions": [], + "source_dir": "/tmp/tests/test_landing/test/test_table/data", + "target_table": "test_lake.test_table" +} \ No newline at end of file diff --git a/src/test/resources/FullLoadTest/failfast_option/params_permissive_mode.json b/src/test/resources/FullLoadTest/failfast_option/params_permissive_mode.json new file mode 100644 index 0000000..48d64c5 --- /dev/null +++ b/src/test/resources/FullLoadTest/failfast_option/params_permissive_mode.json @@ -0,0 +1,13 @@ +{ + "backup_dir": "/tmp/tests/test_lake/test/test_table/data_backup/", + "current_dir": "/tmp/tests/test_lake/test/test_table/data/", + "delimiter": "|", + "file_format": "dsv", + "has_header": false, + "partition_column": "", + "partition_column_format": "", + "target_partitions": [], + "source_dir": "/tmp/tests/test_landing/test/test_table/data", + "target_table": "test_lake.test_table", + "reader_mode": "PERMISSIVE" +} \ No newline at end of file diff --git a/src/test/resources/FullLoadTest/failfast_option/target_schema.json b/src/test/resources/FullLoadTest/failfast_option/target_schema.json new file mode 100644 index 0000000..944e5df --- /dev/null +++ b/src/test/resources/FullLoadTest/failfast_option/target_schema.json @@ -0,0 +1,40 @@ +{ + "type" : "struct", + "fields" : [ { + "name" : "salesorder", + "type" : "integer", + "nullable" : true, + "metadata" : { } + }, { + "name" : "item", + "type" : "integer", + "nullable" : true, + "metadata" : { } + }, { + "name" : "recordmode", + "type" : "string", + "nullable" : true, + "metadata" : { } + }, { + "name" : "date", + "type" : "string", + "nullable" : true, + "metadata" : { } + }, { + "name" : "customer", + "type" : "string", + "nullable" : true, + "metadata" : { } + }, { + "name" : "article", + "type" : "string", + "nullable" : true, + "metadata" : { } + }, { + "name" : "amount", + "type" : "integer", + "nullable" : true, + "metadata" : { } + } + ] +} diff --git a/src/test/resources/FullLoadTest/landing/new_data_weekly.psv b/src/test/resources/FullLoadTest/landing/new_data_weekly.psv new file mode 100644 index 0000000..e88b55f --- /dev/null +++ b/src/test/resources/FullLoadTest/landing/new_data_weekly.psv @@ -0,0 +1,25 @@ +1|1||201606|customer1|article1|150 +1|2||201606|customer1|article2|200 +1|3||201606|customer1|article3|50 +2|1||201702|customer2|article4|10 +2|2||201702|customer2|article5|50 +2|3||201702|customer2|article1|30 +3|1||201702|customer1|article6|200 +3|2||201702|customer1|article2|120 +3|3||201702|customer1|article4|90 +4|1||201704|customer3|article3|80 +4|2||201704|customer3|article7|70 +4|3||201704|customer3|article1|30 +4|4||201704|customer3|article2|50 +5|1||201705|customer4|article5|150 +5|2||201705|customer4|article3|100 +5|3||201705|customer4|article6|80 +6|1||201806|customer2|article4|100 +6|2||201806|customer2|article1|50 +6|3||201806|customer2|article2|90 +7|1|N|201801|customer5|article2|120 +7|2|N|201801|customer5|article4|180 +7|3|N|201801|customer5|article1|220 +8|1|N|201801|customer5|article2|200 +8|2|N|201801|customer5|article4|80 +8|3|N|201801|customer5|article1|20 \ No newline at end of file diff --git a/src/test/resources/FullLoadTest/non_partitioned/params.json b/src/test/resources/FullLoadTest/non_partitioned/params.json index 3c8f42a..d80538d 100644 --- a/src/test/resources/FullLoadTest/non_partitioned/params.json +++ b/src/test/resources/FullLoadTest/non_partitioned/params.json @@ -6,7 +6,7 @@ "has_header": false, "partition_column": "", "partition_column_format": "", - "partition_columns": [], + "target_partitions": [], "source_dir": "/tmp/tests/test_landing/test/test_table/data", "target_table": "test_lake.test_table" } \ No newline at end of file diff --git a/src/test/resources/FullLoadTest/partitioned/params.json b/src/test/resources/FullLoadTest/partitioned/params.json index 7b4d454..20d6efb 100644 --- a/src/test/resources/FullLoadTest/partitioned/params.json +++ b/src/test/resources/FullLoadTest/partitioned/params.json @@ -6,7 +6,7 @@ "has_header": false, "partition_column": "date", "partition_column_format": "yyyyMMdd", - "partition_columns": [ + "target_partitions": [ "year", "month", "day" diff --git a/src/test/resources/FullLoadTest/partitioned_date_format_wrong/params.json b/src/test/resources/FullLoadTest/partitioned_date_format_wrong/params.json index a6f8153..8bd62b2 100644 --- a/src/test/resources/FullLoadTest/partitioned_date_format_wrong/params.json +++ b/src/test/resources/FullLoadTest/partitioned_date_format_wrong/params.json @@ -6,7 +6,7 @@ "has_header": false, "partition_column": "date", "partition_column_format": "ddMMyy", - "partition_columns": [ + "target_partitions": [ "year", "month", "day" diff --git a/src/test/resources/FullLoadTest/partitioned_not_exist_dir/params.json b/src/test/resources/FullLoadTest/partitioned_not_exist_dir/params.json index c363b3c..a154fe2 100644 --- a/src/test/resources/FullLoadTest/partitioned_not_exist_dir/params.json +++ b/src/test/resources/FullLoadTest/partitioned_not_exist_dir/params.json @@ -6,7 +6,7 @@ "has_header": false, "partition_column": "date", "partition_column_format": "yyyyMMdd", - "partition_columns": [ + "target_partitions": [ "year", "month", "day" diff --git a/src/test/resources/FullLoadTest/partitioned_partition_column_wrong/params.json b/src/test/resources/FullLoadTest/partitioned_partition_column_wrong/params.json index e2a9f11..3242559 100644 --- a/src/test/resources/FullLoadTest/partitioned_partition_column_wrong/params.json +++ b/src/test/resources/FullLoadTest/partitioned_partition_column_wrong/params.json @@ -6,7 +6,7 @@ "has_header": false, "partition_column": "date_missing_column", "partition_column_format": "yyyyMMdd", - "partition_columns": [ + "target_partitions": [ "year", "month", "day" diff --git a/src/test/resources/FullLoadTest/partitioned_weekly/lake_data_post.psv b/src/test/resources/FullLoadTest/partitioned_weekly/lake_data_post.psv new file mode 100644 index 0000000..78ebcb6 --- /dev/null +++ b/src/test/resources/FullLoadTest/partitioned_weekly/lake_data_post.psv @@ -0,0 +1,25 @@ +1|1||201606|customer1|article1|150|2016|06 +1|2||201606|customer1|article2|200|2016|06 +1|3||201606|customer1|article3|50|2016|06 +2|1||201702|customer2|article4|10|2017|02 +2|2||201702|customer2|article5|50|2017|02 +2|3||201702|customer2|article1|30|2017|02 +3|1||201702|customer1|article6|200|2017|02 +3|2||201702|customer1|article2|120|2017|02 +3|3||201702|customer1|article4|90|2017|02 +4|1||201704|customer3|article3|80|2017|04 +4|2||201704|customer3|article7|70|2017|04 +4|3||201704|customer3|article1|30|2017|04 +4|4||201704|customer3|article2|50|2017|04 +5|1||201705|customer4|article5|150|2017|05 +5|2||201705|customer4|article3|100|2017|05 +5|3||201705|customer4|article6|80|2017|05 +6|1||201806|customer2|article4|100|2018|06 +6|2||201806|customer2|article1|50|2018|06 +6|3||201806|customer2|article2|90|2018|06 +7|1|N|201801|customer5|article2|120|2018|01 +7|2|N|201801|customer5|article4|180|2018|01 +7|3|N|201801|customer5|article1|220|2018|01 +8|1|N|201801|customer5|article2|200|2018|01 +8|2|N|201801|customer5|article4|80|2018|01 +8|3|N|201801|customer5|article1|20|2018|01 \ No newline at end of file diff --git a/src/test/resources/FullLoadTest/partitioned_weekly/lake_data_pre.psv b/src/test/resources/FullLoadTest/partitioned_weekly/lake_data_pre.psv new file mode 100644 index 0000000..cff62a6 --- /dev/null +++ b/src/test/resources/FullLoadTest/partitioned_weekly/lake_data_pre.psv @@ -0,0 +1,19 @@ +1|1||201606|customer1|article1|150|2016|06 +1|2||201606|customer1|article2|200|2016|06 +1|3||201606|customer1|article3|50|2016|06 +2|1||201702|customer2|article4|10|2017|02 +2|2||201702|customer2|article5|50|2017|02 +2|3||201702|customer2|article1|30|2017|02 +3|1||201702|customer1|article6|200|2017|02 +3|2||201702|customer1|article2|120|2017|02 +3|3||201702|customer1|article4|90|2017|02 +4|1||201704|customer3|article3|80|2017|04 +4|2||201704|customer3|article7|70|2017|04 +4|3||201704|customer3|article1|30|2017|04 +4|4||201704|customer3|article2|50|2017|04 +5|1||201705|customer4|article5|150|2017|05 +5|2||201705|customer4|article3|100|2017|05 +5|3||201705|customer4|article6|80|2017|05 +6|1||201806|customer2|article4|100|2018|06 +6|2||201806|customer2|article1|50|2018|06 +6|3||201806|customer2|article2|90|2018|06 \ No newline at end of file diff --git a/src/test/resources/FullLoadTest/partitioned_weekly/params.json b/src/test/resources/FullLoadTest/partitioned_weekly/params.json new file mode 100644 index 0000000..6837382 --- /dev/null +++ b/src/test/resources/FullLoadTest/partitioned_weekly/params.json @@ -0,0 +1,15 @@ +{ + "backup_dir": "/tmp/tests/test_lake/test/test_table/data_backup/", + "current_dir": "/tmp/tests/test_lake/test/test_table/data/", + "delimiter": "|", + "file_format": "dsv", + "has_header": false, + "partition_column": "date", + "partition_column_format": "yyyyww", + "target_partitions": [ + "year", + "week" + ], + "source_dir": "/tmp/tests/test_landing/test/test_table/data", + "target_table": "test_lake.test_table" +} \ No newline at end of file diff --git a/src/test/resources/FullLoadTest/partitioned_weekly/target_schema.json b/src/test/resources/FullLoadTest/partitioned_weekly/target_schema.json new file mode 100644 index 0000000..aa8036e --- /dev/null +++ b/src/test/resources/FullLoadTest/partitioned_weekly/target_schema.json @@ -0,0 +1,50 @@ +{ + "type" : "struct", + "fields" : [ { + "name" : "salesorder", + "type" : "integer", + "nullable" : true, + "metadata" : { } + }, { + "name" : "item", + "type" : "integer", + "nullable" : true, + "metadata" : { } + }, { + "name" : "recordmode", + "type" : "string", + "nullable" : true, + "metadata" : { } + }, { + "name" : "date", + "type" : "integer", + "nullable" : true, + "metadata" : { } + }, { + "name" : "customer", + "type" : "string", + "nullable" : true, + "metadata" : { } + }, { + "name" : "article", + "type" : "string", + "nullable" : true, + "metadata" : { } + }, { + "name" : "amount", + "type" : "integer", + "nullable" : true, + "metadata" : { } + }, { + "name" : "year", + "type" : "short", + "nullable" : true, + "metadata" : { } + }, { + "name" : "week", + "type" : "short", + "nullable" : true, + "metadata" : { } + } + ] +} diff --git a/src/test/resources/NestedFlattenerTest/expected_target_data.psv b/src/test/resources/NestedFlattenerTest/expected_target_data.psv new file mode 100644 index 0000000..7d42116 --- /dev/null +++ b/src/test/resources/NestedFlattenerTest/expected_target_data.psv @@ -0,0 +1,4 @@ +1111111111111111111|production|1564531201371|events|iPhone11,6|us|application_state_transition||3445194752 +111111111111111111|production|1564531200301|events|iPhone11,8|us|application_state_transition||2404974592 +-1111111111111111|production|1564531202763|events|iPhone10,1|us|screen_view|Feed - View|1703641088 +-1111111111111111|production|1564531202763|events|iPhone10,1|us|application_state_transition||1704853504 \ No newline at end of file diff --git a/src/test/resources/NestedFlattenerTest/nest_test/data/part-00000-3924d987-4115-47e0-8460-578de41057cc-c000.snappy.parquet b/src/test/resources/NestedFlattenerTest/nest_test/data/part-00000-3924d987-4115-47e0-8460-578de41057cc-c000.snappy.parquet new file mode 100644 index 0000000..fc57ad4 Binary files /dev/null and b/src/test/resources/NestedFlattenerTest/nest_test/data/part-00000-3924d987-4115-47e0-8460-578de41057cc-c000.snappy.parquet differ diff --git a/src/test/resources/NestedFlattenerTest/scenario1/params.json b/src/test/resources/NestedFlattenerTest/scenario1/params.json new file mode 100644 index 0000000..900289e --- /dev/null +++ b/src/test/resources/NestedFlattenerTest/scenario1/params.json @@ -0,0 +1,24 @@ +{ + "source_location": "/tmp/tests/test_lake/nest/nest_test/data", + "target_table": "test_lake.nest_flattened", + "chars_to_replace": "[.:#]+", + "replacement_char": "_", + "fields_to_flatten": [ + "user_attributes", + "device_info", + "events", + "events__data", + "events__data__device_current_state" + ], + "column_mapping": { + "batch_id": "batch_id", + "environment": "environment", + "timestamp_unixtime_ms": "event_timestamp", + "message_type": "message_type", + "device_info__brand": "device_brand", + "device_info__network_country": "network_country", + "events__event_type": "event_type", + "events__data__screen_name": "screen_name", + "events__data__device_current_state__total_system_memory_usage_bytes": "memory_usage_bytes" + } +} \ No newline at end of file diff --git a/src/test/resources/NestedFlattenerTest/scenario2/params.json b/src/test/resources/NestedFlattenerTest/scenario2/params.json new file mode 100644 index 0000000..51f9b5a --- /dev/null +++ b/src/test/resources/NestedFlattenerTest/scenario2/params.json @@ -0,0 +1,25 @@ +{ + "source_location": "/tmp/tests/test_lake/nest/nest_test/data", + "target_table": "test_lake.nest_flattened", + "target_partitions": ["device_brand"], + "chars_to_replace": "[.:#]+", + "replacement_char": "_", + "fields_to_flatten": [ + "user_attributes", + "device_info", + "events", + "events__data", + "events__data__device_current_state" + ], + "column_mapping": { + "batch_id": "batch_id", + "environment": "environment", + "timestamp_unixtime_ms": "event_timestamp", + "message_type": "message_type", + "device_info__brand": "device_brand", + "device_info__network_country": "network_country", + "events__event_type": "event_type", + "events__data__device_current_state__total_system_memory_usage_bytes": "memory_usage_bytes", + "events__data__screen_name": "screen_name" + } +} \ No newline at end of file diff --git a/src/test/resources/NestedFlattenerTest/target_schema.json b/src/test/resources/NestedFlattenerTest/target_schema.json new file mode 100644 index 0000000..8149390 --- /dev/null +++ b/src/test/resources/NestedFlattenerTest/target_schema.json @@ -0,0 +1,59 @@ +{ + "type": "struct", + "fields": [ + { + "name": "batch_id", + "type": "long", + "nullable": true, + "metadata": {} + }, + { + "name": "environment", + "type": "string", + "nullable": true, + "metadata": {} + }, + { + "name": "event_timestamp", + "type": "long", + "nullable": true, + "metadata": {} + }, + { + "name": "message_type", + "type": "string", + "nullable": true, + "metadata": {} + }, + { + "name": "device_brand", + "type": "string", + "nullable": true, + "metadata": {} + }, + { + "name": "network_country", + "type": "string", + "nullable": true, + "metadata": {} + }, + { + "name": "event_type", + "type": "string", + "nullable": true, + "metadata": {} + }, + { + "name": "screen_name", + "type": "string", + "nullable": true, + "metadata": {} + }, + { + "name": "memory_usage_bytes", + "type": "long", + "nullable": true, + "metadata": {} + } + ] +} \ No newline at end of file diff --git a/src/test/resources/SemiStructuredLoadTest/semistructured_json_load/params.json b/src/test/resources/SemiStructuredLoadTest/semistructured_json_load/params.json index 75e0703..44d8c7e 100644 --- a/src/test/resources/SemiStructuredLoadTest/semistructured_json_load/params.json +++ b/src/test/resources/SemiStructuredLoadTest/semistructured_json_load/params.json @@ -10,7 +10,7 @@ "(?<=year=[0-9]{4})/month=([0-9]{2})/(?=day=[0-9]{2})", "(?<=year=[0-9]{4}/month=[0-9]{2})/day=([0-9]{2})" ], - "partition_columns": [ + "target_partitions": [ "year", "month", "day" diff --git a/src/test/resources/SemiStructuredLoadTest/semistructured_json_load_dropping_column/params.json b/src/test/resources/SemiStructuredLoadTest/semistructured_json_load_dropping_column/params.json index fd42e88..7e18564 100644 --- a/src/test/resources/SemiStructuredLoadTest/semistructured_json_load_dropping_column/params.json +++ b/src/test/resources/SemiStructuredLoadTest/semistructured_json_load_dropping_column/params.json @@ -10,7 +10,7 @@ "(?<=year=[0-9]{4})/month=([0-9]{2})/(?=day=[0-9]{2})", "(?<=year=[0-9]{4}/month=[0-9]{2})/day=([0-9]{2})" ], - "partition_columns": [ + "target_partitions": [ "year", "month", "day" diff --git a/src/test/resources/SemiStructuredLoadTest/semistructured_json_load_dropping_column/params_column_dropped.json b/src/test/resources/SemiStructuredLoadTest/semistructured_json_load_dropping_column/params_column_dropped.json index 893624f..1879179 100644 --- a/src/test/resources/SemiStructuredLoadTest/semistructured_json_load_dropping_column/params_column_dropped.json +++ b/src/test/resources/SemiStructuredLoadTest/semistructured_json_load_dropping_column/params_column_dropped.json @@ -10,7 +10,7 @@ "(?<=year=[0-9]{4})/month=([0-9]{2})/(?=day=[0-9]{2})", "(?<=year=[0-9]{4}/month=[0-9]{2})/day=([0-9]{2})" ], - "partition_columns": [ + "target_partitions": [ "year", "month", "day" diff --git a/src/test/resources/SemiStructuredLoadTest/semistructured_json_load_evolving_schema/params.json b/src/test/resources/SemiStructuredLoadTest/semistructured_json_load_evolving_schema/params.json index 75e0703..44d8c7e 100644 --- a/src/test/resources/SemiStructuredLoadTest/semistructured_json_load_evolving_schema/params.json +++ b/src/test/resources/SemiStructuredLoadTest/semistructured_json_load_evolving_schema/params.json @@ -10,7 +10,7 @@ "(?<=year=[0-9]{4})/month=([0-9]{2})/(?=day=[0-9]{2})", "(?<=year=[0-9]{4}/month=[0-9]{2})/day=([0-9]{2})" ], - "partition_columns": [ + "target_partitions": [ "year", "month", "day" diff --git a/src/test/resources/SemiStructuredLoadTest/semistructured_json_load_evolving_schema/params_evolved.json b/src/test/resources/SemiStructuredLoadTest/semistructured_json_load_evolving_schema/params_evolved.json index a352e7f..fb89123 100644 --- a/src/test/resources/SemiStructuredLoadTest/semistructured_json_load_evolving_schema/params_evolved.json +++ b/src/test/resources/SemiStructuredLoadTest/semistructured_json_load_evolving_schema/params_evolved.json @@ -10,7 +10,7 @@ "(?<=year=[0-9]{4})/month=([0-9]{2})/(?=day=[0-9]{2})", "(?<=year=[0-9]{4}/month=[0-9]{2})/day=([0-9]{2})" ], - "partition_columns": [ + "target_partitions": [ "year", "month", "day" diff --git a/src/test/resources/SemiStructuredLoadTest/semistructured_json_load_mismatching_schema/params.json b/src/test/resources/SemiStructuredLoadTest/semistructured_json_load_mismatching_schema/params.json index 893624f..1879179 100644 --- a/src/test/resources/SemiStructuredLoadTest/semistructured_json_load_mismatching_schema/params.json +++ b/src/test/resources/SemiStructuredLoadTest/semistructured_json_load_mismatching_schema/params.json @@ -10,7 +10,7 @@ "(?<=year=[0-9]{4})/month=([0-9]{2})/(?=day=[0-9]{2})", "(?<=year=[0-9]{4}/month=[0-9]{2})/day=([0-9]{2})" ], - "partition_columns": [ + "target_partitions": [ "year", "month", "day" diff --git a/src/test/resources/SemiStructuredLoadTest/semistructured_json_load_wrong_configuration/params.json b/src/test/resources/SemiStructuredLoadTest/semistructured_json_load_wrong_configuration/params.json index f2baa21..6f0bf16 100644 --- a/src/test/resources/SemiStructuredLoadTest/semistructured_json_load_wrong_configuration/params.json +++ b/src/test/resources/SemiStructuredLoadTest/semistructured_json_load_wrong_configuration/params.json @@ -9,7 +9,7 @@ "(?<=year=[0-9]{4})/month=([0-9]{2})/(?=day=[0-9]{2})", "(?<=year=[0-9]{4}/month=[0-9]{2})/day=([0-9]{2})" ], - "partition_columns": [ + "target_partitions": [ "year", "month", "day" diff --git a/src/test/resources/SemiStructuredLoadTest/semistructured_load_with_existing_header/params.json b/src/test/resources/SemiStructuredLoadTest/semistructured_load_with_existing_header/params.json index 75e0703..44d8c7e 100644 --- a/src/test/resources/SemiStructuredLoadTest/semistructured_load_with_existing_header/params.json +++ b/src/test/resources/SemiStructuredLoadTest/semistructured_load_with_existing_header/params.json @@ -10,7 +10,7 @@ "(?<=year=[0-9]{4})/month=([0-9]{2})/(?=day=[0-9]{2})", "(?<=year=[0-9]{4}/month=[0-9]{2})/day=([0-9]{2})" ], - "partition_columns": [ + "target_partitions": [ "year", "month", "day" diff --git a/src/test/resources/SemiStructuredLoadTest/semistructured_nested_json_load/params.json b/src/test/resources/SemiStructuredLoadTest/semistructured_nested_json_load/params.json index 93df79d..dbc4c4f 100644 --- a/src/test/resources/SemiStructuredLoadTest/semistructured_nested_json_load/params.json +++ b/src/test/resources/SemiStructuredLoadTest/semistructured_nested_json_load/params.json @@ -10,7 +10,7 @@ "(?<=year=[0-9]{4})/month=([0-9]{2})/(?=day=[0-9]{2})", "(?<=year=[0-9]{4}/month=[0-9]{2})/day=([0-9]{2})" ], - "partition_columns": [ + "target_partitions": [ "year", "month", "day" diff --git a/src/test/resources/SemiStructuredLoadTest/semistructured_parquet_test/params.json b/src/test/resources/SemiStructuredLoadTest/semistructured_parquet_test/params.json index 693fa18..a032a13 100644 --- a/src/test/resources/SemiStructuredLoadTest/semistructured_parquet_test/params.json +++ b/src/test/resources/SemiStructuredLoadTest/semistructured_parquet_test/params.json @@ -10,7 +10,7 @@ "(?<=year=[0-9]{4})/month=([0-9]{2})/(?=day=[0-9]{2})", "(?<=year=[0-9]{4}/month=[0-9]{2})/day=([0-9]{2})" ], - "partition_columns": [ + "target_partitions": [ "year", "month", "day" diff --git a/src/test/resources/SparkRecoverPartitionsCustomIntegrationTest/multiple_source_files/data_20180101-part-00000.psv b/src/test/resources/SparkRecoverPartitionsCustomIntegrationTest/multiple_source_files/data_20180101-part-00000.psv new file mode 100644 index 0000000..7865741 --- /dev/null +++ b/src/test/resources/SparkRecoverPartitionsCustomIntegrationTest/multiple_source_files/data_20180101-part-00000.psv @@ -0,0 +1,4 @@ +7|1|N|20180110|customer5|article2|120 +7|2|N|20180110|customer5|article4|180 +7|3|N|20180110|customer5|article1|220 +7|3|N|20180110|customer5|BAD RECORD \ No newline at end of file diff --git a/src/test/resources/SparkRecoverPartitionsCustomIntegrationTest/multiple_source_files/data_20180101-part-00001.psv b/src/test/resources/SparkRecoverPartitionsCustomIntegrationTest/multiple_source_files/data_20180101-part-00001.psv new file mode 100644 index 0000000..a6ab327 --- /dev/null +++ b/src/test/resources/SparkRecoverPartitionsCustomIntegrationTest/multiple_source_files/data_20180101-part-00001.psv @@ -0,0 +1,3 @@ +8|1|N|20180110|customer5|article2|200 +8|2|N|20180110|customer5|article4|80 +8|3|N|20180110|customer5|article1|20 \ No newline at end of file diff --git a/src/test/resources/SparkRecoverPartitionsCustomIntegrationTest/multiple_source_files/expected_partitions.txt b/src/test/resources/SparkRecoverPartitionsCustomIntegrationTest/multiple_source_files/expected_partitions.txt new file mode 100644 index 0000000..79e3cee --- /dev/null +++ b/src/test/resources/SparkRecoverPartitionsCustomIntegrationTest/multiple_source_files/expected_partitions.txt @@ -0,0 +1,6 @@ +year=2016/month=6/day=1 +year=2017/month=2/day=15 +year=2017/month=4/day=30 +year=2017/month=5/day=10 +year=2018/month=1/day=1 +year=2018/month=6/day=1 \ No newline at end of file diff --git a/src/test/resources/SparkRecoverPartitionsCustomIntegrationTest/multiple_source_files/expected_partitions_schema.json b/src/test/resources/SparkRecoverPartitionsCustomIntegrationTest/multiple_source_files/expected_partitions_schema.json new file mode 100644 index 0000000..0f4cce7 --- /dev/null +++ b/src/test/resources/SparkRecoverPartitionsCustomIntegrationTest/multiple_source_files/expected_partitions_schema.json @@ -0,0 +1,10 @@ +{ + "type" : "struct", + "fields" : [ { + "name" : "partition", + "type" : "string", + "nullable" : true, + "metadata" : { } + } + ] +} diff --git a/src/test/resources/SparkRecoverPartitionsCustomIntegrationTest/multiple_source_files/lake_data_post.psv b/src/test/resources/SparkRecoverPartitionsCustomIntegrationTest/multiple_source_files/lake_data_post.psv new file mode 100644 index 0000000..ff625ff --- /dev/null +++ b/src/test/resources/SparkRecoverPartitionsCustomIntegrationTest/multiple_source_files/lake_data_post.psv @@ -0,0 +1,25 @@ +1|1||20160601|customer1|article1|150|2016|6|1 +1|2||20160601|customer1|article2|200|2016|6|1 +1|3||20160601|customer1|article3|50|2016|6|1 +2|1||20170215|customer2|article4|10|2017|2|15 +2|2||20170215|customer2|article5|50|2017|2|15 +2|3||20170215|customer2|article1|30|2017|2|15 +3|1||20170215|customer1|article6|200|2017|2|15 +3|2||20170215|customer1|article2|120|2017|2|15 +3|3||20170215|customer1|article4|90|2017|2|15 +4|1||20170430|customer3|article3|80|2017|4|30 +4|2||20170430|customer3|article7|70|2017|4|30 +4|3||20170430|customer3|article1|30|2017|4|30 +4|4||20170430|customer3|article2|50|2017|4|30 +5|1||20170510|customer4|article5|150|2017|5|10 +5|2||20170510|customer4|article3|100|2017|5|10 +5|3||20170510|customer4|article6|80|2017|5|10 +6|1||20180601|customer2|article4|100|2018|6|1 +6|2||20180601|customer2|article1|50|2018|6|1 +6|3||20180601|customer2|article2|90|2018|6|1 +7|1|N|20180110|customer5|article2|120|2018|1|1 +7|2|N|20180110|customer5|article4|180|2018|1|1 +7|3|N|20180110|customer5|article1|220|2018|1|1 +8|1|N|20180110|customer5|article2|200|2018|1|1 +8|2|N|20180110|customer5|article4|80|2018|1|1 +8|3|N|20180110|customer5|article1|20|2018|1|1 \ No newline at end of file diff --git a/src/test/resources/SparkRecoverPartitionsCustomIntegrationTest/multiple_source_files/lake_data_pre.psv b/src/test/resources/SparkRecoverPartitionsCustomIntegrationTest/multiple_source_files/lake_data_pre.psv new file mode 100644 index 0000000..b1bca8d --- /dev/null +++ b/src/test/resources/SparkRecoverPartitionsCustomIntegrationTest/multiple_source_files/lake_data_pre.psv @@ -0,0 +1,19 @@ +1|1||20160601|customer1|article1|150|2016|6|1 +1|2||20160601|customer1|article2|200|2016|6|1 +1|3||20160601|customer1|article3|50|2016|6|1 +2|1||20170215|customer2|article4|10|2017|2|15 +2|2||20170215|customer2|article5|50|2017|2|15 +2|3||20170215|customer2|article1|30|2017|2|15 +3|1||20170215|customer1|article6|200|2017|2|15 +3|2||20170215|customer1|article2|120|2017|2|15 +3|3||20170215|customer1|article4|90|2017|2|15 +4|1||20170430|customer3|article3|80|2017|4|30 +4|2||20170430|customer3|article7|70|2017|4|30 +4|3||20170430|customer3|article1|30|2017|4|30 +4|4||20170430|customer3|article2|50|2017|4|30 +5|1||20170510|customer4|article5|150|2017|5|10 +5|2||20170510|customer4|article3|100|2017|5|10 +5|3||20170510|customer4|article6|80|2017|5|10 +6|1||20180601|customer2|article4|100|2018|6|1 +6|2||20180601|customer2|article1|50|2018|6|1 +6|3||20180601|customer2|article2|90|2018|6|1 \ No newline at end of file diff --git a/src/test/resources/SparkRecoverPartitionsCustomIntegrationTest/multiple_source_files/params.json b/src/test/resources/SparkRecoverPartitionsCustomIntegrationTest/multiple_source_files/params.json new file mode 100644 index 0000000..9a5a71d --- /dev/null +++ b/src/test/resources/SparkRecoverPartitionsCustomIntegrationTest/multiple_source_files/params.json @@ -0,0 +1,19 @@ +{ + "target_table": "test_lake.test_table", + "source_dir": "/tmp/tests/test_landing/test_table/data", + "header_dir": "/tmp/tests/test_landing/test_table/header", + "file_format": "dsv", + "delimiter": "|", + "metadata_update_strategy": "SparkRecoverPartitionsCustom", + "has_header": false, + "regex_filename": [ + "([0-9]{4})(?=[0-9]{2})(?=[0-9]{2})", + "(?<=[0-9]{4})([0-9]{2})(?=[0-9]{2})", + "(?<=[0-9]{6})([0-9]{2})" + ], + "target_partitions": [ + "year", + "month", + "day" + ] +} \ No newline at end of file diff --git a/src/test/resources/SparkRecoverPartitionsCustomIntegrationTest/multiple_source_files/target_schema.json b/src/test/resources/SparkRecoverPartitionsCustomIntegrationTest/multiple_source_files/target_schema.json new file mode 100644 index 0000000..bdd9036 --- /dev/null +++ b/src/test/resources/SparkRecoverPartitionsCustomIntegrationTest/multiple_source_files/target_schema.json @@ -0,0 +1,54 @@ +{ + "type" : "struct", + "fields" : [ { + "name" : "salesorder", + "type" : "integer", + "nullable" : true, + "metadata" : { } + }, { + "name" : "item", + "type" : "integer", + "nullable" : true, + "metadata" : { } + }, { + "name" : "recordmode", + "type" : "string", + "nullable" : true, + "metadata" : { } + }, { + "name" : "date", + "type" : "integer", + "nullable" : true, + "metadata" : { } + }, { + "name" : "customer", + "type" : "string", + "nullable" : true, + "metadata" : { } + }, { + "name" : "article", + "type" : "string", + "nullable" : true, + "metadata" : { } + }, { + "name" : "amount", + "type" : "integer", + "nullable" : true, + "metadata" : { } + }, { + "name" : "year", + "type" : "short", + "nullable" : true, + "metadata" : { } + }, { + "name" : "month", + "type" : "short", + "nullable" : true, + "metadata" : { } + }, { + "name" : "day", + "type" : "short", + "nullable" : true, + "metadata" : { } + } ] +} diff --git a/src/test/resources/SparkRecoverPartitionsNativeIntegrationTest/multiple_source_files/data_20180101-part-00000.psv b/src/test/resources/SparkRecoverPartitionsNativeIntegrationTest/multiple_source_files/data_20180101-part-00000.psv new file mode 100644 index 0000000..7865741 --- /dev/null +++ b/src/test/resources/SparkRecoverPartitionsNativeIntegrationTest/multiple_source_files/data_20180101-part-00000.psv @@ -0,0 +1,4 @@ +7|1|N|20180110|customer5|article2|120 +7|2|N|20180110|customer5|article4|180 +7|3|N|20180110|customer5|article1|220 +7|3|N|20180110|customer5|BAD RECORD \ No newline at end of file diff --git a/src/test/resources/SparkRecoverPartitionsNativeIntegrationTest/multiple_source_files/data_20180101-part-00001.psv b/src/test/resources/SparkRecoverPartitionsNativeIntegrationTest/multiple_source_files/data_20180101-part-00001.psv new file mode 100644 index 0000000..a6ab327 --- /dev/null +++ b/src/test/resources/SparkRecoverPartitionsNativeIntegrationTest/multiple_source_files/data_20180101-part-00001.psv @@ -0,0 +1,3 @@ +8|1|N|20180110|customer5|article2|200 +8|2|N|20180110|customer5|article4|80 +8|3|N|20180110|customer5|article1|20 \ No newline at end of file diff --git a/src/test/resources/SparkRecoverPartitionsNativeIntegrationTest/multiple_source_files/expected_partitions.txt b/src/test/resources/SparkRecoverPartitionsNativeIntegrationTest/multiple_source_files/expected_partitions.txt new file mode 100644 index 0000000..79e3cee --- /dev/null +++ b/src/test/resources/SparkRecoverPartitionsNativeIntegrationTest/multiple_source_files/expected_partitions.txt @@ -0,0 +1,6 @@ +year=2016/month=6/day=1 +year=2017/month=2/day=15 +year=2017/month=4/day=30 +year=2017/month=5/day=10 +year=2018/month=1/day=1 +year=2018/month=6/day=1 \ No newline at end of file diff --git a/src/test/resources/SparkRecoverPartitionsNativeIntegrationTest/multiple_source_files/expected_partitions_schema.json b/src/test/resources/SparkRecoverPartitionsNativeIntegrationTest/multiple_source_files/expected_partitions_schema.json new file mode 100644 index 0000000..0f4cce7 --- /dev/null +++ b/src/test/resources/SparkRecoverPartitionsNativeIntegrationTest/multiple_source_files/expected_partitions_schema.json @@ -0,0 +1,10 @@ +{ + "type" : "struct", + "fields" : [ { + "name" : "partition", + "type" : "string", + "nullable" : true, + "metadata" : { } + } + ] +} diff --git a/src/test/resources/SparkRecoverPartitionsNativeIntegrationTest/multiple_source_files/lake_data_post.psv b/src/test/resources/SparkRecoverPartitionsNativeIntegrationTest/multiple_source_files/lake_data_post.psv new file mode 100644 index 0000000..ff625ff --- /dev/null +++ b/src/test/resources/SparkRecoverPartitionsNativeIntegrationTest/multiple_source_files/lake_data_post.psv @@ -0,0 +1,25 @@ +1|1||20160601|customer1|article1|150|2016|6|1 +1|2||20160601|customer1|article2|200|2016|6|1 +1|3||20160601|customer1|article3|50|2016|6|1 +2|1||20170215|customer2|article4|10|2017|2|15 +2|2||20170215|customer2|article5|50|2017|2|15 +2|3||20170215|customer2|article1|30|2017|2|15 +3|1||20170215|customer1|article6|200|2017|2|15 +3|2||20170215|customer1|article2|120|2017|2|15 +3|3||20170215|customer1|article4|90|2017|2|15 +4|1||20170430|customer3|article3|80|2017|4|30 +4|2||20170430|customer3|article7|70|2017|4|30 +4|3||20170430|customer3|article1|30|2017|4|30 +4|4||20170430|customer3|article2|50|2017|4|30 +5|1||20170510|customer4|article5|150|2017|5|10 +5|2||20170510|customer4|article3|100|2017|5|10 +5|3||20170510|customer4|article6|80|2017|5|10 +6|1||20180601|customer2|article4|100|2018|6|1 +6|2||20180601|customer2|article1|50|2018|6|1 +6|3||20180601|customer2|article2|90|2018|6|1 +7|1|N|20180110|customer5|article2|120|2018|1|1 +7|2|N|20180110|customer5|article4|180|2018|1|1 +7|3|N|20180110|customer5|article1|220|2018|1|1 +8|1|N|20180110|customer5|article2|200|2018|1|1 +8|2|N|20180110|customer5|article4|80|2018|1|1 +8|3|N|20180110|customer5|article1|20|2018|1|1 \ No newline at end of file diff --git a/src/test/resources/SparkRecoverPartitionsNativeIntegrationTest/multiple_source_files/lake_data_pre.psv b/src/test/resources/SparkRecoverPartitionsNativeIntegrationTest/multiple_source_files/lake_data_pre.psv new file mode 100644 index 0000000..b1bca8d --- /dev/null +++ b/src/test/resources/SparkRecoverPartitionsNativeIntegrationTest/multiple_source_files/lake_data_pre.psv @@ -0,0 +1,19 @@ +1|1||20160601|customer1|article1|150|2016|6|1 +1|2||20160601|customer1|article2|200|2016|6|1 +1|3||20160601|customer1|article3|50|2016|6|1 +2|1||20170215|customer2|article4|10|2017|2|15 +2|2||20170215|customer2|article5|50|2017|2|15 +2|3||20170215|customer2|article1|30|2017|2|15 +3|1||20170215|customer1|article6|200|2017|2|15 +3|2||20170215|customer1|article2|120|2017|2|15 +3|3||20170215|customer1|article4|90|2017|2|15 +4|1||20170430|customer3|article3|80|2017|4|30 +4|2||20170430|customer3|article7|70|2017|4|30 +4|3||20170430|customer3|article1|30|2017|4|30 +4|4||20170430|customer3|article2|50|2017|4|30 +5|1||20170510|customer4|article5|150|2017|5|10 +5|2||20170510|customer4|article3|100|2017|5|10 +5|3||20170510|customer4|article6|80|2017|5|10 +6|1||20180601|customer2|article4|100|2018|6|1 +6|2||20180601|customer2|article1|50|2018|6|1 +6|3||20180601|customer2|article2|90|2018|6|1 \ No newline at end of file diff --git a/src/test/resources/SparkRecoverPartitionsNativeIntegrationTest/multiple_source_files/params.json b/src/test/resources/SparkRecoverPartitionsNativeIntegrationTest/multiple_source_files/params.json new file mode 100644 index 0000000..6791238 --- /dev/null +++ b/src/test/resources/SparkRecoverPartitionsNativeIntegrationTest/multiple_source_files/params.json @@ -0,0 +1,19 @@ +{ + "target_table": "test_lake.test_table", + "source_dir": "/tmp/tests/test_landing/test_table/data", + "header_dir": "/tmp/tests/test_landing/test_table/header", + "file_format": "dsv", + "delimiter": "|", + "metadata_update_strategy": "SparkRecoverPartitionsNative", + "has_header": false, + "regex_filename": [ + "([0-9]{4})(?=[0-9]{2})(?=[0-9]{2})", + "(?<=[0-9]{4})([0-9]{2})(?=[0-9]{2})", + "(?<=[0-9]{6})([0-9]{2})" + ], + "target_partitions": [ + "year", + "month", + "day" + ] +} \ No newline at end of file diff --git a/src/test/resources/SparkRecoverPartitionsNativeIntegrationTest/multiple_source_files/target_schema.json b/src/test/resources/SparkRecoverPartitionsNativeIntegrationTest/multiple_source_files/target_schema.json new file mode 100644 index 0000000..bdd9036 --- /dev/null +++ b/src/test/resources/SparkRecoverPartitionsNativeIntegrationTest/multiple_source_files/target_schema.json @@ -0,0 +1,54 @@ +{ + "type" : "struct", + "fields" : [ { + "name" : "salesorder", + "type" : "integer", + "nullable" : true, + "metadata" : { } + }, { + "name" : "item", + "type" : "integer", + "nullable" : true, + "metadata" : { } + }, { + "name" : "recordmode", + "type" : "string", + "nullable" : true, + "metadata" : { } + }, { + "name" : "date", + "type" : "integer", + "nullable" : true, + "metadata" : { } + }, { + "name" : "customer", + "type" : "string", + "nullable" : true, + "metadata" : { } + }, { + "name" : "article", + "type" : "string", + "nullable" : true, + "metadata" : { } + }, { + "name" : "amount", + "type" : "integer", + "nullable" : true, + "metadata" : { } + }, { + "name" : "year", + "type" : "short", + "nullable" : true, + "metadata" : { } + }, { + "name" : "month", + "type" : "short", + "nullable" : true, + "metadata" : { } + }, { + "name" : "day", + "type" : "short", + "nullable" : true, + "metadata" : { } + } ] +} diff --git a/src/test/scala/com/adidas/analytics/algo/AlgorithmTemplateTest.scala b/src/test/scala/com/adidas/analytics/feature/AlgorithmTemplateTest.scala similarity index 95% rename from src/test/scala/com/adidas/analytics/algo/AlgorithmTemplateTest.scala rename to src/test/scala/com/adidas/analytics/feature/AlgorithmTemplateTest.scala index a7f81ab..c785eb4 100644 --- a/src/test/scala/com/adidas/analytics/algo/AlgorithmTemplateTest.scala +++ b/src/test/scala/com/adidas/analytics/feature/AlgorithmTemplateTest.scala @@ -1,12 +1,13 @@ -package com.adidas.analytics.algo +package com.adidas.analytics.feature -import com.adidas.analytics.{FileReader, Table} +import com.adidas.analytics.algo.FullLoad import com.adidas.analytics.util.{DFSWrapper, HiveTableAttributeReader, LoadMode} +import com.adidas.utils.TestUtils._ +import com.adidas.utils.{BaseAlgorithmTest, FileReader, Table} import org.apache.hadoop.fs.Path import org.apache.spark.sql.types.{DataType, StructType} import org.scalatest.FeatureSpec import org.scalatest.Matchers._ -import com.adidas.analytics.TestUtils._ class AlgorithmTemplateTest extends FeatureSpec with BaseAlgorithmTest { diff --git a/src/test/scala/com/adidas/analytics/algo/AppendLoadTest.scala b/src/test/scala/com/adidas/analytics/feature/AppendLoadTest.scala similarity index 76% rename from src/test/scala/com/adidas/analytics/algo/AppendLoadTest.scala rename to src/test/scala/com/adidas/analytics/feature/AppendLoadTest.scala index 717afaf..f91c0a0 100644 --- a/src/test/scala/com/adidas/analytics/algo/AppendLoadTest.scala +++ b/src/test/scala/com/adidas/analytics/feature/AppendLoadTest.scala @@ -1,10 +1,12 @@ -package com.adidas.analytics.algo +package com.adidas.analytics.feature -import com.adidas.analytics.TestUtils._ +import com.adidas.analytics.algo.AppendLoad import com.adidas.analytics.util.DFSWrapper._ import com.adidas.analytics.util.{DFSWrapper, LoadMode} -import com.adidas.analytics.{FileReader, Table} +import com.adidas.utils.TestUtils._ +import com.adidas.utils.{BaseAlgorithmTest, FileReader, Table} import org.apache.hadoop.fs.Path +import org.apache.spark.SparkException import org.apache.spark.sql.types.{DataType, StructType} import org.scalatest.FeatureSpec import org.scalatest.Matchers._ @@ -23,6 +25,125 @@ class AppendLoadTest extends FeatureSpec with BaseAlgorithmTest { private val headerDirPath: Path = new Path(hdfsRootTestPath, s"$sourceDatabase/$tableName/header") private val targetDirPath: Path = new Path(hdfsRootTestPath, s"$targetDatabase/$tableName") + feature("Reader mode can be specified in configuration") { + scenario("when reader_mode is set to an invalid string, an exception is thrown") { + val testResourceDir = "reader_mode_specification" + val headerPath20180422 = new Path(headerDirPath, "year=2018/month=4/day=22/header.json") + val targetPath20180422 = new Path(targetDirPath, "year=2018/month=4/day=22") + + val targetSchema = DataType.fromJson(getResourceAsText(s"$testResourceDir/target_schema.json")).asInstanceOf[StructType] + val dataReader = FileReader.newDSVFileReader(Some(targetSchema)) + + val targetTable = createTargetTable(testResourceDir, Seq("year", "month", "day"), targetSchema) + prepareSourceData(testResourceDir, Seq("wrong_data_20180422-00001.psv")) + setupInitialState(targetTable, s"$testResourceDir/lake_data_pre.psv", dataReader) + uploadParameters(testResourceDir, "params_invalid_reader_mode.json") + + // checking pre-conditions + spark.read.option("header", "true").csv(sourceDirPath.toString).count() shouldBe 8 + targetTable.read().count() shouldBe 19 + + fs.exists(targetPath20180422) shouldBe false + fs.exists(headerPath20180422) shouldBe false + + // executing load + val caught = intercept[RuntimeException] { + AppendLoad(spark, dfs, paramsFileHdfsPath.toString).run() + } + + caught.getMessage shouldBe "Invalid reader mode: invalid_mode provided" + } + + scenario("when reader mode is not set, DROPMALFORMED is used") { + val testResourceDir = "reader_mode_specification" + val headerPath20180422 = new Path(headerDirPath, "year=2018/month=4/day=22/header.json") + val targetPath20180422 = new Path(targetDirPath, "year=2018/month=4/day=22") + + val targetSchema = DataType.fromJson(getResourceAsText(s"$testResourceDir/target_schema.json")).asInstanceOf[StructType] + val dataReader = FileReader.newDSVFileReader(Some(targetSchema)) + + val targetTable = createTargetTable(testResourceDir, Seq("year", "month", "day"), targetSchema) + prepareSourceData(testResourceDir, Seq("wrong_data_20180422-00001.psv")) + setupInitialState(targetTable, s"$testResourceDir/lake_data_pre.psv", dataReader) + uploadParameters(testResourceDir, "params_no_reader_mode.json") + + // checking pre-conditions + spark.read.option("header", "true").csv(sourceDirPath.toString).count() shouldBe 8 + targetTable.read().count() shouldBe 19 + + fs.exists(targetPath20180422) shouldBe false + fs.exists(headerPath20180422) shouldBe false + + // executing load + AppendLoad(spark, dfs, paramsFileHdfsPath.toString).run() + + // validating result + val expectedDataLocation = resolveResource(s"$testResourceDir/lake_data_post.psv", withProtocol = true) + val expectedDf = dataReader.read(spark, expectedDataLocation) + val actualDf = targetTable.read() + actualDf.hasDiff(expectedDf) shouldBe true + + fs.exists(targetPath20180422) shouldBe true + fs.exists(headerPath20180422) shouldBe true + } + + scenario("when reader mode is set to FAILFAST and there are malformed records, an exception is thrown") { + val testResourceDir = "reader_mode_specification" + val headerPath20180422 = new Path(headerDirPath, "year=2018/month=4/day=22/header.json") + val targetPath20180422 = new Path(targetDirPath, "year=2018/month=4/day=22") + + val targetSchema = DataType.fromJson(getResourceAsText(s"$testResourceDir/target_schema.json")).asInstanceOf[StructType] + val dataReader = FileReader.newDSVFileReader(Some(targetSchema)) + + val targetTable = createTargetTable(testResourceDir, Seq("year", "month", "day"), targetSchema) + prepareSourceData(testResourceDir, Seq("wrong_data_20180422-00001.psv")) + setupInitialState(targetTable, s"$testResourceDir/lake_data_pre.psv", dataReader) + uploadParameters(testResourceDir, "params_failfast_mode.json") + + // checking pre-conditions + spark.read.option("header", "true").csv(sourceDirPath.toString).count() shouldBe 8 + targetTable.read().count() shouldBe 19 + + fs.exists(targetPath20180422) shouldBe false + fs.exists(headerPath20180422) shouldBe false + + // executing load + assertThrows[SparkException] { + AppendLoad(spark, dfs, paramsFileHdfsPath.toString).run() + } + } + + scenario("when reader mode is set to PERMISSIVE and there are malformed records, data is incorrectly loaded") { + val testResourceDir = "reader_mode_specification" + val headerPath20180422 = new Path(headerDirPath, "year=2018/month=4/day=22/header.json") + val targetPath20180422 = new Path(targetDirPath, "year=2018/month=4/day=22") + + val targetSchema = DataType.fromJson(getResourceAsText(s"$testResourceDir/target_schema.json")).asInstanceOf[StructType] + val dataReader = FileReader.newDSVFileReader(Some(targetSchema)) + + val targetTable = createTargetTable(testResourceDir, Seq("year", "month", "day"), targetSchema) + prepareSourceData(testResourceDir, Seq("wrong_data_20180422-00001.psv")) + setupInitialState(targetTable, s"$testResourceDir/lake_data_pre.psv", dataReader) + uploadParameters(testResourceDir, "params_permissive_mode.json") + + // checking pre-conditions + spark.read.option("header", "true").csv(sourceDirPath.toString).count() shouldBe 8 + targetTable.read().count() shouldBe 19 + + fs.exists(targetPath20180422) shouldBe false + fs.exists(headerPath20180422) shouldBe false + + // executing load + AppendLoad(spark, dfs, paramsFileHdfsPath.toString).run() + + // validating result + val expectedDataLocation = resolveResource(s"$testResourceDir/lake_data_post.psv", withProtocol = true) + val expectedDf = dataReader.read(spark, expectedDataLocation) + val actualDf = targetTable.read() + actualDf.hasDiff(expectedDf) shouldBe true + } + } + feature("Data can be loaded from source to target with append mode") { scenario("Data can be loaded with append mode by creating partitions from full path") { @@ -405,10 +526,10 @@ class AppendLoadTest extends FeatureSpec with BaseAlgorithmTest { copyResourceFileToHdfs(s"$testResourceDir/$whichParamsFile", whichParamsPath) } - private def createTargetTable(testResourceDir: String, partitionColumns: Seq[String], targetSchema: StructType): Table = { + private def createTargetTable(testResourceDir: String, targetPartitions: Seq[String], targetSchema: StructType): Table = { val targetTableLocation = fs.makeQualified(new Path(hdfsRootTestPath, targetDirPath)).toString Table.newBuilder(tableName, targetDatabase, targetTableLocation, targetSchema) - .withPartitions(partitionColumns) + .withPartitions(targetPartitions) .buildParquetTable(DFSWrapper(fs.getConf), spark, external = true) } diff --git a/src/test/scala/com/adidas/analytics/algo/DeltaLoadTest.scala b/src/test/scala/com/adidas/analytics/feature/DeltaLoadTest.scala similarity index 95% rename from src/test/scala/com/adidas/analytics/algo/DeltaLoadTest.scala rename to src/test/scala/com/adidas/analytics/feature/DeltaLoadTest.scala index 2acb6a2..f456669 100644 --- a/src/test/scala/com/adidas/analytics/algo/DeltaLoadTest.scala +++ b/src/test/scala/com/adidas/analytics/feature/DeltaLoadTest.scala @@ -1,8 +1,9 @@ -package com.adidas.analytics.algo +package com.adidas.analytics.feature -import com.adidas.analytics.TestUtils._ +import com.adidas.analytics.algo.DeltaLoad import com.adidas.analytics.util.{DFSWrapper, LoadMode} -import com.adidas.analytics.{FileReader, Table} +import com.adidas.utils.TestUtils._ +import com.adidas.utils.{BaseAlgorithmTest, FileReader, Table} import org.apache.hadoop.fs.Path import org.apache.spark.sql.types.{DataType, StructType} import org.scalatest.FeatureSpec @@ -156,14 +157,14 @@ class DeltaLoadTest extends FeatureSpec with BaseAlgorithmTest { .write.parquet(s"hdfs:/tmp/tests/${dsvResource.replace(".psv", ".parquet")}") } - private def setupEnvironment(partitionColumns: Seq[String], paramsFileName: String, testResourceDir: String): Unit = { + private def setupEnvironment(targetPartitions: Seq[String], paramsFileName: String, testResourceDir: String): Unit = { def createTable(tableName: String, dataFile: String, parquet: Boolean): Table = { val dataLocation = resolveResource(dataFile, withProtocol = true) val schema: StructType = dsvReader.read(spark, dataLocation).schema val tableLocation = fs.makeQualified(new Path(hdfsRootTestPath, s"$testDatabase/$tableName")) val tableBuilder = Table.newBuilder(tableName, testDatabase, tableLocation.toString, schema) - .withPartitions(partitionColumns) + .withPartitions(targetPartitions) val table = if (parquet) { tableBuilder.buildParquetTable(DFSWrapper(fs.getConf), spark, external = true) @@ -185,13 +186,13 @@ class DeltaLoadTest extends FeatureSpec with BaseAlgorithmTest { copyResourceFileToHdfs(s"$testResourceDir/$paramsFileName", paramsFileHdfsPath) } - private def createTableWithSchema(tableName: String, partitionColumns: Seq[String], testResourceDir: String, dsvFileName: Option[String], schemaFileName: String): Table = { + private def createTableWithSchema(tableName: String, targetPartitions: Seq[String], testResourceDir: String, dsvFileName: Option[String], schemaFileName: String): Table = { val tableLocation = fs.makeQualified(new Path(hdfsRootTestPath, s"$testDatabase/$tableName")) val schema = DataType.fromJson(getResourceAsText(s"$testResourceDir/$schemaFileName")).asInstanceOf[StructType] val table = Table.newBuilder(tableName, testDatabase, tableLocation.toString, schema) - .withPartitions(partitionColumns) + .withPartitions(targetPartitions) .buildParquetTable(DFSWrapper(fs.getConf), spark, external = true) dsvFileName match { diff --git a/src/test/scala/com/adidas/analytics/algo/FixedSizeStringExtractorTest.scala b/src/test/scala/com/adidas/analytics/feature/FixedSizeStringExtractorTest.scala similarity index 91% rename from src/test/scala/com/adidas/analytics/algo/FixedSizeStringExtractorTest.scala rename to src/test/scala/com/adidas/analytics/feature/FixedSizeStringExtractorTest.scala index ceed160..d83b15a 100644 --- a/src/test/scala/com/adidas/analytics/algo/FixedSizeStringExtractorTest.scala +++ b/src/test/scala/com/adidas/analytics/feature/FixedSizeStringExtractorTest.scala @@ -1,8 +1,9 @@ -package com.adidas.analytics.algo +package com.adidas.analytics.feature -import com.adidas.analytics.TestUtils.ExtendedDataFrame +import com.adidas.utils.TestUtils._ +import com.adidas.analytics.algo.FixedSizeStringExtractor import com.adidas.analytics.util.{DFSWrapper, LoadMode} -import com.adidas.analytics.{FileReader, Table} +import com.adidas.utils.{BaseAlgorithmTest, FileReader, Table} import org.apache.hadoop.fs.Path import org.apache.spark.sql.types.{DataType, StructType} import org.scalatest.FeatureSpec @@ -96,17 +97,17 @@ class FixedSizeStringExtractorTest extends FeatureSpec with BaseAlgorithmTest { spark.sql(s"CREATE DATABASE $database") } - private def createTable(tableName: String, database: String, schema: StructType, partitionColumns: Seq[String]): Table = { + private def createTable(tableName: String, database: String, schema: StructType, targetPartitions: Seq[String]): Table = { val table = Table.newBuilder(tableName, database, fs.makeQualified(new Path(hdfsRootTestPath, tableName)).toString, schema) - if (partitionColumns.nonEmpty) { - table.withPartitions(partitionColumns).buildParquetTable(DFSWrapper(fs.getConf), spark, external = true) + if (targetPartitions.nonEmpty) { + table.withPartitions(targetPartitions).buildParquetTable(DFSWrapper(fs.getConf), spark, external = true) } else { table.buildParquetTable(DFSWrapper(fs.getConf), spark, external = true) } } - private def prepare(testResourceDir: String, partitionColumns: Seq[String] = Seq.empty, initialData: Boolean = true): Unit = { + private def prepare(testResourceDir: String, targetPartitions: Seq[String] = Seq.empty, initialData: Boolean = true): Unit = { val sourceSchema = DataType.fromJson(getResourceAsText(s"$testResourceDir/source_schema.json")).asInstanceOf[StructType] val targetSchema = DataType.fromJson(getResourceAsText(s"$testResourceDir/target_schema.json")).asInstanceOf[StructType] @@ -114,8 +115,8 @@ class FixedSizeStringExtractorTest extends FeatureSpec with BaseAlgorithmTest { copyResourceFileToHdfs(s"$testResourceDir/$paramsFileName", paramsFileHdfsPath) // create tables - sourceTable = createTable(sourceTableName, database, sourceSchema, partitionColumns) - targetTable = createTable(targetTableName, database, targetSchema, partitionColumns) + sourceTable = createTable(sourceTableName, database, sourceSchema, targetPartitions) + targetTable = createTable(targetTableName, database, targetSchema, targetPartitions) // add data to source table if (initialData) { diff --git a/src/test/scala/com/adidas/analytics/algo/FullLoadTest.scala b/src/test/scala/com/adidas/analytics/feature/FullLoadTest.scala similarity index 59% rename from src/test/scala/com/adidas/analytics/algo/FullLoadTest.scala rename to src/test/scala/com/adidas/analytics/feature/FullLoadTest.scala index d8666a4..62327d2 100644 --- a/src/test/scala/com/adidas/analytics/algo/FullLoadTest.scala +++ b/src/test/scala/com/adidas/analytics/feature/FullLoadTest.scala @@ -1,8 +1,9 @@ -package com.adidas.analytics.algo +package com.adidas.analytics.feature -import com.adidas.analytics.TestUtils._ +import com.adidas.analytics.algo.FullLoad import com.adidas.analytics.util.{DFSWrapper, HiveTableAttributeReader, LoadMode} -import com.adidas.analytics.{FileReader, Table} +import com.adidas.utils.TestUtils._ +import com.adidas.utils.{BaseAlgorithmTest, FileReader, Table} import org.apache.hadoop.fs.Path import org.apache.spark.sql.types.{DataType, StructType} import org.scalatest.FeatureSpec @@ -21,6 +22,134 @@ class FullLoadTest extends FeatureSpec with BaseAlgorithmTest { private val targetDirPath: Path = new Path(hdfsRootTestPath, s"$targetDatabase/test/$tableName/data") private val backupDirPath: Path = new Path(hdfsRootTestPath, s"$targetDatabase/test/$tableName/data_backup") + feature("Reader mode can be specified in configuration") { + scenario("when reader_mode is invalid string an exception is thrown") { + val resourceDir = "failfast_option" + copyResourceFileToHdfs(s"$resourceDir/params_invalid_reader_mode.json", paramsFileHdfsPath) + + val targetSchema = DataType.fromJson(getResourceAsText(s"$resourceDir/target_schema.json")).asInstanceOf[StructType] + val dataReader = FileReader.newDSVFileReader(Some(targetSchema)) + + val targetTable = createNonPartitionedTargetTable(targetSchema) + setupInitialState(targetTable, s"$resourceDir/lake_data_pre.psv", dataReader) + prepareSourceData(Seq(s"$resourceDir/new_data_wrong.psv")) + + // checking pre-conditions + spark.read.csv(sourceDirPath.toString).count() shouldBe 25 + targetTable.read().count() shouldBe 19 + + val caught = intercept[RuntimeException] { + FullLoad(spark, dfs, paramsFileHdfsPath.toString).run() + } + caught.getMessage shouldBe "Invalid reader mode: invalid_mode provided" + } + + scenario("when reader mode is FailFast and malformed records are present, an exception is thrown") { + val resourceDir = "failfast_option" + copyResourceFileToHdfs(s"$resourceDir/params.json", paramsFileHdfsPath) + + val targetSchema = DataType.fromJson(getResourceAsText(s"$resourceDir/target_schema.json")).asInstanceOf[StructType] + val dataReader = FileReader.newDSVFileReader(Some(targetSchema)) + + val targetTable = createNonPartitionedTargetTable(targetSchema) + setupInitialState(targetTable, s"$resourceDir/lake_data_pre.psv", dataReader) + prepareSourceData(Seq(s"$resourceDir/new_data_wrong.psv")) + + // checking pre-conditions + spark.read.csv(sourceDirPath.toString).count() shouldBe 25 + targetTable.read().count() shouldBe 19 + + val caught = intercept[RuntimeException] { + FullLoad(spark, dfs, paramsFileHdfsPath.toString).run() + } + caught.getMessage shouldBe "Unable to process data" + } + + scenario("when reader mode is FailFast and no malformed records are present, load is completed correctly") { + val resourceDir = "failfast_option" + copyResourceFileToHdfs(s"$resourceDir/params.json", paramsFileHdfsPath) + + val targetSchema = DataType.fromJson(getResourceAsText(s"$resourceDir/target_schema.json")).asInstanceOf[StructType] + val dataReader = FileReader.newDSVFileReader(Some(targetSchema)) + + val targetTable = createNonPartitionedTargetTable(targetSchema) + setupInitialState(targetTable, s"$resourceDir/lake_data_pre.psv", dataReader) + prepareDefaultSourceData() + + // checking pre-conditions + spark.read.csv(sourceDirPath.toString).count() shouldBe 25 + targetTable.read().count() shouldBe 19 + + FullLoad(spark, dfs, paramsFileHdfsPath.toString).run() + + // validating result + val expectedDataLocation = resolveResource(s"$resourceDir/lake_data_post.psv", withProtocol = true) + val expectedDf = dataReader.read(spark, expectedDataLocation) + val actualDf = targetTable.read() + actualDf.hasDiff(expectedDf) shouldBe false + + // check the resulting table location is /data folder + val tableLocation = HiveTableAttributeReader(targetTable.table, spark).getTableLocation + tableLocation shouldBe fs.makeQualified(new Path(hdfsRootTestPath, targetDirPath)).toString + } + + scenario("when reader mode is DROPMALFORMED and malformed records are present, some records are not loaded") { + val resourceDir = "failfast_option" + copyResourceFileToHdfs(s"$resourceDir/params_dropmalformed_mode.json", paramsFileHdfsPath) + + val targetSchema = DataType.fromJson(getResourceAsText(s"$resourceDir/target_schema.json")).asInstanceOf[StructType] + val dataReader = FileReader.newDSVFileReader(Some(targetSchema)) + + val targetTable = createNonPartitionedTargetTable(targetSchema) + setupInitialState(targetTable, s"$resourceDir/lake_data_pre.psv", dataReader) + prepareSourceData(Seq(s"$resourceDir/new_data_wrong.psv")) + + // checking pre-conditions + spark.read.csv(sourceDirPath.toString).count() shouldBe 25 + targetTable.read().count() shouldBe 19 + + FullLoad(spark, dfs, paramsFileHdfsPath.toString).run() + + // validating result + val expectedDataLocation = resolveResource(s"$resourceDir/lake_data_post.psv", withProtocol = true) + val expectedDf = dataReader.read(spark, expectedDataLocation) + val actualDf = targetTable.read() + assert(actualDf.count() < expectedDf.count()) + + // check the resulting table location is /data folder + val tableLocation = HiveTableAttributeReader(targetTable.table, spark).getTableLocation + tableLocation shouldBe fs.makeQualified(new Path(hdfsRootTestPath, targetDirPath)).toString + } + + scenario("when reader mode is PERMISSIVE and malformed records are present, malformed records are also loaded") { + val resourceDir = "failfast_option" + copyResourceFileToHdfs(s"$resourceDir/params_permissive_mode.json", paramsFileHdfsPath) + + val targetSchema = DataType.fromJson(getResourceAsText(s"$resourceDir/target_schema.json")).asInstanceOf[StructType] + val dataReader = FileReader.newDSVFileReader(Some(targetSchema)) + + val targetTable = createNonPartitionedTargetTable(targetSchema) + setupInitialState(targetTable, s"$resourceDir/lake_data_pre.psv", dataReader) + prepareSourceData(Seq(s"$resourceDir/new_data_wrong.psv")) + + // checking pre-conditions + spark.read.csv(sourceDirPath.toString).count() shouldBe 25 + targetTable.read().count() shouldBe 19 + + FullLoad(spark, dfs, paramsFileHdfsPath.toString).run() + + // validating result + val expectedDataLocation = resolveResource(s"$resourceDir/lake_data_post.psv", withProtocol = true) + val expectedDf = dataReader.read(spark, expectedDataLocation) + val actualDf = targetTable.read() + actualDf.hasDiff(expectedDf) shouldBe true + actualDf.count() shouldBe expectedDf.count() + + // check the resulting table location is /data folder + val tableLocation = HiveTableAttributeReader(targetTable.table, spark).getTableLocation + tableLocation shouldBe fs.makeQualified(new Path(hdfsRootTestPath, targetDirPath)).toString + } + } feature("Data can be loaded from source to target with full mode") { scenario("Loading data to non-partitioned table") { @@ -90,6 +219,43 @@ class FullLoadTest extends FeatureSpec with BaseAlgorithmTest { fs.listStatus(backupDirPath).length shouldBe 0 } + scenario("Loading data to partitioned table in weekly mode") { + val resourceDir = "partitioned_weekly" + copyResourceFileToHdfs(s"$resourceDir/$paramsFileName", paramsFileHdfsPath) + + val targetPath201801 = new Path(targetDirPath, "year=2018/week=1") + val targetSchema = DataType.fromJson(getResourceAsText(s"$resourceDir/target_schema.json")).asInstanceOf[StructType] + val dataReader = FileReader.newDSVFileReader(Some(targetSchema)) + + val targetTable = createPartitionedTargetTable(Seq("year", "week"), targetSchema, tableName) + setupInitialState(targetTable, s"$resourceDir/lake_data_pre.psv", dataReader) + prepareDefaultSourceData("landing/new_data_weekly.psv") + + // checking pre-conditions + spark.read.csv(sourceDirPath.toString).count() shouldBe 25 + targetTable.read().count() shouldBe 19 + + fs.exists(targetPath201801) shouldBe false + + // executing load + FullLoad(spark, dfs, paramsFileHdfsPath.toString).run() + + // validating result + val expectedDataLocation = resolveResource(s"$resourceDir/lake_data_post.psv", withProtocol = true) + val expectedDf = dataReader.read(spark, expectedDataLocation) + val actualDf = targetTable.read() + actualDf.hasDiff(expectedDf) shouldBe false + + fs.exists(targetPath201801) shouldBe true + + // check the resulting table location is /data folder + val tableLocation = HiveTableAttributeReader(targetTable.table, spark).getTableLocation + tableLocation shouldBe fs.makeQualified(new Path(hdfsRootTestPath, targetDirPath)).toString + + //check backUp dir is empty + fs.listStatus(backupDirPath).length shouldBe 0 + } + scenario("Try loading data from location that does not exist and expect the data to be as it was before load") { val resourceDir = "partitioned" copyResourceFileToHdfs(s"partitioned_not_exist_dir/$paramsFileName", paramsFileHdfsPath) @@ -248,10 +414,10 @@ class FullLoadTest extends FeatureSpec with BaseAlgorithmTest { fs.mkdirs(targetDirPath) } - private def createPartitionedTargetTable(partitionColumns: Seq[String], targetSchema: StructType, tableName: String): Table = { + private def createPartitionedTargetTable(targetPartitions: Seq[String], targetSchema: StructType, tableName: String): Table = { val targetTableLocation = fs.makeQualified(new Path(hdfsRootTestPath, targetDirPath)).toString Table.newBuilder(tableName, targetDatabase, targetTableLocation, targetSchema) - .withPartitions(partitionColumns) + .withPartitions(targetPartitions) .buildParquetTable(DFSWrapper(fs.getConf), spark, external = true) } @@ -261,8 +427,8 @@ class FullLoadTest extends FeatureSpec with BaseAlgorithmTest { .buildParquetTable(DFSWrapper(fs.getConf), spark, external = true) } - private def prepareDefaultSourceData(): Unit = { - prepareSourceData(Seq("landing/new_data.psv")) + private def prepareDefaultSourceData(sourceData: String = "landing/new_data.psv"): Unit = { + prepareSourceData(Seq(sourceData)) } private def prepareSourceData(sourceFiles: Seq[String]): Unit = { diff --git a/src/test/scala/com/adidas/analytics/algo/GzipDecompressorTest.scala b/src/test/scala/com/adidas/analytics/feature/GzipDecompressorTest.scala similarity index 96% rename from src/test/scala/com/adidas/analytics/algo/GzipDecompressorTest.scala rename to src/test/scala/com/adidas/analytics/feature/GzipDecompressorTest.scala index 535fb3b..69f2961 100644 --- a/src/test/scala/com/adidas/analytics/algo/GzipDecompressorTest.scala +++ b/src/test/scala/com/adidas/analytics/feature/GzipDecompressorTest.scala @@ -1,7 +1,9 @@ -package com.adidas.analytics.algo +package com.adidas.analytics.feature -import com.adidas.analytics.TestUtils._ +import com.adidas.utils.TestUtils._ +import com.adidas.analytics.algo.GzipDecompressor import com.adidas.analytics.util.DFSWrapper._ +import com.adidas.utils.BaseAlgorithmTest import org.apache.hadoop.fs.Path import org.scalatest.FeatureSpec import org.scalatest.Matchers._ diff --git a/src/test/scala/com/adidas/analytics/feature/NestedFlattenerTest.scala b/src/test/scala/com/adidas/analytics/feature/NestedFlattenerTest.scala new file mode 100644 index 0000000..96ad025 --- /dev/null +++ b/src/test/scala/com/adidas/analytics/feature/NestedFlattenerTest.scala @@ -0,0 +1,114 @@ +package com.adidas.analytics.feature + +import com.adidas.utils.TestUtils._ +import com.adidas.analytics.algo.NestedFlattener +import com.adidas.utils.{BaseAlgorithmTest, FileReader} +import org.apache.hadoop.fs.Path +import org.apache.spark.sql.types.{DataType, StructType} +import org.scalatest.FeatureSpec +import org.scalatest.Matchers._ + +class NestedFlattenerTest extends FeatureSpec with BaseAlgorithmTest { + + private val database = "test_lake" + private val paramsFileName = "params.json" + private val paramsFileHdfsPath: Path = new Path(hdfsRootTestPath, paramsFileName) + + private val rootSourceDirPath: Path = new Path(hdfsRootTestPath, s"$database/nest") + private val sourceDirPath: Path = new Path(hdfsRootTestPath, s"$database/nest/nest_test/data") + private val sourceDataLocalDir = "nest_test" + + private val expectedTargetTableName = "expected_nest_flattened" + private val expectedDataFileName = "expected_target_data.psv" + + private val targetTableName = "nest_flattened" + private val targetDirPath: Path = new Path(hdfsRootTestPath, s"$database/$targetTableName/data") + private val targetSchema = DataType.fromJson(getResourceAsText(s"target_schema.json")).asInstanceOf[StructType] + + feature("Semi-structured data is fully flattened ... and problematic characters are replaced") { + + scenario("Test Case 1: target Schema is correct and non-partitioned table was successfully flattened and exploded") { + val testCaseId = "scenario1" + + copyResourceFileToHdfs(s"$testCaseId/$paramsFileName", paramsFileHdfsPath) + + val sourceDf = spark.read.parquet(sourceDirPath.toString) + + // source table has the expected number of records + sourceDf.count() shouldBe 3 + + val targetTable = createParquetTable(database, targetTableName, schema = targetSchema) + val nestedFlattener = NestedFlattener(spark, dfs, paramsFileHdfsPath.toString) + nestedFlattener.run() + + // target table has correct schema + targetTable.schema.equals(targetSchema) shouldBe true + + val targetDf = targetTable.read() + + val expectedTargetTable = createAndLoadParquetTable( + database = database, + tableName = expectedTargetTableName, + schema = targetSchema, + filePath = expectedDataFileName, + reader = FileReader.newDSVFileReader(Some(targetSchema))) + val expectedTargetDf = expectedTargetTable.read() + + // target table has exactly the same data as the expected data + targetDf.hasDiff(expectedTargetDf) shouldBe false + } + + scenario("Test Case 2: target Schema is correct and partitioned table was successfully flattened and exploded") { + val testCaseId = "scenario2" + + copyResourceFileToHdfs(s"$testCaseId/$paramsFileName", paramsFileHdfsPath) + + val sourceDf = spark.read.parquet(sourceDirPath.toString) + + // source table has the expected number of records + sourceDf.count() shouldBe 3 + + val targetTable = createParquetTable(database, targetTableName, partitionColumns = Some(Seq("device_brand")), schema = targetSchema) + val nestedFlattener = NestedFlattener(spark, dfs, paramsFileHdfsPath.toString) + nestedFlattener.run() + + // target table has correct schema + // columns have been renamed correctly + targetTable.schema.equals(targetSchema) shouldBe true + + val targetDf = targetTable.read() + + val expectedTargetTable = createAndLoadParquetTable( + database = database, + tableName = expectedTargetTableName, + partitionColumns = Some(Seq("device_brand")), + schema = targetSchema, + filePath = expectedDataFileName, + reader = FileReader.newDSVFileReader(Some(targetSchema))) + val expectedTargetDf = expectedTargetTable.read() + + // target table has exactly the same data as the expected data + targetDf.hasDiff(expectedTargetDf) shouldBe false + } + + } + + /* + * Creates the FS folders, sends the parameters and data to FS, and creates the database + */ + override def beforeEach(): Unit = { + super.beforeEach() + + fs.mkdirs(rootSourceDirPath) + fs.mkdirs(targetDirPath) + + val sourceDataLocalDirPath = resolveResource(s"$sourceDataLocalDir") + copyResourceFileToHdfs(sourceDataLocalDirPath, rootSourceDirPath) + + val targetDataFilePathLocal = resolveResource(s"$expectedDataFileName") + copyResourceFileToHdfs(targetDataFilePathLocal, targetDirPath) + + spark.sql(s"DROP DATABASE IF EXISTS $database CASCADE") + spark.sql(s"CREATE DATABASE $database") + } +} diff --git a/src/test/scala/com/adidas/analytics/algo/PartitionMaterializationTest.scala b/src/test/scala/com/adidas/analytics/feature/PartitionMaterializationTest.scala similarity index 98% rename from src/test/scala/com/adidas/analytics/algo/PartitionMaterializationTest.scala rename to src/test/scala/com/adidas/analytics/feature/PartitionMaterializationTest.scala index eeb7cad..09fd9fb 100644 --- a/src/test/scala/com/adidas/analytics/algo/PartitionMaterializationTest.scala +++ b/src/test/scala/com/adidas/analytics/feature/PartitionMaterializationTest.scala @@ -1,8 +1,9 @@ -package com.adidas.analytics.algo +package com.adidas.analytics.feature -import com.adidas.analytics.TestUtils._ +import com.adidas.analytics.algo.PartitionMaterialization import com.adidas.analytics.util.{DFSWrapper, LoadMode} -import com.adidas.analytics.{FileReader, Table} +import com.adidas.utils.TestUtils._ +import com.adidas.utils.{BaseAlgorithmTest, FileReader, Table} import org.apache.hadoop.fs.Path import org.apache.spark.sql.types.{DataType, StructType} import org.scalatest.FeatureSpec diff --git a/src/test/scala/com/adidas/analytics/algo/SemiStructuredLoadTest.scala b/src/test/scala/com/adidas/analytics/feature/SemiStructuredLoadTest.scala similarity index 98% rename from src/test/scala/com/adidas/analytics/algo/SemiStructuredLoadTest.scala rename to src/test/scala/com/adidas/analytics/feature/SemiStructuredLoadTest.scala index 0b8c80c..10a3012 100644 --- a/src/test/scala/com/adidas/analytics/algo/SemiStructuredLoadTest.scala +++ b/src/test/scala/com/adidas/analytics/feature/SemiStructuredLoadTest.scala @@ -1,10 +1,11 @@ -package com.adidas.analytics.algo +package com.adidas.analytics.feature -import com.adidas.analytics.TestUtils._ +import com.adidas.utils.TestUtils._ +import com.adidas.analytics.algo.AppendLoad import com.adidas.analytics.util.DFSWrapper._ -import com.adidas.analytics.util.DataFormat.{JSONFormat, ParquetFormat} -import com.adidas.analytics.util.{DFSWrapper, LoadMode, OutputWriter} -import com.adidas.analytics.{FileReader, Table} +import com.adidas.analytics.util.DataFormat.ParquetFormat +import com.adidas.analytics.util.OutputWriter +import com.adidas.utils.{BaseAlgorithmTest, FileReader} import org.apache.hadoop.fs.Path import org.apache.spark.sql.types.{DataType, StructType} import org.scalatest.FeatureSpec diff --git a/src/test/scala/com/adidas/analytics/integration/BaseIntegrationTest.scala b/src/test/scala/com/adidas/analytics/integration/BaseIntegrationTest.scala new file mode 100644 index 0000000..6ab8a41 --- /dev/null +++ b/src/test/scala/com/adidas/analytics/integration/BaseIntegrationTest.scala @@ -0,0 +1,71 @@ +package com.adidas.analytics.integration + +import com.adidas.analytics.util.{DFSWrapper, LoadMode} +import com.adidas.utils.{BaseAlgorithmTest, FileReader, Table} +import org.apache.hadoop.fs.Path +import org.apache.spark.sql.types.StructType + +trait BaseIntegrationTest extends BaseAlgorithmTest { + + protected val sourceDatabase: String = "test_landing" + protected val targetDatabase: String = "test_lake" + protected val tableName: String = "test_table" + + protected val paramsFileName: String = "params.json" + protected val paramsFileHdfsPath: Path = new Path(hdfsRootTestPath, paramsFileName) + + protected val sourceDirPath: Path = new Path(hdfsRootTestPath, s"$sourceDatabase/$tableName/data") + protected val headerDirPath: Path = new Path(hdfsRootTestPath, s"$sourceDatabase/$tableName/header") + protected val targetDirPath: Path = new Path(hdfsRootTestPath, s"$targetDatabase/$tableName") + + + override def beforeEach(): Unit = { + super.beforeEach() + spark.sql(s"DROP DATABASE IF EXISTS $targetDatabase CASCADE") + spark.sql(s"DROP DATABASE IF EXISTS $sourceDatabase CASCADE") + spark.sql(s"CREATE DATABASE $sourceDatabase") + spark.sql(s"CREATE DATABASE $targetDatabase") + fs.mkdirs(sourceDirPath) + fs.mkdirs(headerDirPath) + fs.mkdirs(targetDirPath) + } + + protected def uploadParameters(testResourceDir: String, whichParamsFile: String = paramsFileName, whichParamsPath: Path = paramsFileHdfsPath): Unit = { + copyResourceFileToHdfs(s"$testResourceDir/$whichParamsFile", whichParamsPath) + } + + protected def createTargetTable(testResourceDir: String, targetPartitions: Seq[String], targetSchema: StructType): Table = { + val targetTableLocation = fs.makeQualified(new Path(hdfsRootTestPath, targetDirPath)).toString + Table.newBuilder(tableName, targetDatabase, targetTableLocation, targetSchema) + .withPartitions(targetPartitions) + .buildParquetTable(DFSWrapper(fs.getConf), spark, external = true) + } + + protected def prepareSourceData(testResourceDir: String, sourceFiles: Seq[String], sourceDirPath: Path = sourceDirPath): Unit = { + sourceFiles.foreach(file => copyResourceFileToHdfs(s"$testResourceDir/$file", sourceDirPath)) + } + + protected def prepareSourceData(sourceFiles: Seq[String]): Unit = { + sourceFiles.foreach { file => + logger.info(s"copyResourceFileToHdfs $file to ${sourceDirPath.toString}") + copyResourceFileToHdfs(s"$file", sourceDirPath) + } + } + + protected def setupInitialState(targetTable: Table, localDataFile: String, dataReader: FileReader): Unit = { + val initialDataLocation = resolveResource(localDataFile, withProtocol = true) + targetTable.write(Seq(initialDataLocation), dataReader, LoadMode.OverwritePartitionsWithAddedColumns) + } + + protected def createPartitionedTargetTable(targetPartitions: Seq[String], targetSchema: StructType, tableName: String): Table = { + val targetTableLocation = fs.makeQualified(new Path(hdfsRootTestPath, targetDirPath)).toString + Table.newBuilder(tableName, targetDatabase, targetTableLocation, targetSchema) + .withPartitions(targetPartitions) + .buildParquetTable(DFSWrapper(fs.getConf), spark, external = true) + } + + protected def prepareDefaultSourceData(sourceDataFile: String): Unit = { + prepareSourceData(Seq(sourceDataFile)) + } + +} diff --git a/src/test/scala/com/adidas/analytics/integration/FailFastIntegrationTest.scala b/src/test/scala/com/adidas/analytics/integration/FailFastIntegrationTest.scala new file mode 100644 index 0000000..6fcda30 --- /dev/null +++ b/src/test/scala/com/adidas/analytics/integration/FailFastIntegrationTest.scala @@ -0,0 +1,112 @@ +package com.adidas.analytics.integration + +import com.adidas.utils.TestUtils._ +import com.adidas.analytics.algo.FullLoad +import com.adidas.analytics.util.HiveTableAttributeReader +import com.adidas.utils.{FileReader, Table} +import org.apache.hadoop.fs.Path +import org.apache.spark.sql.types.{DataType, StructType} +import org.apache.spark.sql.{Dataset, Encoders} +import org.scalatest.Matchers._ +import org.scalatest.{Assertion, FeatureSpec} + +import scala.collection.JavaConverters._ +import scala.util.{Failure, Success, Try} + +class FailFastIntegrationTest extends FeatureSpec with BaseIntegrationTest { + + override val sourceDirPath: Path = new Path(hdfsRootTestPath, s"$sourceDatabase/test/$tableName/data") + override val targetDirPath: Path = new Path(hdfsRootTestPath, s"$targetDatabase/test/$tableName/data") + protected val backupDirPath: Path = new Path(hdfsRootTestPath, s"$targetDatabase/test/$tableName/data_backup") + + feature("FailFast Option should fail safely regarding data and metadata") { + + scenario("Full Load Algorithm running in FailFast mode and failing safely!") { + val resourceDir = "partitioned" + copyResourceFileToHdfs(s"$resourceDir/$paramsFileName", paramsFileHdfsPath) + + val targetPath20180110 = new Path(targetDirPath, "year=2018/month=1/day=10") + val targetSchema = DataType.fromJson(getResourceAsText(s"$resourceDir/target_schema.json")).asInstanceOf[StructType] + val dataReader = FileReader.newDSVFileReader(Some(targetSchema)) + val expectedPartitionsSchema = DataType.fromJson(getResourceAsText(s"$resourceDir/expected_partitions_schema.json")).asInstanceOf[StructType] + val expectedPartitionsDataReader = FileReader.newDSVFileReader(Some(expectedPartitionsSchema)) + + val targetTable = createPartitionedTargetTable(Seq("year", "month", "day"), targetSchema, tableName) + + // Populate the table with data and Partitions + integrationTestStep(sourceDataFile = "landing/new_data.psv", + resourceDir = resourceDir, + targetPath = targetPath20180110, + shouldFail = false, + dataReader = dataReader, + metadataReader = expectedPartitionsDataReader, + targetTable = targetTable + ) + + // Wrong Data Should not affect table data and partitioning + integrationTestStep(sourceDataFile = "landing/new_data_wrong_format.psv", + resourceDir = resourceDir, + targetPath = targetPath20180110, + shouldFail = true, + dataReader = dataReader, + metadataReader = expectedPartitionsDataReader, + targetTable = targetTable + ) + + } + + } + + private def integrationTestStep(sourceDataFile: String, + shouldFail: Boolean, + resourceDir: String, + targetPath: Path, + dataReader: FileReader, + metadataReader: FileReader, + targetTable: Table): Assertion = { + prepareDefaultSourceData(sourceDataFile) + + // executing load + val isPipelineFailing = Try { + FullLoad(spark, dfs, paramsFileHdfsPath.toString).run() + } match { + case Failure(_) => true + case Success(_) => false + } + + isPipelineFailing should equal(shouldFail) + + // validating result + val expectedDataLocation = resolveResource(s"$resourceDir/lake_data_post.psv", withProtocol = true) + val expectedDf = dataReader.read(spark, expectedDataLocation) + val actualDf = targetTable.read() + + actualDf.hasDiff(expectedDf) shouldBe false + fs.exists(targetPath) shouldBe true + + // check the resulting table location is /data folder + val tableLocation = HiveTableAttributeReader(targetTable.table, spark).getTableLocation + tableLocation shouldBe fs.makeQualified(new Path(hdfsRootTestPath, targetDirPath)).toString + + //check backUp dir is empty + fs.listStatus(backupDirPath).length shouldBe 0 + + // MetaData Specific Tests + val producedPartitionsNumber: Dataset[String] = spark + .sql(s"SHOW PARTITIONS ${targetDatabase}.${tableName}") + .as(Encoders.STRING) + + val expectedPartitionsLocation = resolveResource(s"$resourceDir/expected_partitions.txt", withProtocol = true) + val expectedPartitions: Dataset[String] = metadataReader + .read(spark, expectedPartitionsLocation) + .as(Encoders.STRING) + + + expectedPartitions + .collectAsList() + .asScala + .toSet + .diff(producedPartitionsNumber.collectAsList().asScala.toSet) should equal(Set()) + + } +} diff --git a/src/test/scala/com/adidas/analytics/integration/SparkRecoverPartitionsCustomIntegrationTest.scala b/src/test/scala/com/adidas/analytics/integration/SparkRecoverPartitionsCustomIntegrationTest.scala new file mode 100644 index 0000000..fbe2721 --- /dev/null +++ b/src/test/scala/com/adidas/analytics/integration/SparkRecoverPartitionsCustomIntegrationTest.scala @@ -0,0 +1,70 @@ +package com.adidas.analytics.integration + +import com.adidas.utils.TestUtils._ +import com.adidas.analytics.algo.AppendLoad +import com.adidas.utils.FileReader +import org.apache.hadoop.fs.Path +import org.apache.spark.sql.types.{DataType, StructType} +import org.apache.spark.sql.{Dataset, Encoders} +import org.scalatest.FeatureSpec +import org.scalatest.Matchers._ + +import scala.collection.JavaConverters._ + + +class SparkRecoverPartitionsCustomIntegrationTest extends FeatureSpec with BaseIntegrationTest { + + feature("Partitions can be updated programmatically using custom logic") { + + scenario("Using Append Load Algorithm with multiple source files") { + val testResourceDir = "multiple_source_files" + val headerPath20180101 = new Path(headerDirPath, "year=2018/month=1/day=1/header.json") + val targetPath20180101 = new Path(targetDirPath, "year=2018/month=1/day=1") + + val targetSchema = DataType.fromJson(getResourceAsText(s"$testResourceDir/target_schema.json")).asInstanceOf[StructType] + val expectedPartitionsSchema = DataType.fromJson(getResourceAsText(s"$testResourceDir/expected_partitions_schema.json")).asInstanceOf[StructType] + val dataReader = FileReader.newDSVFileReader(Some(targetSchema)) + val expectedPartitionsDataReader = FileReader.newDSVFileReader(Some(expectedPartitionsSchema)) + + val targetTable = createTargetTable(testResourceDir, Seq("year", "month", "day"), targetSchema) + setupInitialState(targetTable, s"$testResourceDir/lake_data_pre.psv", dataReader) + prepareSourceData(testResourceDir, Seq("data_20180101-part-00000.psv", "data_20180101-part-00001.psv")) + uploadParameters(testResourceDir) + + // checking pre-conditions + spark.read.csv(sourceDirPath.toString).count() shouldBe 7 + targetTable.read().count() shouldBe 19 + + fs.exists(targetPath20180101) shouldBe false + fs.exists(headerPath20180101) shouldBe false + + // executing load + AppendLoad(spark, dfs, paramsFileHdfsPath.toString).run() + + // validating result + val expectedDataLocation = resolveResource(s"$testResourceDir/lake_data_post.psv", withProtocol = true) + val expectedPartitionsLocation = resolveResource(s"$testResourceDir/expected_partitions.txt", withProtocol = true) + val expectedDf = dataReader.read(spark, expectedDataLocation) + val actualDf = targetTable.read() + + val producedPartitionsNumber: Dataset[String] = spark + .sql(s"SHOW PARTITIONS ${targetDatabase}.${tableName}") + .as(Encoders.STRING) + + // MetaData Specific Tests + val expectedPartitions: Dataset[String] = expectedPartitionsDataReader + .read(spark, expectedPartitionsLocation) + .as(Encoders.STRING) + + expectedPartitions.collectAsList().asScala.sorted.toSet should + equal(producedPartitionsNumber.collectAsList().asScala.sorted.toSet) + + actualDf.hasDiff(expectedDf) shouldBe false + + fs.exists(targetPath20180101) shouldBe true + fs.exists(headerPath20180101) shouldBe true + } + } + + +} diff --git a/src/test/scala/com/adidas/analytics/integration/SparkRecoverPartitionsNativeIntegrationTest.scala b/src/test/scala/com/adidas/analytics/integration/SparkRecoverPartitionsNativeIntegrationTest.scala new file mode 100644 index 0000000..d0cd335 --- /dev/null +++ b/src/test/scala/com/adidas/analytics/integration/SparkRecoverPartitionsNativeIntegrationTest.scala @@ -0,0 +1,70 @@ +package com.adidas.analytics.integration + +import com.adidas.utils.TestUtils._ +import com.adidas.analytics.algo.AppendLoad +import com.adidas.utils.FileReader +import org.apache.hadoop.fs.Path +import org.apache.spark.sql.types.{DataType, StructType} +import org.apache.spark.sql.{Dataset, Encoders} +import org.scalatest.FeatureSpec +import org.scalatest.Matchers._ + +import scala.collection.JavaConverters._ + + +class SparkRecoverPartitionsNativeIntegrationTest extends FeatureSpec with BaseIntegrationTest { + + feature("Partitions can be updated with native spark.recoverPartitions()") { + + scenario("Using Append Load Algorithm with multiple source files") { + val testResourceDir = "multiple_source_files" + val headerPath20180101 = new Path(headerDirPath, "year=2018/month=1/day=1/header.json") + val targetPath20180101 = new Path(targetDirPath, "year=2018/month=1/day=1") + + val targetSchema = DataType.fromJson(getResourceAsText(s"$testResourceDir/target_schema.json")).asInstanceOf[StructType] + val expectedPartitionsSchema = DataType.fromJson(getResourceAsText(s"$testResourceDir/expected_partitions_schema.json")).asInstanceOf[StructType] + val dataReader = FileReader.newDSVFileReader(Some(targetSchema)) + val expectedPartitionsDataReader = FileReader.newDSVFileReader(Some(expectedPartitionsSchema)) + + val targetTable = createTargetTable(testResourceDir, Seq("year", "month", "day"), targetSchema) + setupInitialState(targetTable, s"$testResourceDir/lake_data_pre.psv", dataReader) + prepareSourceData(testResourceDir, Seq("data_20180101-part-00000.psv", "data_20180101-part-00001.psv")) + uploadParameters(testResourceDir) + + // checking pre-conditions + spark.read.csv(sourceDirPath.toString).count() shouldBe 7 + targetTable.read().count() shouldBe 19 + + fs.exists(targetPath20180101) shouldBe false + fs.exists(headerPath20180101) shouldBe false + + // executing load + AppendLoad(spark, dfs, paramsFileHdfsPath.toString).run() + + // validating result + val expectedDataLocation = resolveResource(s"$testResourceDir/lake_data_post.psv", withProtocol = true) + val expectedPartitionsLocation = resolveResource(s"$testResourceDir/expected_partitions.txt", withProtocol = true) + val expectedDf = dataReader.read(spark, expectedDataLocation) + val actualDf = targetTable.read() + + val producedPartitionsNumber: Dataset[String] = spark + .sql(s"SHOW PARTITIONS ${targetDatabase}.${tableName}") + .as(Encoders.STRING) + + // MetaData Specific Tests + val expectedPartitions: Dataset[String] = expectedPartitionsDataReader + .read(spark, expectedPartitionsLocation) + .as(Encoders.STRING) + + expectedPartitions.collectAsList().asScala.sorted.toSet should + equal(producedPartitionsNumber.collectAsList().asScala.sorted.toSet) + + actualDf.hasDiff(expectedDf) shouldBe false + + fs.exists(targetPath20180101) shouldBe true + fs.exists(headerPath20180101) shouldBe true + } + } + + +} diff --git a/src/test/scala/com/adidas/analytics/unit/DateComponentDerivationTest.scala b/src/test/scala/com/adidas/analytics/unit/DateComponentDerivationTest.scala new file mode 100644 index 0000000..092d721 --- /dev/null +++ b/src/test/scala/com/adidas/analytics/unit/DateComponentDerivationTest.scala @@ -0,0 +1,181 @@ +package com.adidas.analytics.unit + +import com.adidas.utils.TestUtils._ +import com.adidas.analytics.algo.shared.DateComponentDerivation +import com.adidas.utils.SparkSessionWrapper +import org.apache.spark.sql.DataFrame +import org.scalatest.{BeforeAndAfterAll, FunSuite, Matchers} + +class DateComponentDerivationTest extends FunSuite with SparkSessionWrapper with Matchers with BeforeAndAfterAll { + + class DateComponentDerivationSubClass extends DateComponentDerivation { + def validateWithDateComponents(sourceDateColumnName: String, + sourceDateFormat: String, + targetDateComponentColumnNames: Seq[String])(inputDf: DataFrame): DataFrame = { + super.withDateComponents(sourceDateColumnName, sourceDateFormat, targetDateComponentColumnNames)(inputDf) + } + + } + + import spark.implicits._ + + override def afterAll(): Unit = { + spark.stop() + } + + test("Partition by year/week with formatter yyyyww") { + + val sampleDf = Seq( + ("201301"), + ("201531"), + ("202001") + ).toDF("zatpweek") + + val dateComponentDerivationTester: DataFrame => DataFrame = new DateComponentDerivationSubClass() + .validateWithDateComponents( + sourceDateColumnName = "zatpweek", + sourceDateFormat = "yyyyww", + targetDateComponentColumnNames = Seq("year", "week") + ) + + val transformedDf = sampleDf.transform(dateComponentDerivationTester) + + val expectedDf = Seq( + ("201301",2013,1), + ("201531",2015,31), + ("202001",2020,1) + ).toDF("zatpweek","year","week") + + transformedDf.hasDiff(expectedDf) shouldBe false + } + + test("Partition by year/month/day with formatter yyyyMMdd") { + + val sampleDf = Seq( + ("20130112"), + ("20150815"), + ("20200325") + ).toDF("partcol") + + val dateComponentDerivationTester: DataFrame => DataFrame = new DateComponentDerivationSubClass() + .validateWithDateComponents( + sourceDateColumnName = "partcol", + sourceDateFormat = "yyyyMMdd", + targetDateComponentColumnNames = Seq("year", "month","day") + ) + + val transformedDf = sampleDf.transform(dateComponentDerivationTester) + + val expectedDf = Seq( + ("20130112",2013,1,12), + ("20150815",2015,8,15), + ("20200325",2020,3,25) + ).toDF("partcol","year","month","day") + + transformedDf.hasDiff(expectedDf) shouldBe false + } + + test("Partition by year/month with formatter yyyyMMdd") { + + val sampleDf = Seq( + ("20130112"), + ("20150815"), + ("20200325") + ).toDF("partcol") + + val dateComponentDerivationTester: DataFrame => DataFrame = new DateComponentDerivationSubClass() + .validateWithDateComponents( + sourceDateColumnName = "partcol", + sourceDateFormat = "yyyyMMdd", + targetDateComponentColumnNames = Seq("year", "month") + ) + + val transformedDf = sampleDf.transform(dateComponentDerivationTester) + + val expectedDf = Seq( + ("20130112",2013,1), + ("20150815",2015,8), + ("20200325",2020,3) + ).toDF("partcol","year","month") + + transformedDf.hasDiff(expectedDf) shouldBe false + } + + test("Partition by year/month with formatter yyyyMMdd - with wrong data") { + + val sampleDf = Seq( + ("20130112"), + ("201508151"), + ("20200325") + ).toDF("partcol") + + val dateComponentDerivationTester: DataFrame => DataFrame = new DateComponentDerivationSubClass() + .validateWithDateComponents( + sourceDateColumnName = "partcol", + sourceDateFormat = "yyyyMMdd", + targetDateComponentColumnNames = Seq("year", "month") + ) + + val transformedDf = sampleDf.transform(dateComponentDerivationTester) + + val expectedDf = Seq( + ("20130112",2013,1), + ("201508151",9999,99), + ("20200325",2020,3) + ).toDF("partcol","year","month") + + transformedDf.hasDiff(expectedDf) shouldBe false + } + + test("Partition by year/month with formatter yyyyMM as IntegerType") { + + val sampleDf = Seq( + (201301), + (2015233), + (202003) + ).toDF("partcol") + + val dateComponentDerivationTester: DataFrame => DataFrame = new DateComponentDerivationSubClass() + .validateWithDateComponents( + sourceDateColumnName = "partcol", + sourceDateFormat = "yyyyMM", + targetDateComponentColumnNames = Seq("year", "month") + ) + + val transformedDf = sampleDf.transform(dateComponentDerivationTester) + val expectedDf = Seq( + (201301,2013,1), + (2015233,9999,99), + (202003,2020,3) + ).toDF("partcol","year","month") + + transformedDf.hasDiff(expectedDf) shouldBe false + } + + test("Partition by year/week/day with formatter yyyywwe as IntegerType") { + + val sampleDf = Seq( + (2013014), + (2015233), + (2020037) + ).toDF("partcol") + + val dateComponentDerivationTester: DataFrame => DataFrame = new DateComponentDerivationSubClass() + .validateWithDateComponents( + sourceDateColumnName = "partcol", + sourceDateFormat = "yyyywwe", + targetDateComponentColumnNames = Seq("year", "week", "day") + ) + + val transformedDf = sampleDf.transform(dateComponentDerivationTester) + + val expectedDf = Seq( + (2013014,2013,1,4), + (2015233,2015,23,3), + (2020037,2020,3,7) + ).toDF("partcol", "year", "week", "day") + + transformedDf.hasDiff(expectedDf) shouldBe false + } + +} diff --git a/src/test/scala/com/adidas/analytics/unit/SparkRecoverPartitionsCustomTest.scala b/src/test/scala/com/adidas/analytics/unit/SparkRecoverPartitionsCustomTest.scala new file mode 100644 index 0000000..b23feeb --- /dev/null +++ b/src/test/scala/com/adidas/analytics/unit/SparkRecoverPartitionsCustomTest.scala @@ -0,0 +1,99 @@ +package com.adidas.analytics.unit + +import com.adidas.analytics.util.SparkRecoverPartitionsCustom +import com.adidas.utils.SparkSessionWrapper +import org.apache.spark.sql.catalyst.encoders.RowEncoder +import org.apache.spark.sql.types.{IntegerType, StringType, StructField, StructType} +import org.apache.spark.sql.{Dataset, Row} +import org.scalatest.{BeforeAndAfterAll, FunSuite, Matchers, PrivateMethodTester} + +import scala.collection.JavaConverters._ + +class SparkRecoverPartitionsCustomTest extends FunSuite + with SparkSessionWrapper + with PrivateMethodTester + with Matchers + with BeforeAndAfterAll{ + + test("test conversion of String Value to HiveQL Partition Parameter") { + val customSparkRecoverPartitions = SparkRecoverPartitionsCustom(tableName="", targetPartitions = Seq()) + val createParameterValue = PrivateMethod[String]('createParameterValue) + val result = customSparkRecoverPartitions invokePrivate createParameterValue("theValue") + + result should be("'theValue'") + } + + test("test conversion of Short Value to HiveQL Partition Parameter") { + val customSparkRecoverPartitions = SparkRecoverPartitionsCustom(tableName="", targetPartitions = Seq()) + val createParameterValue = PrivateMethod[String]('createParameterValue) + val result = customSparkRecoverPartitions invokePrivate createParameterValue(java.lang.Short.valueOf("2")) + + result should be("2") + } + + test("test conversion of Integer Value to HiveQL Partition Parameter") { + val customSparkRecoverPartitions = SparkRecoverPartitionsCustom(tableName="", targetPartitions = Seq()) + val createParameterValue = PrivateMethod[String]('createParameterValue) + val result = customSparkRecoverPartitions invokePrivate createParameterValue(java.lang.Integer.valueOf("4")) + + result should be("4") + } + + test("test conversion of null Value to HiveQL Partition Parameter") { + val customSparkRecoverPartitions = SparkRecoverPartitionsCustom(tableName="", targetPartitions = Seq()) + val createParameterValue = PrivateMethod[String]('createParameterValue) + an [Exception] should be thrownBy { + customSparkRecoverPartitions invokePrivate createParameterValue(null) + } + } + + test("test conversion of not supported Value to HiveQL Partition Parameter") { + val customSparkRecoverPartitions = SparkRecoverPartitionsCustom(tableName="", targetPartitions = Seq()) + val createParameterValue = PrivateMethod[String]('createParameterValue) + an [Exception] should be thrownBy { + customSparkRecoverPartitions invokePrivate createParameterValue(false) + } + } + + test("test HiveQL statements Generation") { + val customSparkRecoverPartitions = SparkRecoverPartitionsCustom( + tableName="test", + targetPartitions = Seq("country","district") + ) + + val rowsInput = Seq( + Row(1, "portugal", "porto"), + Row(2, "germany", "herzogenaurach"), + Row(3, "portugal", "coimbra") + ) + + val inputSchema = StructType( + List( + StructField("number", IntegerType, nullable = true), + StructField("country", StringType, nullable = true), + StructField("district", StringType, nullable = true) + ) + ) + + val expectedStatements: Seq[String] = Seq( + "ALTER TABLE test ADD IF NOT EXISTS PARTITION(country='portugal',district='porto')", + "ALTER TABLE test ADD IF NOT EXISTS PARTITION(country='germany',district='herzogenaurach')", + "ALTER TABLE test ADD IF NOT EXISTS PARTITION(country='portugal',district='coimbra')" + ) + + val testDataset: Dataset[Row] = spark.createDataset(rowsInput)(RowEncoder(inputSchema)) + + val createParameterValue = PrivateMethod[Dataset[String]]('generateAddPartitionStatements) + + val producedStatements: Seq[String] = (customSparkRecoverPartitions invokePrivate createParameterValue(testDataset)) + .collectAsList() + .asScala + + expectedStatements.sorted.toSet should equal(producedStatements.sorted.toSet) + } + + override def afterAll(): Unit = { + spark.stop() + } + +} diff --git a/src/test/scala/com/adidas/analytics/algo/BaseAlgorithmTest.scala b/src/test/scala/com/adidas/utils/BaseAlgorithmTest.scala similarity index 60% rename from src/test/scala/com/adidas/analytics/algo/BaseAlgorithmTest.scala rename to src/test/scala/com/adidas/utils/BaseAlgorithmTest.scala index 7457a6d..31ca9b5 100644 --- a/src/test/scala/com/adidas/analytics/algo/BaseAlgorithmTest.scala +++ b/src/test/scala/com/adidas/utils/BaseAlgorithmTest.scala @@ -1,17 +1,16 @@ -package com.adidas.analytics.algo +package com.adidas.utils import java.util.UUID -import com.adidas.analytics.util.DFSWrapper -import com.adidas.analytics.{HDFSSupport, SparkSupport} +import com.adidas.analytics.util.{DFSWrapper, LoadMode} import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path +import org.apache.spark.sql.types.StructType import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, Suite} import org.slf4j.{Logger, LoggerFactory} import scala.io.Source - trait BaseAlgorithmTest extends Suite with BeforeAndAfterAll with BeforeAndAfterEach with HDFSSupport with SparkSupport { override val logger: Logger = LoggerFactory.getLogger(getClass) @@ -63,4 +62,28 @@ trait BaseAlgorithmTest extends Suite with BeforeAndAfterAll with BeforeAndAfter logger.info(s"Copying local resource to HDFS $sourcePath -> $targetPath") fs.copyFromLocalFile(sourcePath, targetPath) } + + /* + * Creates (but does not load) a Parquet table for testing purposes + */ + def createParquetTable(database: String, tableName: String, partitionColumns: Option[Seq[String]] = None, schema: StructType): Table = { + val inputTableLocation = fs.makeQualified(new Path(hdfsRootTestPath, s"$database/$tableName")).toString + if (partitionColumns.isEmpty) + Table.newBuilder(tableName, database, inputTableLocation, schema) + .buildParquetTable(DFSWrapper(fs.getConf), spark, external = true) + else + Table.newBuilder(tableName, database, inputTableLocation, schema) + .withPartitions(partitionColumns.get) + .buildParquetTable(DFSWrapper(fs.getConf), spark, external = true) + } + + /* + * Creates and Loads Parquet table for testing purposes + */ + def createAndLoadParquetTable(database: String, tableName: String, partitionColumns: Option[Seq[String]] = None, schema: StructType, filePath: String, reader: FileReader): Table = { + val table = createParquetTable(database, tableName, partitionColumns, schema) + val inputTableDataURI = resolveResource(filePath, withProtocol = true) + table.write(Seq(inputTableDataURI), reader, LoadMode.OverwritePartitions) + table + } } diff --git a/src/test/scala/com/adidas/analytics/FileReader.scala b/src/test/scala/com/adidas/utils/FileReader.scala similarity index 98% rename from src/test/scala/com/adidas/analytics/FileReader.scala rename to src/test/scala/com/adidas/utils/FileReader.scala index bcc45ee..c56ecb2 100644 --- a/src/test/scala/com/adidas/analytics/FileReader.scala +++ b/src/test/scala/com/adidas/utils/FileReader.scala @@ -1,4 +1,4 @@ -package com.adidas.analytics +package com.adidas.utils import com.adidas.analytics.util.DataFormat import com.adidas.analytics.util.DataFormat.{DSVFormat, JSONFormat, ParquetFormat} diff --git a/src/test/scala/com/adidas/analytics/HDFSSupport.scala b/src/test/scala/com/adidas/utils/HDFSSupport.scala similarity index 97% rename from src/test/scala/com/adidas/analytics/HDFSSupport.scala rename to src/test/scala/com/adidas/utils/HDFSSupport.scala index 73a35d9..6cb20ad 100644 --- a/src/test/scala/com/adidas/analytics/HDFSSupport.scala +++ b/src/test/scala/com/adidas/utils/HDFSSupport.scala @@ -1,4 +1,4 @@ -package com.adidas.analytics +package com.adidas.utils import java.io.File diff --git a/src/test/scala/com/adidas/utils/SparkSessionWrapper.scala b/src/test/scala/com/adidas/utils/SparkSessionWrapper.scala new file mode 100644 index 0000000..09de5be --- /dev/null +++ b/src/test/scala/com/adidas/utils/SparkSessionWrapper.scala @@ -0,0 +1,30 @@ +package com.adidas.utils + +import org.apache.hadoop.conf.Configuration +import org.apache.spark.SparkConf +import org.apache.spark.sql.SparkSession + +import scala.collection.JavaConversions._ + +trait SparkSessionWrapper { + + lazy val spark: SparkSession = startSpark(sparkHadoopConf) + + def sparkHadoopConf: Option[Configuration] = Option.empty + + def startSpark(hadoopConf: Option[Configuration]): SparkSession = { + + val sparkConf = hadoopConf.foldLeft { + new SparkConf(false) + } { (sparkConf, hadoopConf) => + hadoopConf.foldLeft(sparkConf)((sc, entry) => sc.set(entry.getKey, entry.getValue)) + } + + SparkSession.builder() + .config(sparkConf) + .appName("spark tests") + .master("local[*]") + .getOrCreate() + } + +} diff --git a/src/test/scala/com/adidas/analytics/SparkSupport.scala b/src/test/scala/com/adidas/utils/SparkSupport.scala similarity index 86% rename from src/test/scala/com/adidas/analytics/SparkSupport.scala rename to src/test/scala/com/adidas/utils/SparkSupport.scala index 32f8f90..3d0637b 100644 --- a/src/test/scala/com/adidas/analytics/SparkSupport.scala +++ b/src/test/scala/com/adidas/utils/SparkSupport.scala @@ -1,4 +1,4 @@ -package com.adidas.analytics +package com.adidas.utils import java.io.File @@ -9,17 +9,15 @@ import org.slf4j.Logger import scala.collection.JavaConversions._ - -trait SparkSupport { - - lazy val spark: SparkSession = startSpark(sparkHadoopConf) +trait SparkSupport extends SparkSessionWrapper { def logger: Logger + def testAppId: String + def localTestDir: String - def sparkHadoopConf: Option[Configuration] = Option.empty - def startSpark(hadoopConf: Option[Configuration]): SparkSession = { + override def startSpark(hadoopConf: Option[Configuration] = None): SparkSession = { // This line makes it possible to use a remote debugger System.setSecurityManager(null) @@ -55,4 +53,5 @@ trait SparkSupport { def addHadoopProperty(key: String, value: String): Unit = { spark.sparkContext.hadoopConfiguration.set(key, value) } + } diff --git a/src/test/scala/com/adidas/analytics/Table.scala b/src/test/scala/com/adidas/utils/Table.scala similarity index 87% rename from src/test/scala/com/adidas/analytics/Table.scala rename to src/test/scala/com/adidas/utils/Table.scala index f84cfbc..6651498 100644 --- a/src/test/scala/com/adidas/analytics/Table.scala +++ b/src/test/scala/com/adidas/utils/Table.scala @@ -1,4 +1,4 @@ -package com.adidas.analytics +package com.adidas.utils import com.adidas.analytics.util.DataFormat.{DSVFormat, ParquetFormat} import com.adidas.analytics.util._ @@ -7,7 +7,7 @@ import org.apache.spark.sql.types.{StructField, StructType} import org.slf4j.{Logger, LoggerFactory} -case class Table private(dfs: DFSWrapper, spark: SparkSession, table: String, location: String, schema: StructType, partitionColumns: Seq[String], format: DataFormat, options: Map[String, String]) { +case class Table private(dfs: DFSWrapper, spark: SparkSession, table: String, location: String, schema: StructType, targetPartitions: Seq[String], format: DataFormat, options: Map[String, String]) { def read(): DataFrame = { spark.read.table(table) @@ -41,7 +41,7 @@ case class Table private(dfs: DFSWrapper, spark: SparkSession, table: String, lo } private def invalidateCaches(): Unit = { - if (partitionColumns.nonEmpty) { + if (targetPartitions.nonEmpty) { spark.catalog.recoverPartitions(table) } else { spark.catalog.refreshTable(table) @@ -49,7 +49,7 @@ case class Table private(dfs: DFSWrapper, spark: SparkSession, table: String, lo } private def createWriter(loadMode: LoadMode): OutputWriter = { - OutputWriter.newFileSystemWriter(location, format, partitionColumns, options + ("emptyValue" -> ""), loadMode) + OutputWriter.newFileSystemWriter(location, format, targetPartitions, options + ("emptyValue" -> ""), loadMode) } } @@ -80,14 +80,14 @@ object Table { private val fullTableName: String = s"$database.$table" - private var partitionColumns: Seq[String] = Seq() + private var targetPartitions: Seq[String] = Seq() private val defaultDSVOptions: Map[String, String] = Map("delimiter" -> "|") private var options: Map[String, String] = Map() - def withPartitions(partitionColumns: Seq[String]): TableBuilder = { - this.partitionColumns = partitionColumns + def withPartitions(targetPartitions: Seq[String]): TableBuilder = { + this.targetPartitions = targetPartitions this } @@ -106,12 +106,12 @@ object Table { private def buildTable(format: DataFormat, options: Map[String, String], dfs: DFSWrapper, spark: SparkSession, external: Boolean): Table = { createHiveTable(format, options, spark, external) - new Table(dfs, spark, fullTableName, location, schema, partitionColumns, format, options) + new Table(dfs, spark, fullTableName, location, schema, targetPartitions, format, options) } private def createHiveTable(format: DataFormat, options: Map[String, String], spark: SparkSession, external: Boolean): Unit = { val fieldMap = schema.fields.map(f => (f.name, f)).toMap - val partitionColumnFields = partitionColumns.map(fieldMap) + val partitionColumnFields = targetPartitions.map(fieldMap) val columnFields = schema.fields.diff(partitionColumnFields) val partitionColumnDefinitions = Table.buildColumnDefinitions(partitionColumnFields) @@ -125,7 +125,7 @@ object Table { statementBuilder += s"CREATE TABLE $fullTableName($columnDefinitions)" } - if (partitionColumns.nonEmpty) { + if (targetPartitions.nonEmpty) { statementBuilder += s"PARTITIONED BY ($partitionColumnDefinitions)" } diff --git a/src/test/scala/com/adidas/analytics/TestUtils.scala b/src/test/scala/com/adidas/utils/TestUtils.scala similarity index 96% rename from src/test/scala/com/adidas/analytics/TestUtils.scala rename to src/test/scala/com/adidas/utils/TestUtils.scala index 94cfd30..c7ff8d8 100644 --- a/src/test/scala/com/adidas/analytics/TestUtils.scala +++ b/src/test/scala/com/adidas/utils/TestUtils.scala @@ -1,4 +1,4 @@ -package com.adidas.analytics +package com.adidas.utils import org.apache.spark.sql.functions.{col, count, lit} import org.apache.spark.sql.{DataFrame, Row}