From 950cc8ad2de5435af24c223a37a0f5e70e1df8bc Mon Sep 17 00:00:00 2001 From: Adrian Olosutean Date: Tue, 15 Feb 2022 13:46:36 +0100 Subject: [PATCH 1/6] #2022 spark-commons init --- .../auth/MenasPlainCredentialsSuite.scala | 2 +- .../UppercaseCustomConformanceRuleSuite.scala | 3 +- .../XPadCustomConformanceRuleSuite.scala | 4 +- .../mq/KafkaErrorSenderPluginImpl.scala | 4 +- .../mq/KafkaErrorSenderPluginSuite.scala | 2 +- pom.xml | 3 +- .../SparkMenasSchemaConvertor.scala | 2 +- .../SparkMenasSchemaConvertorSuite.scala | 2 +- .../enceladus/common/CommonJobExecution.scala | 10 +- .../performance/PerformanceMetricTools.scala | 4 +- .../conformance/ConformanceExecution.scala | 4 +- .../conformance/HyperConformance.scala | 10 +- .../interpreter/DynamicInterpreter.scala | 10 +- .../interpreter/ExplosionState.scala | 2 +- .../interpreter/OptimizerTimeTracker.scala | 4 +- .../rules/ArrayCollapseInterpreter.scala | 3 +- .../rules/ArrayExplodeInterpreter.scala | 2 +- .../rules/CastingRuleInterpreter.scala | 9 +- .../rules/DropRuleInterpreter.scala | 4 +- .../rules/FillNullsRuleInterpreter.scala | 4 +- .../rules/NegationRuleInterpreter.scala | 4 +- .../CommonMappingRuleInterpreter.scala | 5 +- .../mapping/MappingRuleInterpreter.scala | 8 +- .../MappingRuleInterpreterBroadcast.scala | 2 +- .../MappingRuleInterpreterGroupExplode.scala | 4 +- .../streaming/InfoDateFactory.scala | 4 +- .../StandardizationExecution.scala | 4 +- .../StandardizationInterpreter.scala | 7 +- .../stages/PlainSchemaGenerator.scala | 5 +- .../interpreter/stages/TypeParser.scala | 11 +- .../common/CommonExecutionSuite.scala | 2 +- .../common/RecordIdGenerationSuite.scala | 2 +- .../config/ConformanceParserSuite.scala | 2 +- .../datasource/DatasourceSuite.scala | 2 +- .../interpreter/ArrayConformanceSuite.scala | 3 +- .../interpreter/ChorusMockSuite.scala | 3 +- .../interpreter/InterpreterSuite.scala | 3 +- .../LiteralJoinMappingRuleTest.scala | 3 +- .../interpreter/NestedStructSuite.scala | 3 +- .../fixtures/MultipleMappingFixture.scala | 2 +- .../fixtures/NestedStructsFixture.scala | 3 +- .../fixtures/StreamingFixture.scala | 2 +- .../interpreter/rules/CastingRuleSuite.scala | 5 +- .../interpreter/rules/CoalesceRuleSuite.scala | 2 +- .../rules/ConcatenationRuleSuite.scala | 2 +- .../interpreter/rules/DropRuleSuite.scala | 2 +- .../rules/FillNullsRuleSuite.scala | 2 +- .../interpreter/rules/LiteralRuleSuite.scala | 2 +- .../interpreter/rules/NegationRuleSuite.scala | 3 +- .../rules/RuleOptimizationSuite.scala | 3 +- .../interpreter/rules/RulesSuite.scala | 2 +- .../rules/SingleColumnRuleSuite.scala | 2 +- .../rules/SparkSessionRuleSuite.scala | 2 +- .../interpreter/rules/TestRuleBehaviors.scala | 3 +- .../rules/UppercaseRuleSuite.scala | 2 +- .../rules/custom/CustomRuleSuite.scala | 3 +- .../JoinMappingRuleInterpreterSuite.scala | 2 +- .../mapping/MappingGroupExplodeSuite.scala | 2 +- .../mapping/MappingInterpreterSuite.scala | 3 +- .../mapping/MappingRuleBroadcastSuite.scala | 2 +- .../rules/mapping/MappingRuleSuite.scala | 3 +- .../StandardizationCobolAsciiSuite.scala | 4 +- .../StandardizationCobolEbcdicSuite.scala | 2 +- .../StandardizationExecutionSuite.scala | 4 +- .../StandardizationFixedWidthSuite.scala | 4 +- .../StandardizationJsonSuite.scala | 4 +- .../StandardizationParquetSuite.scala | 5 +- .../StandardizationRerunSuite.scala | 5 +- .../StandardizationXmlSuite.scala | 4 +- .../config/StandardizationParserSuite.scala | 2 +- .../csv/EnhancedStandardizationCsvSuite.scala | 2 +- ...NoneParameterStandardizationCsvSuite.scala | 2 +- .../NullValueStandardizationCsvSuite.scala | 4 +- .../csv/StandardizationCsvSuite.scala | 2 +- .../WhiteSpaceStandardizationCsvSuite.scala | 2 +- .../fixtures/CsvFileFixture.scala | 2 +- .../interpreter/CounterPartySuite.scala | 3 +- .../interpreter/DateTimeSuite.scala | 4 +- .../interpreter/SampleDataSuite.scala | 3 +- .../StandardizationInterpreterSuite.scala | 5 +- ...tandardizationInterpreter_ArraySuite.scala | 7 +- ...andardizationInterpreter_BinarySuite.scala | 3 +- ...StandardizationInterpreter_DateSuite.scala | 4 +- ...ndardizationInterpreter_DecimalSuite.scala | 4 +- ...rdizationInterpreter_FractionalSuite.scala | 3 +- ...dardizationInterpreter_IntegralSuite.scala | 4 +- ...ardizationInterpreter_TimestampSuite.scala | 4 +- .../interpreter/StdInterpreterSuite.scala | 4 +- .../stages/PlainSchemaGeneratorSuite.scala | 2 +- .../stages/SchemaCheckerSuite.scala | 2 +- .../interpreter/stages/TypeParserSuite.scala | 2 +- .../stages/TypeParserSuiteTemplate.scala | 2 +- utils/pom.xml | 5 + .../utils/broadcast/LocalMappingTable.scala | 14 +- .../utils/explode/ExplodeTools.scala | 382 --------- .../enceladus/utils/explode/Explosion.scala | 27 - .../utils/explode/ExplosionContext.scala | 49 -- .../enceladus/utils/general/JsonUtils.scala | 60 -- .../utils/implicits/DataFrameImplicits.scala | 73 -- ...s.scala => EnceladusColumnImplicits.scala} | 51 +- .../implicits/StructFieldImplicits.scala | 47 - .../enceladus/utils/schema/SchemaUtils.scala | 505 +---------- .../enceladus/utils/schema/SparkUtils.scala | 9 +- .../utils/testUtils/LoggerTestBase.scala | 4 +- .../utils/testUtils/SparkLocalMaster.scala | 20 - .../utils/testUtils/SparkTestBase.scala | 136 --- .../ArrayTransformations.scala | 18 +- .../utils/types/TypedStructField.scala | 27 +- .../field/BinaryFieldValidator.scala | 4 +- .../field/DateTimeFieldValidator.scala | 5 +- .../validation/field/FieldValidator.scala | 10 +- .../field/IntegralFieldValidator.scala | 3 +- .../absa/enceladus/utils/ExplosionSuite.scala | 808 ------------------ .../utils/broadcast/BroadcastUtilsSuite.scala | 3 +- .../broadcast/LocalMappingTableSuite.scala | 4 +- .../enceladus/utils/fs/HdfsUtilsSpec.scala | 3 +- .../utils/general/JsonUtilsSuite.scala | 63 -- .../implicits/DataFrameImplicitsSuite.scala | 4 +- .../utils/schema/SchemaUtilsSuite.scala | 476 ----------- .../utils/schema/SparkUtilsSuite.scala | 4 +- .../ArrayTransformationsSuite.scala | 3 +- 121 files changed, 264 insertions(+), 2876 deletions(-) delete mode 100644 utils/src/main/scala/za/co/absa/enceladus/utils/explode/ExplodeTools.scala delete mode 100644 utils/src/main/scala/za/co/absa/enceladus/utils/explode/Explosion.scala delete mode 100644 utils/src/main/scala/za/co/absa/enceladus/utils/explode/ExplosionContext.scala delete mode 100644 utils/src/main/scala/za/co/absa/enceladus/utils/general/JsonUtils.scala delete mode 100644 utils/src/main/scala/za/co/absa/enceladus/utils/implicits/DataFrameImplicits.scala rename utils/src/main/scala/za/co/absa/enceladus/utils/implicits/{ColumnImplicits.scala => EnceladusColumnImplicits.scala} (55%) delete mode 100644 utils/src/main/scala/za/co/absa/enceladus/utils/implicits/StructFieldImplicits.scala delete mode 100644 utils/src/main/scala/za/co/absa/enceladus/utils/testUtils/SparkLocalMaster.scala delete mode 100644 utils/src/main/scala/za/co/absa/enceladus/utils/testUtils/SparkTestBase.scala delete mode 100644 utils/src/test/scala/za/co/absa/enceladus/utils/ExplosionSuite.scala delete mode 100644 utils/src/test/scala/za/co/absa/enceladus/utils/general/JsonUtilsSuite.scala delete mode 100644 utils/src/test/scala/za/co/absa/enceladus/utils/schema/SchemaUtilsSuite.scala diff --git a/dao/src/test/scala/za/co/absa/enceladus/dao/rest/auth/MenasPlainCredentialsSuite.scala b/dao/src/test/scala/za/co/absa/enceladus/dao/rest/auth/MenasPlainCredentialsSuite.scala index e7c8429d9..40247982d 100644 --- a/dao/src/test/scala/za/co/absa/enceladus/dao/rest/auth/MenasPlainCredentialsSuite.scala +++ b/dao/src/test/scala/za/co/absa/enceladus/dao/rest/auth/MenasPlainCredentialsSuite.scala @@ -18,7 +18,7 @@ package za.co.absa.enceladus.dao.rest.auth import org.scalatest.wordspec.AnyWordSpec import za.co.absa.enceladus.dao.auth.MenasPlainCredentials import za.co.absa.enceladus.utils.fs.LocalFsUtils -import za.co.absa.enceladus.utils.testUtils.SparkTestBase +import za.co.absa.spark.commons.test.SparkTestBase class MenasPlainCredentialsSuite extends AnyWordSpec with SparkTestBase { diff --git a/examples/src/test/scala/za/co/absa/enceladus/examples/interpreter/rules/custom/UppercaseCustomConformanceRuleSuite.scala b/examples/src/test/scala/za/co/absa/enceladus/examples/interpreter/rules/custom/UppercaseCustomConformanceRuleSuite.scala index 5c0ba1fbd..2046aaa38 100644 --- a/examples/src/test/scala/za/co/absa/enceladus/examples/interpreter/rules/custom/UppercaseCustomConformanceRuleSuite.scala +++ b/examples/src/test/scala/za/co/absa/enceladus/examples/interpreter/rules/custom/UppercaseCustomConformanceRuleSuite.scala @@ -24,7 +24,8 @@ import za.co.absa.enceladus.conformance.interpreter.{DynamicInterpreter, Feature import za.co.absa.enceladus.dao.MenasDAO import za.co.absa.enceladus.model.Dataset import za.co.absa.enceladus.utils.fs.HadoopFsUtils -import za.co.absa.enceladus.utils.testUtils.{HadoopFsTestBase, SparkTestBase} +import za.co.absa.enceladus.utils.testUtils.HadoopFsTestBase +import za.co.absa.spark.commons.test.SparkTestBase case class TestInputRow(id: Int, mandatoryString: String, nullableString: Option[String]) diff --git a/examples/src/test/scala/za/co/absa/enceladus/examples/interpreter/rules/custom/XPadCustomConformanceRuleSuite.scala b/examples/src/test/scala/za/co/absa/enceladus/examples/interpreter/rules/custom/XPadCustomConformanceRuleSuite.scala index 78daedc01..e2f82e6f2 100644 --- a/examples/src/test/scala/za/co/absa/enceladus/examples/interpreter/rules/custom/XPadCustomConformanceRuleSuite.scala +++ b/examples/src/test/scala/za/co/absa/enceladus/examples/interpreter/rules/custom/XPadCustomConformanceRuleSuite.scala @@ -26,8 +26,8 @@ import za.co.absa.enceladus.dao.MenasDAO import za.co.absa.enceladus.dao.auth.MenasKerberosCredentials import za.co.absa.enceladus.dao.rest.{MenasConnectionStringParser, RestDaoFactory} import za.co.absa.enceladus.model.Dataset -import za.co.absa.enceladus.utils.fs.HadoopFsUtils -import za.co.absa.enceladus.utils.testUtils.{HadoopFsTestBase, SparkTestBase} +import za.co.absa.enceladus.utils.testUtils.HadoopFsTestBase +import za.co.absa.spark.commons.test.SparkTestBase case class XPadTestInputRow(intField: Int, stringField: Option[String]) case class XPadTestOutputRow(intField: Int, stringField: Option[String], targetField: String) diff --git a/plugins-builtin/src/main/scala/za/co/absa/enceladus/plugins/builtin/errorsender/mq/KafkaErrorSenderPluginImpl.scala b/plugins-builtin/src/main/scala/za/co/absa/enceladus/plugins/builtin/errorsender/mq/KafkaErrorSenderPluginImpl.scala index 7a62a35ba..62300a0ce 100644 --- a/plugins-builtin/src/main/scala/za/co/absa/enceladus/plugins/builtin/errorsender/mq/KafkaErrorSenderPluginImpl.scala +++ b/plugins-builtin/src/main/scala/za/co/absa/enceladus/plugins/builtin/errorsender/mq/KafkaErrorSenderPluginImpl.scala @@ -23,12 +23,12 @@ import za.co.absa.enceladus.plugins.api.postprocessor.PostProcessor import za.co.absa.enceladus.plugins.builtin.common.mq.kafka.{KafkaConnectionParams, KafkaSecurityParams} import za.co.absa.enceladus.plugins.builtin.errorsender.DceError import za.co.absa.enceladus.plugins.builtin.errorsender.mq.KafkaErrorSenderPluginImpl.SingleErrorStardardized -import za.co.absa.enceladus.utils.schema.SchemaUtils import KafkaErrorSenderPluginImpl._ import za.co.absa.enceladus.plugins.builtin.errorsender.mq.kafka.KafkaErrorSenderPlugin import za.co.absa.enceladus.plugins.builtin.errorsender.params.ErrorSenderPluginParams import za.co.absa.enceladus.utils.error.ErrorMessage.ErrorCodes import za.co.absa.enceladus.utils.modules._ +import za.co.absa.spark.commons.implicits.StructTypeImplicits.StructTypeEnhancements import scala.util.{Failure, Success, Try} @@ -48,7 +48,7 @@ case class KafkaErrorSenderPluginImpl(connectionParams: KafkaConnectionParams, * @param paramsMap Additional key/value parameters provided by Enceladus. */ override def onDataReady(dataFrame: DataFrame, paramsMap: Map[String, String]): Unit = { - if (!SchemaUtils.fieldExists(ColumnNames.enceladusRecordId, dataFrame.schema)) { + if (!dataFrame.schema.fieldExists(ColumnNames.enceladusRecordId)) { throw new IllegalStateException( s"${this.getClass.getName} requires ${ColumnNames.enceladusRecordId} column to be present in the dataframe!" ) diff --git a/plugins-builtin/src/test/scala/za/co/absa/enceladus/plugins/builtin/errorsender/mq/KafkaErrorSenderPluginSuite.scala b/plugins-builtin/src/test/scala/za/co/absa/enceladus/plugins/builtin/errorsender/mq/KafkaErrorSenderPluginSuite.scala index b9f236541..d205f9eb6 100644 --- a/plugins-builtin/src/test/scala/za/co/absa/enceladus/plugins/builtin/errorsender/mq/KafkaErrorSenderPluginSuite.scala +++ b/plugins-builtin/src/test/scala/za/co/absa/enceladus/plugins/builtin/errorsender/mq/KafkaErrorSenderPluginSuite.scala @@ -32,7 +32,7 @@ import za.co.absa.enceladus.plugins.builtin.errorsender.mq.KafkaErrorSenderPlugi import za.co.absa.enceladus.plugins.builtin.errorsender.mq.kafka.KafkaErrorSenderPlugin import za.co.absa.enceladus.plugins.builtin.errorsender.params.ErrorSenderPluginParams import za.co.absa.enceladus.utils.modules.SourcePhase -import za.co.absa.enceladus.utils.testUtils.SparkTestBase +import za.co.absa.spark.commons.test.SparkTestBase class KafkaErrorSenderPluginSuite extends AnyFlatSpec with SparkTestBase with Matchers with BeforeAndAfterAll { diff --git a/pom.xml b/pom.xml index ca56bcd31..b89a90a0b 100644 --- a/pom.xml +++ b/pom.xml @@ -140,7 +140,8 @@ 1.3.0 3.1.1 - 0.0.27 + 1.0.0 + 0.2.0-SNAPSHOT 3.7.0 2.7.3 3.5.4 diff --git a/rest-api/src/main/scala/za/co/absa/enceladus/rest_api/utils/converters/SparkMenasSchemaConvertor.scala b/rest-api/src/main/scala/za/co/absa/enceladus/rest_api/utils/converters/SparkMenasSchemaConvertor.scala index 5716bf857..1567ab57c 100644 --- a/rest-api/src/main/scala/za/co/absa/enceladus/rest_api/utils/converters/SparkMenasSchemaConvertor.scala +++ b/rest-api/src/main/scala/za/co/absa/enceladus/rest_api/utils/converters/SparkMenasSchemaConvertor.scala @@ -23,8 +23,8 @@ import com.fasterxml.jackson.databind.ObjectMapper import org.springframework.stereotype.Component import java.io.ByteArrayOutputStream +import za.co.absa.spark.commons.utils.SchemaUtils import za.co.absa.enceladus.rest_api.models.rest.exceptions.SchemaParsingException -import za.co.absa.enceladus.utils.schema.SchemaUtils import scala.util.control.NonFatal diff --git a/rest-api/src/test/scala/za/co/absa/enceladus/rest_api/utils/converters/SparkMenasSchemaConvertorSuite.scala b/rest-api/src/test/scala/za/co/absa/enceladus/rest_api/utils/converters/SparkMenasSchemaConvertorSuite.scala index 3f71eb50a..eb0da7054 100644 --- a/rest-api/src/test/scala/za/co/absa/enceladus/rest_api/utils/converters/SparkMenasSchemaConvertorSuite.scala +++ b/rest-api/src/test/scala/za/co/absa/enceladus/rest_api/utils/converters/SparkMenasSchemaConvertorSuite.scala @@ -17,13 +17,13 @@ package za.co.absa.enceladus.rest_api.utils.converters import org.apache.spark.sql.types._ import za.co.absa.enceladus.model._ -import za.co.absa.enceladus.utils.testUtils.SparkTestBase import com.fasterxml.jackson.databind.ObjectMapper import com.fasterxml.jackson.module.scala.DefaultScalaModule import com.fasterxml.jackson.datatype.jsr310.JavaTimeModule import com.fasterxml.jackson.databind.SerializationFeature import org.scalatest.funsuite.AnyFunSuite import za.co.absa.enceladus.rest_api.models.rest.exceptions.SchemaParsingException +import za.co.absa.spark.commons.test.SparkTestBase class SparkMenasSchemaConvertorSuite extends AnyFunSuite with SparkTestBase { private val objectMapper = new ObjectMapper() diff --git a/spark-jobs/src/main/scala/za/co/absa/enceladus/common/CommonJobExecution.scala b/spark-jobs/src/main/scala/za/co/absa/enceladus/common/CommonJobExecution.scala index 4246aff1f..61dc9ccd0 100644 --- a/spark-jobs/src/main/scala/za/co/absa/enceladus/common/CommonJobExecution.scala +++ b/spark-jobs/src/main/scala/za/co/absa/enceladus/common/CommonJobExecution.scala @@ -326,11 +326,13 @@ trait CommonJobExecution extends ProjectMetadata { } protected def addInfoColumns(intoDf: DataFrame, reportDate: String, reportVersion: Int): DataFrame = { - import za.co.absa.enceladus.utils.implicits.DataFrameImplicits.DataFrameEnhancements + import za.co.absa.spark.commons.implicits.DataFrameImplicits.DataFrameEnhancements + val function: (DataFrame, String) => DataFrame = (df: DataFrame, _) => + df.withColumn("errCol", lit(Array.emptyIntArray)) intoDf - .withColumnIfDoesNotExist(InfoDateColumn, to_date(lit(reportDate), ReportDateFormat)) - .withColumnIfDoesNotExist(InfoDateColumnString, lit(reportDate)) - .withColumnIfDoesNotExist(InfoVersionColumn, lit(reportVersion)) + .withColumnIfDoesNotExist(function)(InfoDateColumn, to_date(lit(reportDate), ReportDateFormat)) + .withColumnIfDoesNotExist(function)(InfoDateColumnString, lit(reportDate)) + .withColumnIfDoesNotExist(function)(InfoVersionColumn, lit(reportVersion)) } private def getReportVersion[T](jobConfig: JobConfigParser[T], dataset: Dataset)(implicit hadoopConf: Configuration): Int = { diff --git a/spark-jobs/src/main/scala/za/co/absa/enceladus/common/performance/PerformanceMetricTools.scala b/spark-jobs/src/main/scala/za/co/absa/enceladus/common/performance/PerformanceMetricTools.scala index e7b0af467..878d2f2be 100644 --- a/spark-jobs/src/main/scala/za/co/absa/enceladus/common/performance/PerformanceMetricTools.scala +++ b/spark-jobs/src/main/scala/za/co/absa/enceladus/common/performance/PerformanceMetricTools.scala @@ -23,7 +23,7 @@ import za.co.absa.enceladus.utils.config.PathWithFs import za.co.absa.enceladus.utils.error.ErrorMessage import za.co.absa.enceladus.utils.general.ProjectMetadata import za.co.absa.enceladus.utils.fs.HadoopFsUtils -import za.co.absa.enceladus.utils.schema.SchemaUtils +import za.co.absa.spark.commons.implicits.StructTypeImplicits.StructTypeEnhancements object PerformanceMetricTools extends ProjectMetadata { @@ -189,7 +189,7 @@ object PerformanceMetricTools extends ProjectMetadata { * when running a Standardization or a Dynamic Conformance job. */ private def getNumberOfErrors(spark: SparkSession, outputPath: String): (Long, Long, Long) = { val df = spark.read.parquet(outputPath) - val errorCountColumn = SchemaUtils.getClosestUniqueName("enceladus_error_count", df.schema) + val errorCountColumn = df.schema.getClosestUniqueName("enceladus_error_count") val errCol = col(ErrorMessage.errorColumnName) val numRecordsFailed = df.filter(size(errCol) > 0).count val numRecordsSuccessful = df.filter(size(errCol) === 0).count diff --git a/spark-jobs/src/main/scala/za/co/absa/enceladus/conformance/ConformanceExecution.scala b/spark-jobs/src/main/scala/za/co/absa/enceladus/conformance/ConformanceExecution.scala index cc69b72a7..be5c448b1 100644 --- a/spark-jobs/src/main/scala/za/co/absa/enceladus/conformance/ConformanceExecution.scala +++ b/spark-jobs/src/main/scala/za/co/absa/enceladus/conformance/ConformanceExecution.scala @@ -37,7 +37,7 @@ import za.co.absa.enceladus.utils.config.{ConfigReader, PathWithFs} import za.co.absa.enceladus.utils.fs.HadoopFsUtils import za.co.absa.enceladus.utils.modules.SourcePhase import za.co.absa.enceladus.common.performance.PerformanceMetricTools -import za.co.absa.enceladus.utils.schema.SchemaUtils +import za.co.absa.spark.commons.implicits.StructTypeImplicits.StructTypeEnhancements import scala.util.control.NonFatal import scala.util.{Failure, Success, Try} @@ -118,7 +118,7 @@ trait ConformanceExecution extends CommonJobExecution { spark.setControlMeasurementError(sourceId.toString, e.getMessage, sw.toString) throw e case Success(conformedDF) => - if (SchemaUtils.fieldExists(Constants.EnceladusRecordId, conformedDF.schema)) { + if (conformedDF.schema.fieldExists(Constants.EnceladusRecordId)) { conformedDF // no new id regeneration } else { RecordIdGeneration.addRecordIdColumnByStrategy(conformedDF, Constants.EnceladusRecordId, recordIdGenerationStrategy) diff --git a/spark-jobs/src/main/scala/za/co/absa/enceladus/conformance/HyperConformance.scala b/spark-jobs/src/main/scala/za/co/absa/enceladus/conformance/HyperConformance.scala index 955c60758..410155bb4 100644 --- a/spark-jobs/src/main/scala/za/co/absa/enceladus/conformance/HyperConformance.scala +++ b/spark-jobs/src/main/scala/za/co/absa/enceladus/conformance/HyperConformance.scala @@ -67,7 +67,7 @@ class HyperConformance (menasBaseUrls: List[String], def applyConformanceTransformations(rawDf: DataFrame, conformance: Dataset) (implicit sparkSession: SparkSession, menasDAO: MenasDAO): DataFrame = { - import za.co.absa.enceladus.utils.implicits.DataFrameImplicits.DataFrameEnhancements + import za.co.absa.spark.commons.implicits.DataFrameImplicits.DataFrameEnhancements val schema: StructType = menasDAO.getSchema(conformance.schemaName, conformance.schemaVersion) val schemaFields = if (schema == null) List() else schema.fields.toList @@ -78,11 +78,13 @@ class HyperConformance (menasBaseUrls: List[String], // using HDFS implementation until HyperConformance is S3-ready implicit val hdfs: FileSystem = FileSystem.get(sparkSession.sparkContext.hadoopConfiguration) implicit val hdfsUtils: HadoopFsUtils = HadoopFsUtils.getOrCreate(hdfs) + val function: (DataFrame, String) => DataFrame = (df: DataFrame, _) => + df.withColumn("errCol", lit(Array.emptyIntArray)) val conformedDf = DynamicInterpreter().interpret(conformance, rawDf) - .withColumnIfDoesNotExist(InfoDateColumn, coalesce(infoDateColumn, current_date())) - .withColumnIfDoesNotExist(InfoDateColumnString, coalesce(date_format(infoDateColumn,"yyyy-MM-dd"), lit(""))) - .withColumnIfDoesNotExist(InfoVersionColumn, infoVersionColumn) + .withColumnIfDoesNotExist(function)(InfoDateColumn, coalesce(infoDateColumn, current_date())) + .withColumnIfDoesNotExist(function)(InfoDateColumnString, coalesce(date_format(infoDateColumn,"yyyy-MM-dd"), lit(""))) + .withColumnIfDoesNotExist(function)(InfoVersionColumn, infoVersionColumn) conformedDf } diff --git a/spark-jobs/src/main/scala/za/co/absa/enceladus/conformance/interpreter/DynamicInterpreter.scala b/spark-jobs/src/main/scala/za/co/absa/enceladus/conformance/interpreter/DynamicInterpreter.scala index e277198a1..039707b34 100644 --- a/spark-jobs/src/main/scala/za/co/absa/enceladus/conformance/interpreter/DynamicInterpreter.scala +++ b/spark-jobs/src/main/scala/za/co/absa/enceladus/conformance/interpreter/DynamicInterpreter.scala @@ -33,11 +33,11 @@ import za.co.absa.enceladus.model.conformanceRule.{ConformanceRule, _} import za.co.absa.enceladus.model.{Dataset => ConfDataset} import za.co.absa.enceladus.utils.config.PathWithFs import za.co.absa.enceladus.utils.error.ErrorMessage -import za.co.absa.enceladus.utils.explode.ExplosionContext import za.co.absa.enceladus.utils.fs.HadoopFsUtils import za.co.absa.enceladus.utils.general.Algorithms -import za.co.absa.enceladus.utils.schema.SchemaUtils import za.co.absa.enceladus.utils.udf.UDFLibrary +import za.co.absa.spark.commons.utils.explode.ExplosionContext +import za.co.absa.spark.commons.implicits.StructTypeImplicits.StructTypeEnhancementsArrays case class DynamicInterpreter(implicit inputFs: FileSystem) { private val log = LoggerFactory.getLogger(this.getClass) @@ -72,7 +72,7 @@ case class DynamicInterpreter(implicit inputFs: FileSystem) { private def findOriginalColumnsModificationRules(steps: List[ConformanceRule], schema: StructType): Seq[ConformanceRule] = { - steps.filter(rule => SchemaUtils.fieldExists(rule.outputColumn, schema)) + steps.filter(rule => schema.fieldExists(rule.outputColumn)) } /** @@ -182,7 +182,7 @@ case class DynamicInterpreter(implicit inputFs: FileSystem) { if (isGroupExplosionUsable(rules) && ictx.featureSwitches.experimentalMappingRuleEnabled) { // Inserting an explosion and a collapse between a group of mapping rules operating on a common array - val optArray = SchemaUtils.getDeepestArrayPath(schema, rules.head.outputColumn) + val optArray = schema.getDeepestArrayPath(rules.head.outputColumn) optArray match { case Some(arrayColumn) => new ArrayExplodeInterpreter(arrayColumn) :: (interpreters :+ new ArrayCollapseInterpreter()) @@ -402,7 +402,7 @@ case class DynamicInterpreter(implicit inputFs: FileSystem) { */ private def groupMappingRules(rules: List[ConformanceRule], schema: StructType): List[List[ConformanceRule]] = { Algorithms.stableGroupByOption[ConformanceRule, String](rules, { - case m: MappingConformanceRule => SchemaUtils.getDeepestArrayPath(schema, m.outputColumn) + case m: MappingConformanceRule => schema.getDeepestArrayPath(m.outputColumn) case _ => None }).map(_.toList).toList } diff --git a/spark-jobs/src/main/scala/za/co/absa/enceladus/conformance/interpreter/ExplosionState.scala b/spark-jobs/src/main/scala/za/co/absa/enceladus/conformance/interpreter/ExplosionState.scala index 6dbc69734..1c79f8fbf 100644 --- a/spark-jobs/src/main/scala/za/co/absa/enceladus/conformance/interpreter/ExplosionState.scala +++ b/spark-jobs/src/main/scala/za/co/absa/enceladus/conformance/interpreter/ExplosionState.scala @@ -15,7 +15,7 @@ package za.co.absa.enceladus.conformance.interpreter -import za.co.absa.enceladus.utils.explode.ExplosionContext +import za.co.absa.spark.commons.utils.explode.ExplosionContext /** * This class is used to encapsulate a state of exploded arrays during processing of dynamic conformance steps diff --git a/spark-jobs/src/main/scala/za/co/absa/enceladus/conformance/interpreter/OptimizerTimeTracker.scala b/spark-jobs/src/main/scala/za/co/absa/enceladus/conformance/interpreter/OptimizerTimeTracker.scala index 54e41702e..a4509b210 100644 --- a/spark-jobs/src/main/scala/za/co/absa/enceladus/conformance/interpreter/OptimizerTimeTracker.scala +++ b/spark-jobs/src/main/scala/za/co/absa/enceladus/conformance/interpreter/OptimizerTimeTracker.scala @@ -19,7 +19,7 @@ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.functions.{col, lit} import org.apache.spark.sql.{DataFrame, SparkSession} import org.slf4j.LoggerFactory -import za.co.absa.enceladus.utils.schema.SchemaUtils +import za.co.absa.spark.commons.implicits.StructTypeImplicits.StructTypeEnhancements class OptimizerTimeTracker(inputDf: DataFrame, isWorkaroundEnabled: Boolean)(implicit spark: SparkSession) { import spark.implicits._ @@ -31,7 +31,7 @@ class OptimizerTimeTracker(inputDf: DataFrame, isWorkaroundEnabled: Boolean)(imp private var baselineTimeMs = initialElapsedTimeBaselineMs private var lastExecutionPlanOptimizationTime = 0L - private val idField1 = SchemaUtils.getUniqueName("tmpId", Option(inputDf.schema)) + private val idField1 = inputDf.schema.getClosestUniqueName("tmpId") private val idField2 = s"${idField1}_2" private val dfWithId = inputDf.withColumn(idField1, lit(1)) private val dfJustId = Seq(1).toDF(idField2).cache() diff --git a/spark-jobs/src/main/scala/za/co/absa/enceladus/conformance/interpreter/rules/ArrayCollapseInterpreter.scala b/spark-jobs/src/main/scala/za/co/absa/enceladus/conformance/interpreter/rules/ArrayCollapseInterpreter.scala index d7797943f..c0e586c6a 100644 --- a/spark-jobs/src/main/scala/za/co/absa/enceladus/conformance/interpreter/rules/ArrayCollapseInterpreter.scala +++ b/spark-jobs/src/main/scala/za/co/absa/enceladus/conformance/interpreter/rules/ArrayCollapseInterpreter.scala @@ -20,7 +20,8 @@ import za.co.absa.enceladus.conformance.interpreter.{ExplosionState, Interpreter import za.co.absa.enceladus.dao.MenasDAO import za.co.absa.enceladus.model.conformanceRule.ConformanceRule import za.co.absa.enceladus.utils.error.ErrorMessage -import za.co.absa.enceladus.utils.explode.{ExplodeTools, ExplosionContext} +import za.co.absa.spark.commons.utils.ExplodeTools +import za.co.absa.spark.commons.utils.explode.ExplosionContext /** * This conformance interpreter collapses previously exploded array(s) back. diff --git a/spark-jobs/src/main/scala/za/co/absa/enceladus/conformance/interpreter/rules/ArrayExplodeInterpreter.scala b/spark-jobs/src/main/scala/za/co/absa/enceladus/conformance/interpreter/rules/ArrayExplodeInterpreter.scala index f57984bdb..c864dcb77 100644 --- a/spark-jobs/src/main/scala/za/co/absa/enceladus/conformance/interpreter/rules/ArrayExplodeInterpreter.scala +++ b/spark-jobs/src/main/scala/za/co/absa/enceladus/conformance/interpreter/rules/ArrayExplodeInterpreter.scala @@ -19,7 +19,7 @@ import org.apache.spark.sql.{Dataset, Row, SparkSession} import za.co.absa.enceladus.conformance.interpreter.{ExplosionState, InterpreterContextArgs} import za.co.absa.enceladus.dao.MenasDAO import za.co.absa.enceladus.model.conformanceRule.ConformanceRule -import za.co.absa.enceladus.utils.explode.ExplodeTools +import za.co.absa.spark.commons.utils.ExplodeTools /** * This conformance interpreter explodes a given array. diff --git a/spark-jobs/src/main/scala/za/co/absa/enceladus/conformance/interpreter/rules/CastingRuleInterpreter.scala b/spark-jobs/src/main/scala/za/co/absa/enceladus/conformance/interpreter/rules/CastingRuleInterpreter.scala index 3767f5bc7..00bd962fe 100644 --- a/spark-jobs/src/main/scala/za/co/absa/enceladus/conformance/interpreter/rules/CastingRuleInterpreter.scala +++ b/spark-jobs/src/main/scala/za/co/absa/enceladus/conformance/interpreter/rules/CastingRuleInterpreter.scala @@ -23,8 +23,9 @@ import za.co.absa.spark.hats.Extensions._ import za.co.absa.enceladus.conformance.interpreter.{ExplosionState, InterpreterContextArgs, RuleValidators} import za.co.absa.enceladus.dao.MenasDAO import za.co.absa.enceladus.model.conformanceRule.{CastingConformanceRule, ConformanceRule} -import za.co.absa.enceladus.utils.schema.SchemaUtils import za.co.absa.enceladus.utils.udf.UDFNames +import za.co.absa.spark.commons.implicits.DataTypeImplicits.DataTypeEnhancements +import za.co.absa.spark.commons.implicits.StructTypeImplicits.StructTypeEnhancements import za.co.absa.spark.hats.transformations.NestedArrayTransformations case class CastingRuleInterpreter(rule: CastingConformanceRule) extends RuleInterpreter { @@ -41,13 +42,13 @@ case class CastingRuleInterpreter(rule: CastingConformanceRule) extends RuleInte RuleValidators.validateOutputField(progArgs.datasetName, ruleName, df.schema, rule.outputColumn) RuleValidators.validateSameParent(progArgs.datasetName, ruleName, rule.inputColumn, rule.outputColumn) - SchemaUtils.getFieldType(rule.inputColumn, df.schema) + df.schema.getFieldType(rule.inputColumn) .foreach(dt => RuleValidators.validateTypeCompatibility(ruleName, rule.inputColumn, dt, rule.outputDataType)) - val sourceDataType = SchemaUtils.getFieldType(rule.inputColumn, df.schema).get + val sourceDataType = df.schema.getFieldType(rule.inputColumn).get val targetDataType = CatalystSqlParser.parseDataType(rule.outputDataType) - if (SchemaUtils.isCastAlwaysSucceeds(sourceDataType, targetDataType)) { + if (sourceDataType.doesCastAlwaysSucceed(targetDataType)) { // Casting to string does not generate errors df.nestedMapColumn(rule.inputColumn, rule.outputColumn, c => c.cast(rule.outputDataType) diff --git a/spark-jobs/src/main/scala/za/co/absa/enceladus/conformance/interpreter/rules/DropRuleInterpreter.scala b/spark-jobs/src/main/scala/za/co/absa/enceladus/conformance/interpreter/rules/DropRuleInterpreter.scala index 7c9072485..537b9df00 100644 --- a/spark-jobs/src/main/scala/za/co/absa/enceladus/conformance/interpreter/rules/DropRuleInterpreter.scala +++ b/spark-jobs/src/main/scala/za/co/absa/enceladus/conformance/interpreter/rules/DropRuleInterpreter.scala @@ -20,7 +20,7 @@ import za.co.absa.spark.hats.Extensions._ import za.co.absa.enceladus.conformance.interpreter.{ExplosionState, InterpreterContextArgs} import za.co.absa.enceladus.dao.MenasDAO import za.co.absa.enceladus.model.conformanceRule.{ConformanceRule, DropConformanceRule} -import za.co.absa.enceladus.utils.schema.SchemaUtils +import za.co.absa.spark.commons.implicits.StructTypeImplicits.StructTypeEnhancements case class DropRuleInterpreter(rule: DropConformanceRule) extends RuleInterpreter { @@ -29,7 +29,7 @@ case class DropRuleInterpreter(rule: DropConformanceRule) extends RuleInterprete def conform(df: Dataset[Row]) (implicit spark: SparkSession, explosionState: ExplosionState, dao: MenasDAO, progArgs: InterpreterContextArgs): Dataset[Row] = { - if (SchemaUtils.fieldExists(rule.outputColumn, df.schema)) { + if (df.schema.fieldExists(rule.outputColumn)) { if (rule.outputColumn.contains('.')) { conformNestedField(df) } else { diff --git a/spark-jobs/src/main/scala/za/co/absa/enceladus/conformance/interpreter/rules/FillNullsRuleInterpreter.scala b/spark-jobs/src/main/scala/za/co/absa/enceladus/conformance/interpreter/rules/FillNullsRuleInterpreter.scala index 7d6b69664..e081b1760 100644 --- a/spark-jobs/src/main/scala/za/co/absa/enceladus/conformance/interpreter/rules/FillNullsRuleInterpreter.scala +++ b/spark-jobs/src/main/scala/za/co/absa/enceladus/conformance/interpreter/rules/FillNullsRuleInterpreter.scala @@ -22,7 +22,7 @@ import org.apache.spark.sql.{Column, Dataset, Row, SparkSession} import za.co.absa.enceladus.conformance.interpreter.{ExplosionState, RuleValidators} import za.co.absa.enceladus.dao.MenasDAO import za.co.absa.enceladus.model.conformanceRule.{ConformanceRule, FillNullsConformanceRule} -import za.co.absa.enceladus.utils.schema.SchemaUtils +import za.co.absa.spark.commons.implicits.StructTypeImplicits.StructTypeEnhancements import za.co.absa.spark.hats.Extensions._ import scala.util.{Failure, Success} @@ -45,7 +45,7 @@ case class FillNullsRuleInterpreter(rule: FillNullsConformanceRule) extends Rule rule.outputColumn ) - val dataType: DataType = SchemaUtils.getFieldType(rule.inputColumn, df.schema).get + val dataType: DataType = df.schema.getFieldType(rule.inputColumn).get val default: Column = simpleLiteralCast(rule.value, dataType) match { case Success(value) => value case Failure(exception) => diff --git a/spark-jobs/src/main/scala/za/co/absa/enceladus/conformance/interpreter/rules/NegationRuleInterpreter.scala b/spark-jobs/src/main/scala/za/co/absa/enceladus/conformance/interpreter/rules/NegationRuleInterpreter.scala index eab15c6d9..0b364fd89 100644 --- a/spark-jobs/src/main/scala/za/co/absa/enceladus/conformance/interpreter/rules/NegationRuleInterpreter.scala +++ b/spark-jobs/src/main/scala/za/co/absa/enceladus/conformance/interpreter/rules/NegationRuleInterpreter.scala @@ -22,10 +22,10 @@ import za.co.absa.spark.hats.Extensions._ import za.co.absa.enceladus.conformance.interpreter.{ExplosionState, InterpreterContextArgs, RuleValidators} import za.co.absa.enceladus.dao.MenasDAO import za.co.absa.enceladus.model.conformanceRule.{ConformanceRule, NegationConformanceRule} -import za.co.absa.enceladus.utils.schema.SchemaUtils import za.co.absa.enceladus.utils.types.GlobalDefaults import za.co.absa.enceladus.utils.udf.UDFNames import za.co.absa.enceladus.utils.validation.SchemaPathValidator +import za.co.absa.spark.commons.implicits.StructTypeImplicits.StructTypeEnhancements import za.co.absa.spark.hats.transformations.NestedArrayTransformations case class NegationRuleInterpreter(rule: NegationConformanceRule) extends RuleInterpreter { @@ -37,7 +37,7 @@ case class NegationRuleInterpreter(rule: NegationConformanceRule) extends RuleIn progArgs: InterpreterContextArgs): Dataset[Row] = { NegationRuleInterpreter.validateInputField(progArgs.datasetName, df.schema, rule.inputColumn) - val field = SchemaUtils.getField(rule.inputColumn, df.schema).get + val field = df.schema.getField(rule.inputColumn).get val negationErrUdfCall = callUDF(UDFNames.confNegErr, lit(rule.outputColumn), col(rule.inputColumn)) val errCol = "errCol" diff --git a/spark-jobs/src/main/scala/za/co/absa/enceladus/conformance/interpreter/rules/mapping/CommonMappingRuleInterpreter.scala b/spark-jobs/src/main/scala/za/co/absa/enceladus/conformance/interpreter/rules/mapping/CommonMappingRuleInterpreter.scala index 3b6c19594..a07811ac3 100644 --- a/spark-jobs/src/main/scala/za/co/absa/enceladus/conformance/interpreter/rules/mapping/CommonMappingRuleInterpreter.scala +++ b/spark-jobs/src/main/scala/za/co/absa/enceladus/conformance/interpreter/rules/mapping/CommonMappingRuleInterpreter.scala @@ -19,6 +19,7 @@ import org.apache.spark.sql.functions.{col, lit} import org.apache.spark.sql.types.{StructField, StructType} import org.apache.spark.sql.{Column, DataFrame, Dataset, Row, SparkSession} import org.slf4j.Logger +import za.co.absa.spark.commons.utils.SchemaUtils import za.co.absa.enceladus.conformance.datasource.DataSource import za.co.absa.enceladus.conformance.interpreter.{ExplosionState, InterpreterContextArgs} import za.co.absa.enceladus.dao.MenasDAO @@ -27,8 +28,8 @@ import za.co.absa.enceladus.model.conformanceRule.MappingConformanceRule import za.co.absa.enceladus.model.dataFrameFilter.DataFrameFilter import za.co.absa.enceladus.conformance.interpreter.rules.ValidationException import za.co.absa.enceladus.utils.error.Mapping -import za.co.absa.enceladus.utils.schema.SchemaUtils import za.co.absa.enceladus.utils.validation.ExpressionValidator +import za.co.absa.spark.commons.implicits.StructTypeImplicits.StructTypeEnhancements import scala.util.Try import scala.util.control.NonFatal @@ -40,7 +41,7 @@ trait CommonMappingRuleInterpreter { protected def outputColumnNames(): String = rule.allOutputColumns().mkString(", ") - protected def getOutputsStructColumnName(df: DataFrame): String = SchemaUtils.getClosestUniqueName("outputs", df.schema) + protected def getOutputsStructColumnName(df: DataFrame): String = df.schema.getClosestUniqueName("outputs") protected val mappings: Seq[Mapping] = rule.attributeMappings.map { case (mappingTableField, dataframeField) => Mapping(mappingTableField, dataframeField) diff --git a/spark-jobs/src/main/scala/za/co/absa/enceladus/conformance/interpreter/rules/mapping/MappingRuleInterpreter.scala b/spark-jobs/src/main/scala/za/co/absa/enceladus/conformance/interpreter/rules/mapping/MappingRuleInterpreter.scala index 1621cffc5..03c857c59 100644 --- a/spark-jobs/src/main/scala/za/co/absa/enceladus/conformance/interpreter/rules/mapping/MappingRuleInterpreter.scala +++ b/spark-jobs/src/main/scala/za/co/absa/enceladus/conformance/interpreter/rules/mapping/MappingRuleInterpreter.scala @@ -25,10 +25,10 @@ import za.co.absa.enceladus.dao.MenasDAO import za.co.absa.enceladus.model.conformanceRule.{ConformanceRule, MappingConformanceRule} import za.co.absa.enceladus.model.{Dataset => ConfDataset} import za.co.absa.enceladus.utils.error._ -import za.co.absa.enceladus.utils.schema.SchemaUtils import za.co.absa.enceladus.utils.transformations.ArrayTransformations import za.co.absa.enceladus.utils.transformations.ArrayTransformations.arrCol import za.co.absa.enceladus.utils.udf.UDFNames +import za.co.absa.spark.commons.implicits.StructTypeImplicits.{StructTypeEnhancementsArrays} case class MappingRuleInterpreter(rule: MappingConformanceRule, conformance: ConfDataset) extends RuleInterpreter with JoinMappingRuleInterpreter { @@ -74,7 +74,7 @@ case class MappingRuleInterpreter(rule: MappingConformanceRule, conformance: Con } } val errNested = errorsDf.groupBy(idField).agg(collect_list(col(ErrorMessage.errorColumnName)) as ErrorMessage.errorColumnName) - val errNestedSchema = SchemaUtils.getFieldType(ErrorMessage.errorColumnName, errNested.schema).get.asInstanceOf[ArrayType] + val errNestedSchema = errNested.schema.getFieldType(ErrorMessage.errorColumnName).get.asInstanceOf[ArrayType] // errNested will duplicate error values if the previous rule has any errCol // and in the current rule the joining key is an array so the error values will duplicate as the size of array : @@ -95,7 +95,7 @@ case class MappingRuleInterpreter(rule: MappingConformanceRule, conformance: Con private def inclErrorNullArr(mappings: Seq[Mapping], schema: StructType) = { val paths = mappings.flatMap { mapping => - SchemaUtils.getAllArraysInPath(mapping.mappedDatasetColumn, schema) + schema.getAllArraysInPath(mapping.mappedDatasetColumn) } MappingRuleInterpreter.includeErrorsCondition(paths, schema) } @@ -117,7 +117,7 @@ object MappingRuleInterpreter { .map(x => (x, ArrayTransformations.arraySizeCols(x))) .foldLeft(lit(true)) { case (acc: Column, (origPath, sizePath)) => - val nullable = lit(SchemaUtils.getFieldNullability(origPath, schema).get) + val nullable = lit(schema.getFieldNullability(origPath).get) val nll = col(sizePath) === lit(-1) val empty = col(sizePath) === lit(0) diff --git a/spark-jobs/src/main/scala/za/co/absa/enceladus/conformance/interpreter/rules/mapping/MappingRuleInterpreterBroadcast.scala b/spark-jobs/src/main/scala/za/co/absa/enceladus/conformance/interpreter/rules/mapping/MappingRuleInterpreterBroadcast.scala index 0267f396a..798ce86b2 100644 --- a/spark-jobs/src/main/scala/za/co/absa/enceladus/conformance/interpreter/rules/mapping/MappingRuleInterpreterBroadcast.scala +++ b/spark-jobs/src/main/scala/za/co/absa/enceladus/conformance/interpreter/rules/mapping/MappingRuleInterpreterBroadcast.scala @@ -16,6 +16,7 @@ package za.co.absa.enceladus.conformance.interpreter.rules.mapping import org.apache.spark.sql.{DataFrame, SparkSession} +import za.co.absa.spark.commons.utils.SchemaUtils import za.co.absa.enceladus.conformance.interpreter.rules.RuleInterpreter import za.co.absa.enceladus.conformance.interpreter.{ExplosionState, InterpreterContextArgs} import za.co.absa.enceladus.dao.MenasDAO @@ -23,7 +24,6 @@ import za.co.absa.enceladus.model.conformanceRule.{ConformanceRule, MappingConfo import za.co.absa.enceladus.model.{Dataset => ConfDataset} import za.co.absa.enceladus.utils.broadcast.{BroadcastUtils, LocalMappingTable} import za.co.absa.enceladus.utils.error.ErrorMessage -import za.co.absa.enceladus.utils.schema.SchemaUtils import za.co.absa.spark.hats.transformations.NestedArrayTransformations import za.co.absa.spark.hats.transformations.NestedArrayTransformations.GetFieldFunction diff --git a/spark-jobs/src/main/scala/za/co/absa/enceladus/conformance/interpreter/rules/mapping/MappingRuleInterpreterGroupExplode.scala b/spark-jobs/src/main/scala/za/co/absa/enceladus/conformance/interpreter/rules/mapping/MappingRuleInterpreterGroupExplode.scala index ec872a21e..a375cb64d 100644 --- a/spark-jobs/src/main/scala/za/co/absa/enceladus/conformance/interpreter/rules/mapping/MappingRuleInterpreterGroupExplode.scala +++ b/spark-jobs/src/main/scala/za/co/absa/enceladus/conformance/interpreter/rules/mapping/MappingRuleInterpreterGroupExplode.scala @@ -24,10 +24,10 @@ import za.co.absa.enceladus.dao.MenasDAO import za.co.absa.enceladus.model.conformanceRule.{ConformanceRule, MappingConformanceRule} import za.co.absa.enceladus.model.{Dataset => ConfDataset} import za.co.absa.enceladus.utils.error._ -import za.co.absa.enceladus.utils.explode.{ExplodeTools, ExplosionContext} -import za.co.absa.enceladus.utils.schema.SchemaUtils import za.co.absa.enceladus.utils.transformations.ArrayTransformations.arrCol import za.co.absa.enceladus.utils.udf.UDFNames +import za.co.absa.spark.commons.utils.explode.ExplosionContext +import za.co.absa.spark.commons.utils.{ExplodeTools, SchemaUtils} import za.co.absa.spark.hats.transformations.NestedArrayTransformations case class MappingRuleInterpreterGroupExplode(rule: MappingConformanceRule, diff --git a/spark-jobs/src/main/scala/za/co/absa/enceladus/conformance/streaming/InfoDateFactory.scala b/spark-jobs/src/main/scala/za/co/absa/enceladus/conformance/streaming/InfoDateFactory.scala index 99a314668..a668cbca0 100644 --- a/spark-jobs/src/main/scala/za/co/absa/enceladus/conformance/streaming/InfoDateFactory.scala +++ b/spark-jobs/src/main/scala/za/co/absa/enceladus/conformance/streaming/InfoDateFactory.scala @@ -22,7 +22,7 @@ import org.apache.spark.sql.{Column, DataFrame} import org.slf4j.{Logger, LoggerFactory} import za.co.absa.enceladus.common.Constants.ReportDateFormat import za.co.absa.enceladus.conformance.datasource.PartitioningUtils -import za.co.absa.enceladus.utils.schema.SchemaUtils +import za.co.absa.spark.commons.implicits.StructTypeImplicits.StructTypeEnhancements /** * Info date factory allows to create an expression for the information date based on the strategy used. @@ -46,7 +46,7 @@ object InfoDateFactory { private class InfoDateFromColumnFactory(columnName: String, pattern: String) extends InfoDateFactory { override def getInfoDateColumn(df: DataFrame): Column = { - val dt = SchemaUtils.getFieldType(columnName, df.schema) + val dt = df.schema.getFieldType(columnName) dt match { case Some(TimestampType) => col(columnName).cast(DateType) diff --git a/spark-jobs/src/main/scala/za/co/absa/enceladus/standardization/StandardizationExecution.scala b/spark-jobs/src/main/scala/za/co/absa/enceladus/standardization/StandardizationExecution.scala index 4dfb213d9..9e7615064 100644 --- a/spark-jobs/src/main/scala/za/co/absa/enceladus/standardization/StandardizationExecution.scala +++ b/spark-jobs/src/main/scala/za/co/absa/enceladus/standardization/StandardizationExecution.scala @@ -17,11 +17,13 @@ package za.co.absa.enceladus.standardization import java.io.{PrintWriter, StringWriter} import java.util.UUID + import org.apache.hadoop.conf.Configuration import org.apache.spark.sql.types.{StructField, StructType} import org.apache.spark.sql.{Column, DataFrame, SparkSession} import za.co.absa.atum.AtumImplicits._ import za.co.absa.atum.core.Atum +import za.co.absa.enceladus.utils.schema.SchemaUtils import za.co.absa.enceladus.common.RecordIdGeneration.getRecordIdGenerationStrategyFromConfig import za.co.absa.enceladus.common.config.{CommonConfConstants, JobConfigParser, PathConfig} import za.co.absa.enceladus.common.plugin.menas.MenasPlugin @@ -36,7 +38,7 @@ import za.co.absa.enceladus.utils.config.{ConfigReader, PathWithFs} import za.co.absa.enceladus.utils.fs.{DistributedFsUtils, HadoopFsUtils} import za.co.absa.enceladus.utils.modules.SourcePhase import za.co.absa.enceladus.common.performance.PerformanceMetricTools -import za.co.absa.enceladus.utils.schema.{MetadataKeys, SchemaUtils, SparkUtils} +import za.co.absa.enceladus.utils.schema.{MetadataKeys, SparkUtils} import za.co.absa.enceladus.utils.types.Defaults import za.co.absa.enceladus.utils.udf.UDFLibrary import za.co.absa.enceladus.utils.validation.ValidationException diff --git a/spark-jobs/src/main/scala/za/co/absa/enceladus/standardization/interpreter/StandardizationInterpreter.scala b/spark-jobs/src/main/scala/za/co/absa/enceladus/standardization/interpreter/StandardizationInterpreter.scala index 090be5f4b..668b4cdd0 100644 --- a/spark-jobs/src/main/scala/za/co/absa/enceladus/standardization/interpreter/StandardizationInterpreter.scala +++ b/spark-jobs/src/main/scala/za/co/absa/enceladus/standardization/interpreter/StandardizationInterpreter.scala @@ -24,11 +24,12 @@ import za.co.absa.enceladus.common.RecordIdGeneration._ import za.co.absa.enceladus.standardization.interpreter.dataTypes._ import za.co.absa.enceladus.standardization.interpreter.stages.{SchemaChecker, TypeParser} import za.co.absa.enceladus.utils.error.ErrorMessage -import za.co.absa.enceladus.utils.schema.{SchemaUtils, SparkUtils} +import za.co.absa.enceladus.utils.schema.SparkUtils import za.co.absa.enceladus.utils.transformations.ArrayTransformations import za.co.absa.enceladus.utils.types.Defaults import za.co.absa.enceladus.utils.udf.{UDFLibrary, UDFNames} import za.co.absa.enceladus.utils.validation.ValidationException +import za.co.absa.spark.commons.implicits.StructTypeImplicits.StructTypeEnhancements /** * Object representing set of tools for performing the actual standardization @@ -59,7 +60,7 @@ object StandardizationInterpreter { logger.info(s"Step 3: Clean the final error column") val cleanedStd = cleanTheFinalErrorColumn(std) - val idedStd = if (SchemaUtils.fieldExists(Constants.EnceladusRecordId, cleanedStd.schema)) { + val idedStd = if (cleanedStd.schema.fieldExists(Constants.EnceladusRecordId)) { cleanedStd // no new id regeneration } else { RecordIdGeneration.addRecordIdColumnByStrategy(cleanedStd, Constants.EnceladusRecordId, recordIdGenerationStrategy) @@ -107,7 +108,7 @@ object StandardizationInterpreter { private def gatherRowErrors(origSchema: StructType)(implicit spark: SparkSession): List[Column] = { val corruptRecordColumn = spark.conf.get(SparkUtils.ColumnNameOfCorruptRecordConf) - SchemaUtils.getField(corruptRecordColumn, origSchema).map {_ => + origSchema.getField(corruptRecordColumn).map {_ => val column = col(corruptRecordColumn) when(column.isNotNull, // input row was not per expected schema array(callUDF(UDFNames.stdSchemaErr, column.cast(StringType)) //column should be StringType but better to be sure diff --git a/spark-jobs/src/main/scala/za/co/absa/enceladus/standardization/interpreter/stages/PlainSchemaGenerator.scala b/spark-jobs/src/main/scala/za/co/absa/enceladus/standardization/interpreter/stages/PlainSchemaGenerator.scala index 8b46e4454..3bb1b5089 100644 --- a/spark-jobs/src/main/scala/za/co/absa/enceladus/standardization/interpreter/stages/PlainSchemaGenerator.scala +++ b/spark-jobs/src/main/scala/za/co/absa/enceladus/standardization/interpreter/stages/PlainSchemaGenerator.scala @@ -17,6 +17,8 @@ package za.co.absa.enceladus.standardization.interpreter.stages import org.apache.spark.sql.types._ import za.co.absa.enceladus.utils.schema.MetadataKeys +import za.co.absa.enceladus.utils.schema.SchemaUtils.FieldWithSource +import za.co.absa.spark.commons.implicits.StructFieldImplicits.StructFieldMetadataEnhancements /** * This component is used in the standardization job. We've got a strongly typed (target) schema. When reading the data however, we do not want spark to apply casts @@ -26,10 +28,9 @@ import za.co.absa.enceladus.utils.schema.MetadataKeys object PlainSchemaGenerator { private def structTypeFieldsConversion(fields: Array[StructField]): Array[StructField] = { - import za.co.absa.enceladus.utils.implicits.StructFieldImplicits.StructFieldEnhancements fields.map { field => // If the meta data value sourcecolumn is set override the field name - val fieldName = field.getMetadataString(MetadataKeys.SourceColumn).getOrElse(field.name) + val fieldName = field.structField.metadata.getOptString(MetadataKeys.SourceColumn).getOrElse(field.name) val dataType = inputSchemaAsStringTypes(field.dataType) StructField(fieldName, dataType, nullable = true, field.metadata) } diff --git a/spark-jobs/src/main/scala/za/co/absa/enceladus/standardization/interpreter/stages/TypeParser.scala b/spark-jobs/src/main/scala/za/co/absa/enceladus/standardization/interpreter/stages/TypeParser.scala index f30152a13..d97fbf16a 100644 --- a/spark-jobs/src/main/scala/za/co/absa/enceladus/standardization/interpreter/stages/TypeParser.scala +++ b/spark-jobs/src/main/scala/za/co/absa/enceladus/standardization/interpreter/stages/TypeParser.scala @@ -27,13 +27,16 @@ import org.apache.spark.sql.types._ import org.slf4j.{Logger, LoggerFactory} import za.co.absa.enceladus.standardization.interpreter.dataTypes.ParseOutput import za.co.absa.enceladus.utils.error.ErrorMessage +import za.co.absa.enceladus.utils.schema.MetadataValues import za.co.absa.enceladus.utils.schema.SchemaUtils.FieldWithSource -import za.co.absa.enceladus.utils.schema.{MetadataValues, SchemaUtils} import za.co.absa.enceladus.utils.time.DateTimePattern import za.co.absa.enceladus.utils.typeClasses.{DoubleLike, LongLike} import za.co.absa.enceladus.utils.types.TypedStructField._ import za.co.absa.enceladus.utils.types.{Defaults, TypedStructField} import za.co.absa.enceladus.utils.udf.{UDFBuilder, UDFLibrary, UDFNames} +import za.co.absa.spark.commons.implicits.ColumnImplicits.ColumnEnhancements +import za.co.absa.spark.commons.implicits.StructTypeImplicits.StructTypeEnhancements +import za.co.absa.spark.commons.utils.SchemaUtils import za.co.absa.spark.hofs.transform import scala.reflect.runtime.universe._ @@ -115,7 +118,7 @@ sealed trait TypeParser[T] { } object TypeParser { - import za.co.absa.enceladus.utils.implicits.ColumnImplicits.ColumnEnhancements + import za.co.absa.enceladus.utils.implicits.EnceladusColumnImplicits.EnceladusColumnEnhancements private val decimalType = DecimalType(30,9) // scalastyle:ignore magic.number private implicit val logger: Logger = LoggerFactory.getLogger(this.getClass) @@ -131,7 +134,7 @@ object TypeParser { (implicit udfLib: UDFLibrary, defaults: Defaults): ParseOutput = { // udfLib implicit is present for error column UDF implementation val sourceName = SchemaUtils.appendPath(path, field.sourceName) - val origField = SchemaUtils.getField(sourceName, origSchema) + val origField = origSchema.getField(sourceName) val origFieldType = origField.map(_.dataType).getOrElse(NullType) val column = origField.fold(nullColumn)(_ => col(sourceName)) TypeParser(field, path, column, origFieldType, failOnInputNotPerSchema).standardize() @@ -194,7 +197,7 @@ object TypeParser { logger.info(s"Creating standardization plan for Array $inputFullPathName") val origArrayType = origType.asInstanceOf[ArrayType] // this should never throw an exception because of `checkSetupForFailure` val arrayField = StructField(fieldInputName, fieldType.elementType, fieldType.containsNull, field.structField.metadata) - val lambdaVariableName = s"${SchemaUtils.unpath(inputFullPathName)}_${Random.nextLong().abs}" + val lambdaVariableName = s"${za.co.absa.enceladus.utils.schema.SchemaUtils.unpath(inputFullPathName)}_${Random.nextLong().abs}" val lambda = (forCol: Column) => TypeParser(arrayField, path, forCol, origArrayType.elementType, failOnInputNotPerSchema, isArrayElement = true) .standardize() diff --git a/spark-jobs/src/test/scala/za/co/absa/enceladus/common/CommonExecutionSuite.scala b/spark-jobs/src/test/scala/za/co/absa/enceladus/common/CommonExecutionSuite.scala index 5f65d0112..91d146899 100644 --- a/spark-jobs/src/test/scala/za/co/absa/enceladus/common/CommonExecutionSuite.scala +++ b/spark-jobs/src/test/scala/za/co/absa/enceladus/common/CommonExecutionSuite.scala @@ -25,8 +25,8 @@ import za.co.absa.enceladus.common.config.PathConfig import za.co.absa.enceladus.dao.MenasDAO import za.co.absa.enceladus.model.{Dataset, Validation} import za.co.absa.enceladus.standardization.config.StandardizationConfig -import za.co.absa.enceladus.utils.testUtils.SparkTestBase import za.co.absa.enceladus.utils.validation.ValidationLevel +import za.co.absa.spark.commons.test.SparkTestBase class CommonExecutionSuite extends AnyFlatSpec with Matchers with SparkTestBase with MockitoSugar { diff --git a/spark-jobs/src/test/scala/za/co/absa/enceladus/common/RecordIdGenerationSuite.scala b/spark-jobs/src/test/scala/za/co/absa/enceladus/common/RecordIdGenerationSuite.scala index 49791c522..c19cab05c 100644 --- a/spark-jobs/src/test/scala/za/co/absa/enceladus/common/RecordIdGenerationSuite.scala +++ b/spark-jobs/src/test/scala/za/co/absa/enceladus/common/RecordIdGenerationSuite.scala @@ -19,11 +19,11 @@ import java.util.UUID import com.typesafe.config.{Config, ConfigException, ConfigFactory, ConfigValueFactory} import za.co.absa.enceladus.common.RecordIdGenerationSuite.{SomeData, SomeDataWithId} -import za.co.absa.enceladus.utils.testUtils.SparkTestBase import RecordIdGeneration._ import IdType._ import org.scalatest.flatspec.AnyFlatSpec import org.scalatest.matchers.should.Matchers +import za.co.absa.spark.commons.test.SparkTestBase class RecordIdGenerationSuite extends AnyFlatSpec with Matchers with SparkTestBase { import spark.implicits._ diff --git a/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/config/ConformanceParserSuite.scala b/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/config/ConformanceParserSuite.scala index a3a925dba..ba4e044a7 100644 --- a/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/config/ConformanceParserSuite.scala +++ b/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/config/ConformanceParserSuite.scala @@ -21,7 +21,7 @@ import org.scalatest.funsuite.AnyFunSuite import za.co.absa.enceladus.conformance.ConformanceExecution import za.co.absa.enceladus.dao.auth.{MenasKerberosCredentials, MenasPlainCredentials} import za.co.absa.enceladus.model.Dataset -import za.co.absa.enceladus.utils.testUtils.SparkTestBase +import za.co.absa.spark.commons.test.SparkTestBase class ConformanceParserSuite extends AnyFunSuite with SparkTestBase { diff --git a/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/datasource/DatasourceSuite.scala b/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/datasource/DatasourceSuite.scala index f40237595..bf63346a0 100644 --- a/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/datasource/DatasourceSuite.scala +++ b/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/datasource/DatasourceSuite.scala @@ -19,7 +19,7 @@ import org.apache.spark.sql.types.IntegerType import org.scalatest.funsuite.AnyFunSuite import za.co.absa.enceladus.conformance.samples.EmployeeConformance import za.co.absa.enceladus.model.dataFrameFilter._ -import za.co.absa.enceladus.utils.testUtils.SparkTestBase +import za.co.absa.spark.commons.test.SparkTestBase class DatasourceSuite extends AnyFunSuite with SparkTestBase { diff --git a/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/ArrayConformanceSuite.scala b/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/ArrayConformanceSuite.scala index 209c39e26..18e1b4353 100644 --- a/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/ArrayConformanceSuite.scala +++ b/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/ArrayConformanceSuite.scala @@ -23,7 +23,8 @@ import za.co.absa.enceladus.conformance.config.ConformanceConfig import za.co.absa.enceladus.conformance.datasource.DataSource import za.co.absa.enceladus.conformance.samples._ import za.co.absa.enceladus.dao.MenasDAO -import za.co.absa.enceladus.utils.testUtils.{HadoopFsTestBase, SparkTestBase} +import za.co.absa.enceladus.utils.testUtils.HadoopFsTestBase +import za.co.absa.spark.commons.test.SparkTestBase class ArrayConformanceSuite extends AnyFunSuite with SparkTestBase with BeforeAndAfterAll with HadoopFsTestBase { diff --git a/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/ChorusMockSuite.scala b/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/ChorusMockSuite.scala index c7343e855..222b30958 100644 --- a/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/ChorusMockSuite.scala +++ b/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/ChorusMockSuite.scala @@ -22,7 +22,8 @@ import za.co.absa.enceladus.conformance.datasource.DataSource import za.co.absa.enceladus.dao.MenasDAO import za.co.absa.enceladus.model.conformanceRule.MappingConformanceRule import za.co.absa.enceladus.model.{MappingTable, Dataset => ConfDataset} -import za.co.absa.enceladus.utils.testUtils.{HadoopFsTestBase, LoggerTestBase, SparkTestBase} +import za.co.absa.enceladus.utils.testUtils.{HadoopFsTestBase, LoggerTestBase} +import za.co.absa.spark.commons.test.SparkTestBase case class MyMappingTable(id: Int, mappedAttr: MyMappingTableInner) case class MyMappingTableInner(description: String, name: String) diff --git a/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/InterpreterSuite.scala b/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/InterpreterSuite.scala index 071bf4fb0..2043d1e2b 100644 --- a/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/InterpreterSuite.scala +++ b/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/InterpreterSuite.scala @@ -27,8 +27,9 @@ import za.co.absa.enceladus.conformance.datasource.DataSource import za.co.absa.enceladus.conformance.samples._ import za.co.absa.enceladus.dao.MenasDAO import za.co.absa.enceladus.utils.fs.FileReader -import za.co.absa.enceladus.utils.testUtils.{HadoopFsTestBase, LoggerTestBase, SparkTestBase} +import za.co.absa.enceladus.utils.testUtils.{HadoopFsTestBase, LoggerTestBase} import za.co.absa.enceladus.utils.validation.ValidationLevel +import za.co.absa.spark.commons.test.SparkTestBase class InterpreterSuite extends AnyFunSuite with SparkTestBase with BeforeAndAfterAll with LoggerTestBase with HadoopFsTestBase { diff --git a/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/LiteralJoinMappingRuleTest.scala b/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/LiteralJoinMappingRuleTest.scala index ce3154728..562c749f3 100644 --- a/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/LiteralJoinMappingRuleTest.scala +++ b/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/LiteralJoinMappingRuleTest.scala @@ -22,7 +22,8 @@ import za.co.absa.enceladus.conformance.datasource.DataSource import za.co.absa.enceladus.dao.MenasDAO import za.co.absa.enceladus.model.conformanceRule.{DropConformanceRule, LiteralConformanceRule, MappingConformanceRule} import za.co.absa.enceladus.model.{MappingTable, Dataset => ConfDataset} -import za.co.absa.enceladus.utils.testUtils.{HadoopFsTestBase, LoggerTestBase, SparkTestBase} +import za.co.absa.enceladus.utils.testUtils.{HadoopFsTestBase, LoggerTestBase} +import za.co.absa.spark.commons.test.SparkTestBase class LiteralJoinMappingRuleTest extends AnyFunSuite with SparkTestBase with LoggerTestBase with HadoopFsTestBase { diff --git a/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/NestedStructSuite.scala b/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/NestedStructSuite.scala index 8d2940894..8c05354ec 100644 --- a/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/NestedStructSuite.scala +++ b/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/NestedStructSuite.scala @@ -17,7 +17,8 @@ package za.co.absa.enceladus.conformance.interpreter import org.scalatest.funsuite.AnyFunSuite import za.co.absa.enceladus.conformance.interpreter.fixtures.NestedStructsFixture -import za.co.absa.enceladus.utils.testUtils.{HadoopFsTestBase, SparkTestBase} +import za.co.absa.enceladus.utils.testUtils.HadoopFsTestBase +import za.co.absa.spark.commons.test.SparkTestBase /** * The purpose of these tests is to ensure Catalyst optimizer issue is handled. diff --git a/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/fixtures/MultipleMappingFixture.scala b/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/fixtures/MultipleMappingFixture.scala index b3bb2d7f4..314ae17e5 100644 --- a/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/fixtures/MultipleMappingFixture.scala +++ b/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/fixtures/MultipleMappingFixture.scala @@ -30,7 +30,7 @@ import za.co.absa.enceladus.conformance.datasource.DataSource import za.co.absa.enceladus.dao.MenasDAO import za.co.absa.enceladus.model.{Dataset, MappingTable} import za.co.absa.enceladus.model.conformanceRule._ -import za.co.absa.enceladus.utils.testUtils.SparkTestBase +import za.co.absa.spark.commons.test.SparkTestBase import scala.io.Source.fromFile import scala.util.control.NonFatal diff --git a/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/fixtures/NestedStructsFixture.scala b/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/fixtures/NestedStructsFixture.scala index cc20cdcc1..8e2d51f99 100644 --- a/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/fixtures/NestedStructsFixture.scala +++ b/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/fixtures/NestedStructsFixture.scala @@ -16,6 +16,7 @@ package za.co.absa.enceladus.conformance.interpreter.fixtures import java.io.File + import org.apache.commons.io.{FileUtils, IOUtils} import org.apache.spark.sql.{DataFrame, SaveMode} import org.mockito.Mockito.{mock, when => mockWhen} @@ -26,8 +27,8 @@ import za.co.absa.enceladus.conformance.datasource.DataSource import za.co.absa.enceladus.dao.MenasDAO import za.co.absa.enceladus.model.Dataset import za.co.absa.enceladus.model.conformanceRule._ -import za.co.absa.enceladus.utils.testUtils.SparkTestBase import za.co.absa.enceladus.utils.validation.ValidationLevel +import za.co.absa.spark.commons.test.SparkTestBase import scala.util.control.NonFatal diff --git a/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/fixtures/StreamingFixture.scala b/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/fixtures/StreamingFixture.scala index 9b33fbfaf..22589b2ae 100644 --- a/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/fixtures/StreamingFixture.scala +++ b/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/fixtures/StreamingFixture.scala @@ -30,7 +30,7 @@ import za.co.absa.enceladus.conformance.interpreter.FeatureSwitches import za.co.absa.enceladus.conformance.streaming.{InfoDateFactory, InfoVersionFactory} import za.co.absa.enceladus.dao.MenasDAO import za.co.absa.enceladus.model.Dataset -import za.co.absa.enceladus.utils.testUtils.SparkTestBase +import za.co.absa.spark.commons.test.SparkTestBase trait StreamingFixture extends AnyFunSuite with SparkTestBase with MockitoSugar { private val menasBaseUrls = List.empty[String] diff --git a/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/rules/CastingRuleSuite.scala b/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/rules/CastingRuleSuite.scala index f6586377d..7e980ecf4 100644 --- a/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/rules/CastingRuleSuite.scala +++ b/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/rules/CastingRuleSuite.scala @@ -23,9 +23,10 @@ import za.co.absa.enceladus.conformance.config.ConformanceConfig import za.co.absa.enceladus.conformance.interpreter.{DynamicInterpreter, FeatureSwitches, RuleValidators} import za.co.absa.enceladus.conformance.samples.CastingRuleSamples import za.co.absa.enceladus.dao.MenasDAO -import za.co.absa.enceladus.utils.general.JsonUtils -import za.co.absa.enceladus.utils.testUtils.{HadoopFsTestBase, LoggerTestBase, SparkTestBase} +import za.co.absa.enceladus.utils.testUtils.{HadoopFsTestBase, LoggerTestBase} import za.co.absa.enceladus.utils.validation.ValidationLevel +import za.co.absa.spark.commons.utils.JsonUtils +import za.co.absa.spark.commons.test.SparkTestBase class CastingRuleSuite extends AnyFunSuite with SparkTestBase with LoggerTestBase with HadoopFsTestBase { private val ruleName = "Casting rule" diff --git a/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/rules/CoalesceRuleSuite.scala b/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/rules/CoalesceRuleSuite.scala index e44c8d1b6..ca725b6fa 100644 --- a/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/rules/CoalesceRuleSuite.scala +++ b/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/rules/CoalesceRuleSuite.scala @@ -17,11 +17,11 @@ package za.co.absa.enceladus.conformance.interpreter.rules import org.apache.spark.sql.DataFrame import org.scalatest.funsuite.AnyFunSuite -import za.co.absa.enceladus.utils.testUtils.SparkTestBase import CoalesceRuleSuite._ import za.co.absa.enceladus.conformance.samples.DeepArraySamples import za.co.absa.enceladus.model.Dataset import za.co.absa.enceladus.model.conformanceRule.{CoalesceConformanceRule, DropConformanceRule, LiteralConformanceRule} +import za.co.absa.spark.commons.test.SparkTestBase object CoalesceRuleSuite { private case class ShopItem(id: String, itemName: String, itemDescription: String, qty: Long) diff --git a/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/rules/ConcatenationRuleSuite.scala b/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/rules/ConcatenationRuleSuite.scala index eac2ed4f0..b2ae3fd84 100644 --- a/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/rules/ConcatenationRuleSuite.scala +++ b/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/rules/ConcatenationRuleSuite.scala @@ -20,7 +20,7 @@ import org.scalatest.funsuite.AnyFunSuite import za.co.absa.enceladus.conformance.samples.DeepArraySamples import za.co.absa.enceladus.model.Dataset import za.co.absa.enceladus.model.conformanceRule.{ConcatenationConformanceRule, UppercaseConformanceRule} -import za.co.absa.enceladus.utils.testUtils.SparkTestBase +import za.co.absa.spark.commons.test.SparkTestBase class ConcatenationRuleSuite extends AnyFunSuite with SparkTestBase with TestRuleBehaviors { private val concatRule = ConcatenationConformanceRule(order = 1, outputColumn = "CombinedName", diff --git a/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/rules/DropRuleSuite.scala b/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/rules/DropRuleSuite.scala index 2fdcac3a5..6574c014a 100644 --- a/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/rules/DropRuleSuite.scala +++ b/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/rules/DropRuleSuite.scala @@ -21,7 +21,7 @@ import org.scalatest.funsuite.AnyFunSuite import za.co.absa.enceladus.conformance.samples.DeepArraySamples import za.co.absa.enceladus.model.Dataset import za.co.absa.enceladus.model.conformanceRule.DropConformanceRule -import za.co.absa.enceladus.utils.testUtils.SparkTestBase +import za.co.absa.spark.commons.test.SparkTestBase class DropRuleSuite extends AnyFunSuite with SparkTestBase with TestRuleBehaviors { // scalastyle:off line.size.limit diff --git a/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/rules/FillNullsRuleSuite.scala b/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/rules/FillNullsRuleSuite.scala index 00c354eb8..38151b3ed 100644 --- a/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/rules/FillNullsRuleSuite.scala +++ b/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/rules/FillNullsRuleSuite.scala @@ -20,7 +20,7 @@ import org.scalatest.funsuite.AnyFunSuite import za.co.absa.enceladus.conformance.samples.DeepArraySamples import za.co.absa.enceladus.model.Dataset import za.co.absa.enceladus.model.conformanceRule.FillNullsConformanceRule -import za.co.absa.enceladus.utils.testUtils.SparkTestBase +import za.co.absa.spark.commons.test.SparkTestBase class FillNullsRuleSuite extends AnyFunSuite with SparkTestBase with TestRuleBehaviors { // scalastyle:off line.size.limit diff --git a/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/rules/LiteralRuleSuite.scala b/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/rules/LiteralRuleSuite.scala index b47ade6ff..db298601d 100644 --- a/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/rules/LiteralRuleSuite.scala +++ b/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/rules/LiteralRuleSuite.scala @@ -20,7 +20,7 @@ import org.scalatest.funsuite.AnyFunSuite import za.co.absa.enceladus.conformance.samples.DeepArraySamples import za.co.absa.enceladus.model.Dataset import za.co.absa.enceladus.model.conformanceRule.LiteralConformanceRule -import za.co.absa.enceladus.utils.testUtils.SparkTestBase +import za.co.absa.spark.commons.test.SparkTestBase class LiteralRuleSuite extends AnyFunSuite with SparkTestBase with TestRuleBehaviors { // scalastyle:off line.size.limit diff --git a/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/rules/NegationRuleSuite.scala b/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/rules/NegationRuleSuite.scala index 311954486..86c7d497c 100644 --- a/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/rules/NegationRuleSuite.scala +++ b/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/rules/NegationRuleSuite.scala @@ -24,8 +24,9 @@ import za.co.absa.enceladus.conformance.interpreter.{DynamicInterpreter, Feature import za.co.absa.enceladus.conformance.samples.NegationRuleSamples import za.co.absa.enceladus.dao.MenasDAO import za.co.absa.enceladus.model.{Dataset => ConfDataset} -import za.co.absa.enceladus.utils.testUtils.{HadoopFsTestBase, LoggerTestBase, SparkTestBase} +import za.co.absa.enceladus.utils.testUtils.{HadoopFsTestBase, LoggerTestBase} import za.co.absa.enceladus.utils.validation.ValidationLevel +import za.co.absa.spark.commons.test.SparkTestBase class NegationRuleSuite extends AnyFunSuite with SparkTestBase with LoggerTestBase with HadoopFsTestBase { diff --git a/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/rules/RuleOptimizationSuite.scala b/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/rules/RuleOptimizationSuite.scala index 224661c61..263a56c78 100644 --- a/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/rules/RuleOptimizationSuite.scala +++ b/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/rules/RuleOptimizationSuite.scala @@ -21,7 +21,8 @@ import za.co.absa.enceladus.conformance.interpreter.rules.mapping.MappingRuleInt import za.co.absa.enceladus.conformance.interpreter.{DynamicInterpreter, FeatureSwitches, InterpreterContext, Never} import za.co.absa.enceladus.model.conformanceRule.{ConformanceRule, MappingConformanceRule} import za.co.absa.enceladus.conformance.samples.TradeConformance._ -import za.co.absa.enceladus.utils.testUtils.{HadoopFsTestBase, SparkTestBase} +import za.co.absa.enceladus.utils.testUtils.HadoopFsTestBase +import za.co.absa.spark.commons.test.SparkTestBase class RuleOptimizationSuite extends AnyFunSuite with SparkTestBase with HadoopFsTestBase{ diff --git a/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/rules/RulesSuite.scala b/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/rules/RulesSuite.scala index 1c1f81ad8..d1d73abfb 100644 --- a/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/rules/RulesSuite.scala +++ b/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/rules/RulesSuite.scala @@ -24,7 +24,7 @@ import za.co.absa.enceladus.conformance.interpreter.{ExplosionState, Interpreter import za.co.absa.enceladus.conformance.samples.EmployeeConformance import za.co.absa.enceladus.dao.MenasDAO import za.co.absa.enceladus.model.conformanceRule.ConformanceRule -import za.co.absa.enceladus.utils.testUtils.SparkTestBase +import za.co.absa.spark.commons.test.SparkTestBase class RulesSuite extends AnyFunSuite with SparkTestBase { diff --git a/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/rules/SingleColumnRuleSuite.scala b/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/rules/SingleColumnRuleSuite.scala index 9f2be7c95..2081d507f 100644 --- a/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/rules/SingleColumnRuleSuite.scala +++ b/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/rules/SingleColumnRuleSuite.scala @@ -20,7 +20,7 @@ import org.scalatest.funsuite.AnyFunSuite import za.co.absa.enceladus.conformance.samples.DeepArraySamples import za.co.absa.enceladus.model.Dataset import za.co.absa.enceladus.model.conformanceRule.SingleColumnConformanceRule -import za.co.absa.enceladus.utils.testUtils.SparkTestBase +import za.co.absa.spark.commons.test.SparkTestBase class SingleColumnRuleSuite extends AnyFunSuite with SparkTestBase with TestRuleBehaviors { // scalastyle:off line.size.limit diff --git a/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/rules/SparkSessionRuleSuite.scala b/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/rules/SparkSessionRuleSuite.scala index b03b11048..9f28f2387 100644 --- a/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/rules/SparkSessionRuleSuite.scala +++ b/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/rules/SparkSessionRuleSuite.scala @@ -19,7 +19,7 @@ import org.scalatest.funsuite.AnyFunSuite import za.co.absa.enceladus.conformance.samples.DeepArraySamples import za.co.absa.enceladus.model.Dataset import za.co.absa.enceladus.model.conformanceRule.SparkSessionConfConformanceRule -import za.co.absa.enceladus.utils.testUtils.SparkTestBase +import za.co.absa.spark.commons.test.SparkTestBase class SparkSessionRuleSuite extends AnyFunSuite with SparkTestBase with TestRuleBehaviors { // scalastyle:off line.size.limit diff --git a/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/rules/TestRuleBehaviors.scala b/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/rules/TestRuleBehaviors.scala index dc2d9e57e..88c0797be 100644 --- a/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/rules/TestRuleBehaviors.scala +++ b/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/rules/TestRuleBehaviors.scala @@ -23,8 +23,9 @@ import za.co.absa.enceladus.conformance.config.ConformanceConfig import za.co.absa.enceladus.conformance.interpreter.{DynamicInterpreter, FeatureSwitches} import za.co.absa.enceladus.dao.MenasDAO import za.co.absa.enceladus.model.Dataset -import za.co.absa.enceladus.utils.testUtils.{HadoopFsTestBase, LoggerTestBase, SparkTestBase} +import za.co.absa.enceladus.utils.testUtils.{HadoopFsTestBase, LoggerTestBase} import za.co.absa.enceladus.utils.validation.ValidationLevel +import za.co.absa.spark.commons.test.SparkTestBase trait TestRuleBehaviors extends AnyFunSuite with SparkTestBase with LoggerTestBase with HadoopFsTestBase { diff --git a/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/rules/UppercaseRuleSuite.scala b/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/rules/UppercaseRuleSuite.scala index 77043aaeb..381d879e7 100644 --- a/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/rules/UppercaseRuleSuite.scala +++ b/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/rules/UppercaseRuleSuite.scala @@ -20,7 +20,7 @@ import org.scalatest.funsuite.AnyFunSuite import za.co.absa.enceladus.conformance.samples.DeepArraySamples import za.co.absa.enceladus.model.Dataset import za.co.absa.enceladus.model.conformanceRule.UppercaseConformanceRule -import za.co.absa.enceladus.utils.testUtils.SparkTestBase +import za.co.absa.spark.commons.test.SparkTestBase class UppercaseRuleSuite extends AnyFunSuite with SparkTestBase with TestRuleBehaviors { // scalastyle:off line.size.limit diff --git a/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/rules/custom/CustomRuleSuite.scala b/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/rules/custom/CustomRuleSuite.scala index dd95ee693..aefabdd22 100644 --- a/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/rules/custom/CustomRuleSuite.scala +++ b/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/rules/custom/CustomRuleSuite.scala @@ -26,7 +26,8 @@ import za.co.absa.enceladus.dao.MenasDAO import za.co.absa.enceladus.model.conformanceRule.ConformanceRule import za.co.absa.enceladus.model.{conformanceRule, Dataset => ConfDataset} import za.co.absa.enceladus.utils.error.ErrorMessage -import za.co.absa.enceladus.utils.testUtils.{HadoopFsTestBase, SparkTestBase} +import za.co.absa.enceladus.utils.testUtils.HadoopFsTestBase +import za.co.absa.spark.commons.test.SparkTestBase case class MyCustomRule( order: Int, diff --git a/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/rules/mapping/JoinMappingRuleInterpreterSuite.scala b/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/rules/mapping/JoinMappingRuleInterpreterSuite.scala index ccfc5fb6e..7d0c6ca3c 100644 --- a/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/rules/mapping/JoinMappingRuleInterpreterSuite.scala +++ b/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/rules/mapping/JoinMappingRuleInterpreterSuite.scala @@ -20,7 +20,7 @@ import za.co.absa.enceladus.conformance.datasource.DataSource import za.co.absa.enceladus.conformance.interpreter.rules.ValidationException import za.co.absa.enceladus.conformance.samples.EmployeeConformance import za.co.absa.enceladus.model.conformanceRule.MappingConformanceRule -import za.co.absa.enceladus.utils.testUtils.SparkTestBase +import za.co.absa.spark.commons.test.SparkTestBase class JoinMappingRuleInterpreterSuite extends AnyFunSuite with SparkTestBase { test("Mapping rule fields existence validation test") { diff --git a/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/rules/mapping/MappingGroupExplodeSuite.scala b/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/rules/mapping/MappingGroupExplodeSuite.scala index 0133216b5..5f7b78016 100644 --- a/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/rules/mapping/MappingGroupExplodeSuite.scala +++ b/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/rules/mapping/MappingGroupExplodeSuite.scala @@ -20,7 +20,7 @@ import za.co.absa.enceladus.conformance.interpreter.DynamicInterpreter import za.co.absa.enceladus.conformance.interpreter.rules.testcasefactories.NestedTestCaseFactory._ import za.co.absa.enceladus.conformance.interpreter.rules.testcasefactories.SimpleTestCaseFactory.{simpleMappingRule, simpleMappingRuleMultipleOutputs, simpleMappingRuleMultipleOutputsWithDefaults, simpleMappingRuleWithDefaultValue} import za.co.absa.enceladus.utils.error.ErrorMessage -import za.co.absa.enceladus.utils.general.JsonUtils +import za.co.absa.spark.commons.utils.JsonUtils class MappingGroupExplodeSuite extends MappingInterpreterSuite { import spark.implicits._ diff --git a/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/rules/mapping/MappingInterpreterSuite.scala b/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/rules/mapping/MappingInterpreterSuite.scala index d2fb5a599..37c7c4deb 100644 --- a/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/rules/mapping/MappingInterpreterSuite.scala +++ b/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/rules/mapping/MappingInterpreterSuite.scala @@ -19,7 +19,8 @@ import org.apache.commons.io.IOUtils import org.scalatest.BeforeAndAfterAll import org.scalatest.funsuite.AnyFunSuite import za.co.absa.enceladus.conformance.interpreter.rules.testcasefactories.{NestedTestCaseFactory, SimpleTestCaseFactory} -import za.co.absa.enceladus.utils.testUtils.{HadoopFsTestBase, LoggerTestBase, SparkTestBase} +import za.co.absa.enceladus.utils.testUtils.{HadoopFsTestBase, LoggerTestBase} +import za.co.absa.spark.commons.test.SparkTestBase trait MappingInterpreterSuite extends AnyFunSuite with SparkTestBase with LoggerTestBase with BeforeAndAfterAll with HadoopFsTestBase{ diff --git a/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/rules/mapping/MappingRuleBroadcastSuite.scala b/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/rules/mapping/MappingRuleBroadcastSuite.scala index 11506a31c..f983b377d 100644 --- a/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/rules/mapping/MappingRuleBroadcastSuite.scala +++ b/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/rules/mapping/MappingRuleBroadcastSuite.scala @@ -20,7 +20,7 @@ import za.co.absa.enceladus.conformance.interpreter.DynamicInterpreter import za.co.absa.enceladus.conformance.interpreter.rules.testcasefactories.NestedTestCaseFactory._ import za.co.absa.enceladus.conformance.interpreter.rules.testcasefactories.SimpleTestCaseFactory._ import za.co.absa.enceladus.utils.error.ErrorMessage -import za.co.absa.enceladus.utils.general.JsonUtils +import za.co.absa.spark.commons.utils.JsonUtils class MappingRuleBroadcastSuite extends MappingInterpreterSuite { import spark.implicits._ diff --git a/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/rules/mapping/MappingRuleSuite.scala b/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/rules/mapping/MappingRuleSuite.scala index efbf5b442..b9d61fb17 100644 --- a/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/rules/mapping/MappingRuleSuite.scala +++ b/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/rules/mapping/MappingRuleSuite.scala @@ -21,7 +21,8 @@ import org.scalatest.funsuite.AnyFunSuite import za.co.absa.enceladus.conformance.interpreter.DynamicInterpreter import za.co.absa.enceladus.conformance.interpreter.rules.testcasefactories.SimpleTestCaseFactory import za.co.absa.enceladus.conformance.interpreter.rules.testcasefactories.SimpleTestCaseFactory._ -import za.co.absa.enceladus.utils.testUtils.{HadoopFsTestBase, LoggerTestBase, SparkTestBase} +import za.co.absa.enceladus.utils.testUtils.{HadoopFsTestBase, LoggerTestBase} +import za.co.absa.spark.commons.test.SparkTestBase class MappingRuleSuite extends AnyFunSuite with SparkTestBase with LoggerTestBase with BeforeAndAfterAll with HadoopFsTestBase { private val testCaseFactory = new SimpleTestCaseFactory() diff --git a/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/StandardizationCobolAsciiSuite.scala b/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/StandardizationCobolAsciiSuite.scala index 9311df421..3c9cdbf40 100644 --- a/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/StandardizationCobolAsciiSuite.scala +++ b/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/StandardizationCobolAsciiSuite.scala @@ -24,10 +24,10 @@ import za.co.absa.enceladus.dao.MenasDAO import za.co.absa.enceladus.model.Dataset import za.co.absa.enceladus.standardization.config.StandardizationConfig import za.co.absa.enceladus.standardization.fixtures.TempFileFixture -import za.co.absa.enceladus.utils.testUtils.SparkTestBase - import java.nio.charset.StandardCharsets +import za.co.absa.spark.commons.test.SparkTestBase + class StandardizationCobolAsciiSuite extends FixtureAnyFunSuite with SparkTestBase with TempFileFixture with MockitoSugar { type FixtureParam = String diff --git a/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/StandardizationCobolEbcdicSuite.scala b/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/StandardizationCobolEbcdicSuite.scala index ea0732a64..2e73178d3 100644 --- a/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/StandardizationCobolEbcdicSuite.scala +++ b/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/StandardizationCobolEbcdicSuite.scala @@ -24,7 +24,7 @@ import za.co.absa.enceladus.dao.MenasDAO import za.co.absa.enceladus.model.Dataset import za.co.absa.enceladus.standardization.config.StandardizationConfig import za.co.absa.enceladus.standardization.fixtures.TempFileFixture -import za.co.absa.enceladus.utils.testUtils.SparkTestBase +import za.co.absa.spark.commons.test.SparkTestBase class StandardizationCobolEbcdicSuite extends FixtureAnyFunSuite with SparkTestBase with TempFileFixture with MockitoSugar { diff --git a/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/StandardizationExecutionSuite.scala b/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/StandardizationExecutionSuite.scala index 35eb1adb6..2aa4d6fd9 100644 --- a/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/StandardizationExecutionSuite.scala +++ b/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/StandardizationExecutionSuite.scala @@ -17,6 +17,7 @@ package za.co.absa.enceladus.standardization import java.io.File import java.nio.file.Files + import org.apache.commons.io.FileUtils import org.apache.hadoop.fs.Path import org.apache.spark.sql.DataFrame @@ -42,8 +43,9 @@ import za.co.absa.enceladus.model.{Dataset, Run, SplineReference} import za.co.absa.enceladus.standardization.config.StandardizationConfig import za.co.absa.enceladus.utils.config.PathWithFs import za.co.absa.enceladus.utils.fs.FileReader -import za.co.absa.enceladus.utils.testUtils.{HadoopFsTestBase, SparkTestBase} +import za.co.absa.enceladus.utils.testUtils.HadoopFsTestBase import za.co.absa.enceladus.utils.types.{Defaults, GlobalDefaults} +import za.co.absa.spark.commons.test.SparkTestBase import scala.util.control.NonFatal diff --git a/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/StandardizationFixedWidthSuite.scala b/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/StandardizationFixedWidthSuite.scala index 7d8ea5db4..1c80c9e22 100644 --- a/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/StandardizationFixedWidthSuite.scala +++ b/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/StandardizationFixedWidthSuite.scala @@ -24,10 +24,10 @@ import za.co.absa.enceladus.standardization.config.StandardizationConfig import za.co.absa.enceladus.standardization.interpreter.StandardizationInterpreter import za.co.absa.enceladus.standardization.interpreter.stages.PlainSchemaGenerator import za.co.absa.enceladus.utils.fs.FileReader -import za.co.absa.enceladus.utils.testUtils.SparkTestBase -import za.co.absa.enceladus.utils.implicits.DataFrameImplicits.DataFrameEnhancements +import za.co.absa.spark.commons.implicits.DataFrameImplicits.DataFrameEnhancements import za.co.absa.enceladus.utils.types.{Defaults, GlobalDefaults} import za.co.absa.enceladus.utils.udf.UDFLibrary +import za.co.absa.spark.commons.test.SparkTestBase class StandardizationFixedWidthSuite extends AnyFunSuite with SparkTestBase with MockitoSugar{ private implicit val udfLibrary: UDFLibrary = new UDFLibrary() diff --git a/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/StandardizationJsonSuite.scala b/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/StandardizationJsonSuite.scala index 9a975d943..ca4fd67e3 100644 --- a/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/StandardizationJsonSuite.scala +++ b/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/StandardizationJsonSuite.scala @@ -24,10 +24,10 @@ import za.co.absa.enceladus.standardization.config.StandardizationConfig import za.co.absa.enceladus.standardization.interpreter.StandardizationInterpreter import za.co.absa.enceladus.standardization.interpreter.stages.PlainSchemaGenerator import za.co.absa.enceladus.utils.fs.FileReader -import za.co.absa.enceladus.utils.testUtils.SparkTestBase -import za.co.absa.enceladus.utils.implicits.DataFrameImplicits.DataFrameEnhancements +import za.co.absa.spark.commons.implicits.DataFrameImplicits.DataFrameEnhancements import za.co.absa.enceladus.utils.types.{Defaults, GlobalDefaults} import za.co.absa.enceladus.utils.udf.UDFLibrary +import za.co.absa.spark.commons.test.SparkTestBase class StandardizationJsonSuite extends AnyFunSuite with SparkTestBase with MockitoSugar{ private implicit val udfLibrary:UDFLibrary = new UDFLibrary() diff --git a/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/StandardizationParquetSuite.scala b/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/StandardizationParquetSuite.scala index 7c477a50a..a9274fdc2 100644 --- a/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/StandardizationParquetSuite.scala +++ b/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/StandardizationParquetSuite.scala @@ -16,6 +16,7 @@ package za.co.absa.enceladus.standardization import java.util.UUID + import org.apache.spark.sql.DataFrame import org.apache.spark.sql.types._ import org.scalatest.funsuite.FixtureAnyFunSuite @@ -29,17 +30,17 @@ import za.co.absa.enceladus.standardization.fixtures.TempFileFixture import za.co.absa.enceladus.standardization.interpreter.StandardizationInterpreter import za.co.absa.enceladus.standardization.interpreter.stages.TypeParserException import za.co.absa.enceladus.utils.schema.MetadataKeys -import za.co.absa.enceladus.utils.testUtils.SparkTestBase import za.co.absa.enceladus.utils.udf.UDFLibrary import org.apache.spark.sql.functions.{col, to_timestamp} import za.co.absa.enceladus.utils.types.{Defaults, GlobalDefaults} +import za.co.absa.spark.commons.test.SparkTestBase class StandardizationParquetSuite extends FixtureAnyFunSuite with SparkTestBase with TempFileFixture with MockitoSugar { type FixtureParam = String import spark.implicits._ - import za.co.absa.enceladus.utils.implicits.DataFrameImplicits.DataFrameEnhancements + import za.co.absa.spark.commons.implicits.DataFrameImplicits.DataFrameEnhancements private val standardizationReader = new StandardizationPropertiesProvider() private implicit val dao: MenasDAO = mock[MenasDAO] diff --git a/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/StandardizationRerunSuite.scala b/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/StandardizationRerunSuite.scala index 22d84f453..47e9a461b 100644 --- a/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/StandardizationRerunSuite.scala +++ b/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/StandardizationRerunSuite.scala @@ -16,6 +16,7 @@ package za.co.absa.enceladus.standardization import java.nio.charset.StandardCharsets + import org.apache.spark.sql.DataFrame import org.apache.spark.sql.functions._ import org.apache.spark.sql.types._ @@ -28,14 +29,14 @@ import za.co.absa.enceladus.standardization.config.StandardizationConfig import za.co.absa.enceladus.standardization.fixtures.TempFileFixture import za.co.absa.enceladus.standardization.interpreter.StandardizationInterpreter import za.co.absa.enceladus.utils.error.ErrorMessage -import za.co.absa.enceladus.utils.testUtils.SparkTestBase import za.co.absa.enceladus.utils.types.{Defaults, GlobalDefaults} import za.co.absa.enceladus.utils.udf.UDFLibrary import za.co.absa.enceladus.utils.validation.ValidationException +import za.co.absa.spark.commons.test.SparkTestBase class StandardizationRerunSuite extends FixtureAnyFunSuite with SparkTestBase with TempFileFixture with MockitoSugar { - import za.co.absa.enceladus.utils.implicits.DataFrameImplicits.DataFrameEnhancements + import za.co.absa.spark.commons.implicits.DataFrameImplicits.DataFrameEnhancements private implicit val udfLib: UDFLibrary = new UDFLibrary private implicit val dao: MenasDAO = mock[MenasDAO] diff --git a/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/StandardizationXmlSuite.scala b/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/StandardizationXmlSuite.scala index 6e7e4b1ce..62c923140 100644 --- a/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/StandardizationXmlSuite.scala +++ b/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/StandardizationXmlSuite.scala @@ -25,10 +25,10 @@ import za.co.absa.enceladus.model.Dataset import za.co.absa.enceladus.standardization.config.StandardizationConfig import za.co.absa.enceladus.standardization.interpreter.StandardizationInterpreter import za.co.absa.enceladus.standardization.interpreter.stages.PlainSchemaGenerator -import za.co.absa.enceladus.utils.implicits.DataFrameImplicits.DataFrameEnhancements -import za.co.absa.enceladus.utils.testUtils.SparkTestBase +import za.co.absa.spark.commons.implicits.DataFrameImplicits.DataFrameEnhancements import za.co.absa.enceladus.utils.types.{Defaults, GlobalDefaults} import za.co.absa.enceladus.utils.udf.UDFLibrary +import za.co.absa.spark.commons.test.SparkTestBase class StandardizationXmlSuite extends AnyFunSuite with SparkTestBase with MockitoSugar{ private implicit val udfLibrary:UDFLibrary = new UDFLibrary() diff --git a/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/config/StandardizationParserSuite.scala b/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/config/StandardizationParserSuite.scala index 91d518a24..dd483532c 100644 --- a/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/config/StandardizationParserSuite.scala +++ b/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/config/StandardizationParserSuite.scala @@ -21,7 +21,7 @@ import org.scalatest.funsuite.AnyFunSuite import za.co.absa.enceladus.dao.auth.{MenasKerberosCredentials, MenasPlainCredentials} import za.co.absa.enceladus.model.Dataset import za.co.absa.enceladus.standardization.StandardizationExecution -import za.co.absa.enceladus.utils.testUtils.SparkTestBase +import za.co.absa.spark.commons.test.SparkTestBase class StandardizationParserSuite extends AnyFunSuite with SparkTestBase { diff --git a/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/csv/EnhancedStandardizationCsvSuite.scala b/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/csv/EnhancedStandardizationCsvSuite.scala index f8d8a5a9a..16fb3b289 100644 --- a/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/csv/EnhancedStandardizationCsvSuite.scala +++ b/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/csv/EnhancedStandardizationCsvSuite.scala @@ -21,7 +21,7 @@ import za.co.absa.enceladus.standardization.fixtures.CsvFileFixture class EnhancedStandardizationCsvSuite extends FixtureAnyFunSuite with CsvFileFixture { - import za.co.absa.enceladus.utils.implicits.DataFrameImplicits.DataFrameEnhancements + import za.co.absa.spark.commons.implicits.DataFrameImplicits.DataFrameEnhancements // A field containing the delimiter with the escape has to be enclosed in specified quotes private val content: String = diff --git a/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/csv/NoneParameterStandardizationCsvSuite.scala b/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/csv/NoneParameterStandardizationCsvSuite.scala index 72b269388..ceea2fb4f 100644 --- a/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/csv/NoneParameterStandardizationCsvSuite.scala +++ b/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/csv/NoneParameterStandardizationCsvSuite.scala @@ -20,7 +20,7 @@ import org.scalatest.Outcome import za.co.absa.enceladus.standardization.fixtures.CsvFileFixture class NoneParameterStandardizationCsvSuite extends FixtureAnyFunSuite with CsvFileFixture { - import za.co.absa.enceladus.utils.implicits.DataFrameImplicits.DataFrameEnhancements + import za.co.absa.spark.commons.implicits.DataFrameImplicits.DataFrameEnhancements // A field containing the delimiter with the escape has to be enclosed in specified quotes private val content: String = diff --git a/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/csv/NullValueStandardizationCsvSuite.scala b/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/csv/NullValueStandardizationCsvSuite.scala index b33eebbe0..1b6c691a0 100644 --- a/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/csv/NullValueStandardizationCsvSuite.scala +++ b/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/csv/NullValueStandardizationCsvSuite.scala @@ -18,6 +18,7 @@ package za.co.absa.enceladus.standardization.csv import org.apache.spark.sql.types.{DataType, StructType} import org.mockito.scalatest.MockitoSugar import org.scalatest.funsuite.AnyFunSuite +import za.co.absa.atum.utils.SparkTestBase import za.co.absa.enceladus.dao.MenasDAO import za.co.absa.enceladus.model.Dataset import za.co.absa.enceladus.standardization.StandardizationPropertiesProvider @@ -25,8 +26,7 @@ import za.co.absa.enceladus.standardization.config.StandardizationConfig import za.co.absa.enceladus.standardization.interpreter.StandardizationInterpreter import za.co.absa.enceladus.standardization.interpreter.stages.PlainSchemaGenerator import za.co.absa.enceladus.utils.fs.FileReader -import za.co.absa.enceladus.utils.implicits.DataFrameImplicits.DataFrameEnhancements -import za.co.absa.enceladus.utils.testUtils.SparkTestBase +import za.co.absa.spark.commons.implicits.DataFrameImplicits.DataFrameEnhancements import za.co.absa.enceladus.utils.types.{Defaults, GlobalDefaults} import za.co.absa.enceladus.utils.udf.UDFLibrary diff --git a/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/csv/StandardizationCsvSuite.scala b/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/csv/StandardizationCsvSuite.scala index 9b12a5410..b30e88ed1 100644 --- a/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/csv/StandardizationCsvSuite.scala +++ b/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/csv/StandardizationCsvSuite.scala @@ -22,7 +22,7 @@ import za.co.absa.enceladus.standardization.fixtures.CsvFileFixture class StandardizationCsvSuite extends FixtureAnyFunSuite with CsvFileFixture{ - import za.co.absa.enceladus.utils.implicits.DataFrameImplicits.DataFrameEnhancements + import za.co.absa.spark.commons.implicits.DataFrameImplicits.DataFrameEnhancements private val csvContent: String = """1¡2¡3¡4¡5 diff --git a/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/csv/WhiteSpaceStandardizationCsvSuite.scala b/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/csv/WhiteSpaceStandardizationCsvSuite.scala index 07964f7da..e68e798db 100644 --- a/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/csv/WhiteSpaceStandardizationCsvSuite.scala +++ b/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/csv/WhiteSpaceStandardizationCsvSuite.scala @@ -25,9 +25,9 @@ import za.co.absa.enceladus.standardization.config.StandardizationConfig import za.co.absa.enceladus.standardization.interpreter.StandardizationInterpreter import za.co.absa.enceladus.standardization.interpreter.stages.PlainSchemaGenerator import za.co.absa.enceladus.utils.fs.FileReader -import za.co.absa.enceladus.utils.testUtils.SparkTestBase import za.co.absa.enceladus.utils.types.{Defaults, GlobalDefaults} import za.co.absa.enceladus.utils.udf.UDFLibrary +import za.co.absa.spark.commons.test.SparkTestBase case class Person(id: String, first_name: String, last_name: String) diff --git a/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/fixtures/CsvFileFixture.scala b/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/fixtures/CsvFileFixture.scala index 3cef918c4..7a7655022 100644 --- a/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/fixtures/CsvFileFixture.scala +++ b/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/fixtures/CsvFileFixture.scala @@ -25,7 +25,7 @@ import za.co.absa.enceladus.dao.MenasDAO import za.co.absa.enceladus.model.Dataset import za.co.absa.enceladus.standardization.StandardizationPropertiesProvider import za.co.absa.enceladus.standardization.config.StandardizationConfig -import za.co.absa.enceladus.utils.testUtils.SparkTestBase +import za.co.absa.spark.commons.test.SparkTestBase trait CsvFileFixture extends MockitoSugar with TempFileFixture with SparkTestBase { private implicit val dao: MenasDAO = mock[MenasDAO] diff --git a/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/interpreter/CounterPartySuite.scala b/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/interpreter/CounterPartySuite.scala index f8a6c158b..74bb7075b 100644 --- a/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/interpreter/CounterPartySuite.scala +++ b/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/interpreter/CounterPartySuite.scala @@ -18,9 +18,10 @@ package za.co.absa.enceladus.standardization.interpreter import org.apache.spark.sql.types._ import org.scalatest.funsuite.AnyFunSuite import za.co.absa.enceladus.utils.error.ErrorMessage -import za.co.absa.enceladus.utils.testUtils.{LoggerTestBase, SparkTestBase} +import za.co.absa.enceladus.utils.testUtils.LoggerTestBase import za.co.absa.enceladus.utils.types.{Defaults, GlobalDefaults} import za.co.absa.enceladus.utils.udf.UDFLibrary +import za.co.absa.spark.commons.test.SparkTestBase case class Root(ConformedParty: Party, errCol: Seq[ErrorMessage] = Seq.empty) case class Party(key: Integer, clientKeys1: Seq[String], clientKeys2: Seq[String]) diff --git a/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/interpreter/DateTimeSuite.scala b/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/interpreter/DateTimeSuite.scala index 2290d2e8e..695a4f4eb 100644 --- a/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/interpreter/DateTimeSuite.scala +++ b/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/interpreter/DateTimeSuite.scala @@ -16,6 +16,7 @@ package za.co.absa.enceladus.standardization.interpreter import java.sql.{Date, Timestamp} + import org.apache.spark.sql.types._ import org.apache.spark.sql.{DataFrame, Dataset, Row} import org.scalatest.funsuite.AnyFunSuite @@ -23,11 +24,12 @@ import za.co.absa.enceladus.standardization.interpreter.stages.SchemaChecker import za.co.absa.enceladus.standardization.samples.TestSamples import za.co.absa.enceladus.utils.error.ErrorMessage import za.co.absa.enceladus.utils.fs.FileReader -import za.co.absa.enceladus.utils.testUtils.{LoggerTestBase, SparkTestBase} +import za.co.absa.enceladus.utils.testUtils.LoggerTestBase import za.co.absa.enceladus.utils.types.{Defaults, GlobalDefaults} import za.co.absa.enceladus.utils.udf.UDFLibrary import za.co.absa.enceladus.utils.validation.field.FieldValidationFailure import za.co.absa.enceladus.utils.validation.{SchemaValidator, ValidationError, ValidationException, ValidationWarning} +import za.co.absa.spark.commons.test.SparkTestBase class DateTimeSuite extends AnyFunSuite with SparkTestBase with LoggerTestBase { diff --git a/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/interpreter/SampleDataSuite.scala b/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/interpreter/SampleDataSuite.scala index 359fcf45b..8721787fd 100644 --- a/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/interpreter/SampleDataSuite.scala +++ b/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/interpreter/SampleDataSuite.scala @@ -19,9 +19,10 @@ import org.apache.spark.sql.types.{DataType, StructType} import org.scalatest.funsuite.AnyFunSuite import za.co.absa.enceladus.standardization.samples.{StdEmployee, TestSamples} import za.co.absa.enceladus.utils.fs.FileReader -import za.co.absa.enceladus.utils.testUtils.{LoggerTestBase, SparkTestBase} +import za.co.absa.enceladus.utils.testUtils.LoggerTestBase import za.co.absa.enceladus.utils.types.{Defaults, GlobalDefaults} import za.co.absa.enceladus.utils.udf.UDFLibrary +import za.co.absa.spark.commons.test.SparkTestBase class SampleDataSuite extends AnyFunSuite with SparkTestBase with LoggerTestBase { private implicit val defaults: Defaults = GlobalDefaults diff --git a/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/interpreter/StandardizationInterpreterSuite.scala b/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/interpreter/StandardizationInterpreterSuite.scala index 36e0d6c17..1100951a1 100644 --- a/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/interpreter/StandardizationInterpreterSuite.scala +++ b/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/interpreter/StandardizationInterpreterSuite.scala @@ -20,10 +20,11 @@ import org.scalatest.funsuite.AnyFunSuite import za.co.absa.enceladus.standardization.interpreter.StandardizationInterpreterSuite._ import za.co.absa.enceladus.utils.error.ErrorMessage import za.co.absa.enceladus.utils.fs.FileReader -import za.co.absa.enceladus.utils.general.JsonUtils -import za.co.absa.enceladus.utils.testUtils.{LoggerTestBase, SparkTestBase} +import za.co.absa.enceladus.utils.testUtils.LoggerTestBase import za.co.absa.enceladus.utils.types.{Defaults, GlobalDefaults} import za.co.absa.enceladus.utils.udf.UDFLibrary +import za.co.absa.spark.commons.utils.JsonUtils +import za.co.absa.spark.commons.test.SparkTestBase class StandardizationInterpreterSuite extends AnyFunSuite with SparkTestBase with LoggerTestBase { import spark.implicits._ diff --git a/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/interpreter/StandardizationInterpreter_ArraySuite.scala b/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/interpreter/StandardizationInterpreter_ArraySuite.scala index 97e179937..562d03532 100644 --- a/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/interpreter/StandardizationInterpreter_ArraySuite.scala +++ b/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/interpreter/StandardizationInterpreter_ArraySuite.scala @@ -19,13 +19,14 @@ import org.apache.spark.sql.types._ import org.scalatest.funsuite.AnyFunSuite import org.scalatest.matchers.should.Matchers import za.co.absa.enceladus.common.error.ErrorMessageFactory -import za.co.absa.enceladus.utils.general.JsonUtils -import za.co.absa.enceladus.utils.testUtils.{LoggerTestBase, SparkTestBase} -import za.co.absa.enceladus.utils.implicits.DataFrameImplicits.DataFrameEnhancements +import za.co.absa.enceladus.utils.testUtils.LoggerTestBase +import za.co.absa.spark.commons.implicits.DataFrameImplicits.DataFrameEnhancements import za.co.absa.enceladus.utils.schema.MetadataKeys import za.co.absa.enceladus.utils.types.{Defaults, GlobalDefaults} import za.co.absa.enceladus.utils.udf.UDFLibrary import za.co.absa.enceladus.utils.validation.ValidationException +import za.co.absa.spark.commons.utils.JsonUtils +import za.co.absa.spark.commons.test.SparkTestBase class StandardizationInterpreter_ArraySuite extends AnyFunSuite with SparkTestBase with LoggerTestBase with Matchers { import spark.implicits._ diff --git a/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/interpreter/StandardizationInterpreter_BinarySuite.scala b/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/interpreter/StandardizationInterpreter_BinarySuite.scala index 696ccf7d9..4bf0ab61d 100644 --- a/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/interpreter/StandardizationInterpreter_BinarySuite.scala +++ b/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/interpreter/StandardizationInterpreter_BinarySuite.scala @@ -19,10 +19,11 @@ import org.apache.spark.sql.types.{BinaryType, Metadata, MetadataBuilder, Struct import org.scalatest.funsuite.AnyFunSuite import org.scalatest.matchers.should.Matchers import za.co.absa.enceladus.utils.error.ErrorMessage -import za.co.absa.enceladus.utils.testUtils.{LoggerTestBase, SparkTestBase} +import za.co.absa.enceladus.utils.testUtils.LoggerTestBase import za.co.absa.enceladus.utils.types.{Defaults, GlobalDefaults} import za.co.absa.enceladus.utils.udf.UDFLibrary import za.co.absa.enceladus.utils.validation.ValidationException +import za.co.absa.spark.commons.test.SparkTestBase class StandardizationInterpreter_BinarySuite extends AnyFunSuite with SparkTestBase with LoggerTestBase with Matchers { diff --git a/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/interpreter/StandardizationInterpreter_DateSuite.scala b/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/interpreter/StandardizationInterpreter_DateSuite.scala index b82d226c6..849e9232d 100644 --- a/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/interpreter/StandardizationInterpreter_DateSuite.scala +++ b/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/interpreter/StandardizationInterpreter_DateSuite.scala @@ -16,12 +16,14 @@ package za.co.absa.enceladus.standardization.interpreter import java.sql.Date + import org.apache.spark.sql.types.{DateType, MetadataBuilder, StructField, StructType} import org.scalatest.funsuite.AnyFunSuite import za.co.absa.enceladus.utils.error.ErrorMessage -import za.co.absa.enceladus.utils.testUtils.{LoggerTestBase, SparkTestBase} +import za.co.absa.enceladus.utils.testUtils.LoggerTestBase import za.co.absa.enceladus.utils.types.{Defaults, GlobalDefaults} import za.co.absa.enceladus.utils.udf.UDFLibrary +import za.co.absa.spark.commons.test.SparkTestBase class StandardizationInterpreter_DateSuite extends AnyFunSuite with SparkTestBase with LoggerTestBase { import spark.implicits._ diff --git a/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/interpreter/StandardizationInterpreter_DecimalSuite.scala b/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/interpreter/StandardizationInterpreter_DecimalSuite.scala index 4beabe59f..e48b7dc74 100644 --- a/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/interpreter/StandardizationInterpreter_DecimalSuite.scala +++ b/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/interpreter/StandardizationInterpreter_DecimalSuite.scala @@ -17,13 +17,15 @@ package za.co.absa.enceladus.standardization.interpreter import java.text.{DecimalFormat, NumberFormat} import java.util.Locale + import org.apache.spark.sql.types._ import org.scalatest.funsuite.AnyFunSuite import za.co.absa.enceladus.utils.error.ErrorMessage import za.co.absa.enceladus.utils.schema.MetadataKeys -import za.co.absa.enceladus.utils.testUtils.{LoggerTestBase, SparkTestBase} +import za.co.absa.enceladus.utils.testUtils.LoggerTestBase import za.co.absa.enceladus.utils.types.{Defaults, GlobalDefaults} import za.co.absa.enceladus.utils.udf.UDFLibrary +import za.co.absa.spark.commons.test.SparkTestBase class StandardizationInterpreter_DecimalSuite extends AnyFunSuite with SparkTestBase with LoggerTestBase { import spark.implicits._ diff --git a/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/interpreter/StandardizationInterpreter_FractionalSuite.scala b/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/interpreter/StandardizationInterpreter_FractionalSuite.scala index 9cd0cd860..a64dd0241 100644 --- a/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/interpreter/StandardizationInterpreter_FractionalSuite.scala +++ b/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/interpreter/StandardizationInterpreter_FractionalSuite.scala @@ -19,9 +19,10 @@ import org.apache.spark.sql.types._ import org.scalatest.funsuite.AnyFunSuite import za.co.absa.enceladus.utils.error.ErrorMessage import za.co.absa.enceladus.utils.schema.MetadataKeys -import za.co.absa.enceladus.utils.testUtils.{LoggerTestBase, SparkTestBase} +import za.co.absa.enceladus.utils.testUtils.LoggerTestBase import za.co.absa.enceladus.utils.types.{Defaults, GlobalDefaults} import za.co.absa.enceladus.utils.udf.UDFLibrary +import za.co.absa.spark.commons.test.SparkTestBase class StandardizationInterpreter_FractionalSuite extends AnyFunSuite with SparkTestBase with LoggerTestBase { import spark.implicits._ diff --git a/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/interpreter/StandardizationInterpreter_IntegralSuite.scala b/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/interpreter/StandardizationInterpreter_IntegralSuite.scala index 6587011c3..4ec12d9a3 100644 --- a/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/interpreter/StandardizationInterpreter_IntegralSuite.scala +++ b/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/interpreter/StandardizationInterpreter_IntegralSuite.scala @@ -17,13 +17,15 @@ package za.co.absa.enceladus.standardization.interpreter import java.text.{DecimalFormat, NumberFormat} import java.util.Locale + import org.apache.spark.sql.types._ import org.scalatest.funsuite.AnyFunSuite import za.co.absa.enceladus.utils.error.ErrorMessage import za.co.absa.enceladus.utils.schema.MetadataKeys -import za.co.absa.enceladus.utils.testUtils.{LoggerTestBase, SparkTestBase} +import za.co.absa.enceladus.utils.testUtils.LoggerTestBase import za.co.absa.enceladus.utils.types.{Defaults, GlobalDefaults} import za.co.absa.enceladus.utils.udf.UDFLibrary +import za.co.absa.spark.commons.test.SparkTestBase class StandardizationInterpreter_IntegralSuite extends AnyFunSuite with SparkTestBase with LoggerTestBase{ diff --git a/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/interpreter/StandardizationInterpreter_TimestampSuite.scala b/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/interpreter/StandardizationInterpreter_TimestampSuite.scala index ee01bc020..a8f551e35 100644 --- a/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/interpreter/StandardizationInterpreter_TimestampSuite.scala +++ b/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/interpreter/StandardizationInterpreter_TimestampSuite.scala @@ -16,12 +16,14 @@ package za.co.absa.enceladus.standardization.interpreter import java.sql.Timestamp + import org.apache.spark.sql.types.{MetadataBuilder, StructField, StructType, TimestampType} import org.scalatest.funsuite.AnyFunSuite import za.co.absa.enceladus.utils.error.ErrorMessage -import za.co.absa.enceladus.utils.testUtils.{LoggerTestBase, SparkTestBase} +import za.co.absa.enceladus.utils.testUtils.LoggerTestBase import za.co.absa.enceladus.utils.types.{Defaults, GlobalDefaults} import za.co.absa.enceladus.utils.udf.UDFLibrary +import za.co.absa.spark.commons.test.SparkTestBase class StandardizationInterpreter_TimestampSuite extends AnyFunSuite with SparkTestBase with LoggerTestBase { import spark.implicits._ diff --git a/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/interpreter/StdInterpreterSuite.scala b/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/interpreter/StdInterpreterSuite.scala index 3da7dfcbe..e093a0052 100644 --- a/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/interpreter/StdInterpreterSuite.scala +++ b/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/interpreter/StdInterpreterSuite.scala @@ -16,13 +16,15 @@ package za.co.absa.enceladus.standardization.interpreter import java.sql.{Date, Timestamp} + import org.apache.spark.sql.functions._ import org.apache.spark.sql.types._ import org.scalatest.funsuite.AnyFunSuite import za.co.absa.enceladus.utils.error.ErrorMessage -import za.co.absa.enceladus.utils.testUtils.{LoggerTestBase, SparkTestBase} +import za.co.absa.enceladus.utils.testUtils.LoggerTestBase import za.co.absa.enceladus.utils.types.{Defaults, GlobalDefaults} import za.co.absa.enceladus.utils.udf.UDFLibrary +import za.co.absa.spark.commons.test.SparkTestBase case class ErrorPreserve(a: String, b: String, errCol: List[ErrorMessage]) case class ErrorPreserveStd(a: String, b: Int, errCol: List[ErrorMessage]) diff --git a/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/interpreter/stages/PlainSchemaGeneratorSuite.scala b/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/interpreter/stages/PlainSchemaGeneratorSuite.scala index 5b739cc84..0d4a15dab 100644 --- a/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/interpreter/stages/PlainSchemaGeneratorSuite.scala +++ b/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/interpreter/stages/PlainSchemaGeneratorSuite.scala @@ -17,7 +17,7 @@ package za.co.absa.enceladus.standardization.interpreter.stages import org.apache.spark.sql.types._ import org.scalatest.funsuite.AnyFunSuite -import za.co.absa.enceladus.utils.testUtils.SparkTestBase +import za.co.absa.spark.commons.test.SparkTestBase class PlainSchemaGeneratorSuite extends AnyFunSuite with SparkTestBase { private val schema = StructType(Seq( diff --git a/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/interpreter/stages/SchemaCheckerSuite.scala b/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/interpreter/stages/SchemaCheckerSuite.scala index 024ef7743..262291330 100644 --- a/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/interpreter/stages/SchemaCheckerSuite.scala +++ b/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/interpreter/stages/SchemaCheckerSuite.scala @@ -18,7 +18,7 @@ package za.co.absa.enceladus.standardization.interpreter.stages import org.apache.spark.sql.types.{DataType, StructType} import org.scalatest.funsuite.AnyFunSuite import za.co.absa.enceladus.utils.fs.FileReader -import za.co.absa.enceladus.utils.testUtils.SparkTestBase +import za.co.absa.spark.commons.test.SparkTestBase class SchemaCheckerSuite extends AnyFunSuite with SparkTestBase { test("Bug") { diff --git a/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/interpreter/stages/TypeParserSuite.scala b/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/interpreter/stages/TypeParserSuite.scala index 3288980db..609276a55 100644 --- a/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/interpreter/stages/TypeParserSuite.scala +++ b/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/interpreter/stages/TypeParserSuite.scala @@ -17,11 +17,11 @@ package za.co.absa.enceladus.standardization.interpreter.stages import org.apache.spark.sql.types._ import org.scalatest.funsuite.AnyFunSuite -import za.co.absa.enceladus.utils.testUtils.SparkTestBase import za.co.absa.enceladus.utils.types.TypedStructField.TypedStructFieldTagged import za.co.absa.enceladus.utils.types.parsers.NumericParser import za.co.absa.enceladus.utils.types.{Defaults, GlobalDefaults} import za.co.absa.enceladus.utils.udf.{UDFLibrary, UDFResult} +import za.co.absa.spark.commons.test.SparkTestBase import scala.util.Success diff --git a/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/interpreter/stages/TypeParserSuiteTemplate.scala b/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/interpreter/stages/TypeParserSuiteTemplate.scala index f940524ef..8a75c2215 100644 --- a/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/interpreter/stages/TypeParserSuiteTemplate.scala +++ b/spark-jobs/src/test/scala/za/co/absa/enceladus/standardization/interpreter/stages/TypeParserSuiteTemplate.scala @@ -23,10 +23,10 @@ import org.apache.spark.sql.types._ import org.scalatest.funsuite.AnyFunSuite import za.co.absa.enceladus.standardization.interpreter.dataTypes.ParseOutput import za.co.absa.enceladus.standardization.interpreter.stages.TypeParserSuiteTemplate._ -import za.co.absa.enceladus.utils.testUtils.SparkTestBase import za.co.absa.enceladus.utils.time.DateTimePattern import za.co.absa.enceladus.utils.types.{Defaults, GlobalDefaults, TypedStructField} import za.co.absa.enceladus.utils.udf.UDFLibrary +import za.co.absa.spark.commons.test.SparkTestBase trait TypeParserSuiteTemplate extends AnyFunSuite with SparkTestBase { diff --git a/utils/pom.xml b/utils/pom.xml index 84f3ae216..d60e7c180 100644 --- a/utils/pom.xml +++ b/utils/pom.xml @@ -57,6 +57,11 @@ commons_${scala.compat.version} ${absa.commons.version} + + za.co.absa + spark-commons_${scala.compat.version} + ${absa.spark.commons.version} + org.scalatest scalatest-flatspec_${scala.compat.version} diff --git a/utils/src/main/scala/za/co/absa/enceladus/utils/broadcast/LocalMappingTable.scala b/utils/src/main/scala/za/co/absa/enceladus/utils/broadcast/LocalMappingTable.scala index 2b8763396..9bead7006 100644 --- a/utils/src/main/scala/za/co/absa/enceladus/utils/broadcast/LocalMappingTable.scala +++ b/utils/src/main/scala/za/co/absa/enceladus/utils/broadcast/LocalMappingTable.scala @@ -19,7 +19,7 @@ import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema import org.apache.spark.sql.{Column, DataFrame} import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.{ArrayType, DataType, StructField, StructType} -import za.co.absa.enceladus.utils.schema.SchemaUtils +import za.co.absa.spark.commons.implicits.StructTypeImplicits.{StructTypeEnhancementsArrays} /** * This class contains all necessary information to apply a mapping rule locally on executors. @@ -61,10 +61,10 @@ object LocalMappingTable { val targetAttributes = outputColumns.values.toSeq validateTargetAttributes(mappingTableDf, targetAttributes) - val keyTypes = keyFields.flatMap(fieldName => SchemaUtils.getFieldType(fieldName, mappingTableDf.schema)) + val keyTypes = keyFields.flatMap(fieldName => mappingTableDf.schema.getFieldType(fieldName)) val valueTypes = targetAttributes.flatMap(targetAttribute => { - SchemaUtils.getFieldType(targetAttribute, mappingTableDf.schema) + mappingTableDf.schema.getFieldType(targetAttribute) }) val structFields: Seq[StructField] = outputColumns.keys.toSeq.zip(valueTypes) .map { case (name: String, fieldType: DataType) => StructField(name, fieldType) } @@ -88,7 +88,7 @@ object LocalMappingTable { private def validateKeyFields(mappingTableDf: DataFrame, keyFields: Seq[String]): Unit = { keyFields.foreach(field => { - SchemaUtils.getFieldType(field, mappingTableDf.schema) match { + mappingTableDf.schema.getFieldType(field) match { case Some(_: ArrayType) => throw new IllegalArgumentException(s"Join condition field cannot be an array: $field.") case Some(_: StructType) => throw new IllegalArgumentException(s"Join condition field cannot be a struct: $field.") case Some(_) => @@ -97,7 +97,7 @@ object LocalMappingTable { }) keyFields.foreach(field => { - val arraySubPath = SchemaUtils.getFirstArrayPath(field, mappingTableDf.schema) + val arraySubPath = mappingTableDf.schema.getFirstArrayPath(field) if (arraySubPath.nonEmpty) { throw new IllegalArgumentException(s"Join key field $field is inside an array $arraySubPath.") } @@ -106,12 +106,12 @@ object LocalMappingTable { private def validateTargetAttributes(mappingTableDf: DataFrame, targetAttributes: Seq[String]): Unit = { targetAttributes.foreach(targetAttribute => { - SchemaUtils.getFieldType(targetAttribute, mappingTableDf.schema) match { + mappingTableDf.schema.getFieldType(targetAttribute) match { case Some(_: ArrayType) => throw new IllegalArgumentException(s"Target attribute cannot be an array: $targetAttribute.") case Some(_) => case None => throw new IllegalArgumentException(s"Target attribute $targetAttribute does not exist in the mapping table.") } - val arraySubPath = SchemaUtils.getFirstArrayPath(targetAttribute, mappingTableDf.schema) + val arraySubPath = mappingTableDf.schema.getFirstArrayPath(targetAttribute) if (arraySubPath.nonEmpty) { throw new IllegalArgumentException(s"Target attribute $targetAttribute is inside an array $arraySubPath.") } diff --git a/utils/src/main/scala/za/co/absa/enceladus/utils/explode/ExplodeTools.scala b/utils/src/main/scala/za/co/absa/enceladus/utils/explode/ExplodeTools.scala deleted file mode 100644 index e8975cae9..000000000 --- a/utils/src/main/scala/za/co/absa/enceladus/utils/explode/ExplodeTools.scala +++ /dev/null @@ -1,382 +0,0 @@ -/* - * Copyright 2018 ABSA Group Limited - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package za.co.absa.enceladus.utils.explode - -import org.apache.log4j.LogManager -import org.apache.spark.sql.{Column, DataFrame} -import org.apache.spark.sql.functions._ -import org.apache.spark.sql.types.StructType -import za.co.absa.spark.hats.Extensions._ -import za.co.absa.enceladus.utils.schema.SchemaUtils -import za.co.absa.enceladus.utils.schema.SchemaUtils._ - -object ExplodeTools { - // scalastyle:off null - - private val log = LogManager.getLogger(this.getClass) - - case class DeconstructedNestedField (df: DataFrame, deconstructedField: String, transientField: Option[String]) - - /** - * Explodes all arrays within the path. - * Context can be used to revert all explosions back. - * - * @param columnPathName An column to be exploded. It can be nested inside array or several levels of array nesting - * @param inputDf A DataFrame that contains an array - * @param explosionContext A context returned by previous explosions. If you do several explosions on the top of - * each other it is very important to pass the previous context here so all explosions could - * be reverted - * @return A pair containing an exploded DataFrame and an explosion context. - */ - def explodeAllArraysInPath(columnPathName: String, - inputDf: DataFrame, - explosionContext: ExplosionContext = ExplosionContext()): (DataFrame, ExplosionContext) = { - val arrays = SchemaUtils.getAllArraysInPath(columnPathName, inputDf.schema) - arrays.foldLeft(inputDf, explosionContext)( - (contextPair, arrayColName) => { - contextPair match { - case (df, context) => - log.info(s"Exploding $arrayColName...") - explodeArray(arrayColName, df, context) - } - }) - } - - /** - * Explodes a specific array inside a dataframe in context. Returns a new dataframe and a new context. - * Context can be used to revert all explosions back. - * - * @param arrayColPathName An array field name to be exploded. It can be inside a nested struct, but cannot be nested - * inside another array. If that is the case you need to explode the topmost array first. - * @param inputDf A DataFrame that contains an array - * @param explosionContext A context returned by previous explosions. If you do several explosions on the top of - * each other it is very important to pass the previous context here so all explosions could - * be reverted - * @return A pair containing an exploded DataFrame and an explosion context. - */ - def explodeArray(arrayColPathName: String, - inputDf: DataFrame, - explosionContext: ExplosionContext = ExplosionContext()): (DataFrame, ExplosionContext) = { - - validateArrayField(inputDf.schema, arrayColPathName) - - val explodedColumnName = getUniqueName(explosionTmpColumnName, Some(inputDf.schema)) - val explodedIdName = getRootLevelPrefix(arrayColPathName, "id", inputDf.schema) - val explodedIndexName = getRootLevelPrefix(arrayColPathName, "idx", inputDf.schema) - val explodedSizeName = getRootLevelPrefix(arrayColPathName, "size", inputDf.schema) - - // Adding an unique row id so we can reconstruct the array later by grouping by that id - val dfWithId = inputDf.withColumn(explodedIdName, monotonically_increasing_id()) - - // Add a transient field if we are exploding an array that is an only column of a struct. - // The rationale for this is that otherwise a struct with all null fields will be treated as null - // And after reverting the explosion empty structs will become nulls. - // Spark works fine if the array is not the only field in the struct. So we add a transient field - // that will exist only between explosion and its restoration. - val (dfWithTransientField, superTransientFieldName) = if (isOnlyField(inputDf.schema, arrayColPathName)) { - val (newDf, transientFldName) = addSuperTransientField(dfWithId, arrayColPathName) - (newDf, Some(transientFldName)) - } else { - (dfWithId, None) - } - - // Exploding... - // The '-1' value as an array size indicates that the array field is null. This is to distinguish - // between array field being empty or null - val nullArrayIndicator = -1 - val explodedDf = dfWithTransientField - .select(dfWithId.schema.map(a => col(a.name)) :+ - when(col(arrayColPathName).isNull, - nullArrayIndicator).otherwise(size(col(arrayColPathName))).as(explodedSizeName) :+ - posexplode_outer(col(arrayColPathName)).as(Seq(explodedIndexName, explodedColumnName)): _*) - - val explodedColRenamed = nestedRenameReplace(explodedDf, explodedColumnName, arrayColPathName) - - val newExplosion = Explosion(arrayColPathName, explodedIdName, explodedIndexName, explodedSizeName, - superTransientFieldName) - val newContext = explosionContext.copy(explosions = newExplosion +: explosionContext.explosions) - (explodedColRenamed, newContext) - } - - /** - * Reverts all explosions done by explodeArray(). - * An explosion context should be a context returned by the latest explosion. - * - * @param inputDf A DataFrame that contains an exploded array - * @param explosionContext A context returned by explodeArray() - * @param errorColumn An optional error column to combine during implosion. It should be a top level array. - * @return A dataframe containing restored ('imploded') arrays. - */ - def revertAllExplosions(inputDf: DataFrame, - explosionContext: ExplosionContext, - errorColumn: Option[String] = None): DataFrame = { - explosionContext.explosions.foldLeft(inputDf)((df, explosion) => { - revertSingleExplosion(df, explosion, errorColumn) - }) - } - - /** - * Reverts aa particular explode made by explodeArray(). - * If there were several explodes they should be reverted in FILO order - * - * @param inputDf A DataFrame that contains an exploded array - * @param explosion An explosion object containing all data necessary to revert the explosion - * @param errorColumn An optional error column to combine during implosion. It should be a top level array. - * @return A dataframe containing restored ('imploded') arrays. - */ - // scalastyle:off method.length - def revertSingleExplosion(inputDf: DataFrame, - explosion: Explosion, - errorColumn: Option[String] = None): DataFrame = { - log.info(s"Reverting explosion $explosion...") - - errorColumn.foreach(validateErrorColumnField(inputDf.schema, _)) - - val isNested = explosion.arrayFieldName.contains('.') - - val (decDf, deconstructedField, transientColumn) = if (isNested) { - val deconstructedData = deconstructNestedColumn(inputDf, explosion.arrayFieldName) - DeconstructedNestedField.unapply(deconstructedData).get - } else { - (inputDf, explosion.arrayFieldName, None) - } - - val orderByInsideArray = col(explosion.indexFieldName) - val orderByRecordCol = col(explosion.idFieldName) - - // Do not group by columns that are explosion artifacts - val groupByColumns = inputDf.schema - .filter(a => a.name != explosion.indexFieldName - && (a.name != explosion.arrayFieldName || isNested) - && (errorColumn.isEmpty || a.name != errorColumn.get) - ) - .map(a => col(a.name)) - - // Implode as a temporary column - val tmpColName = getUniqueName(explosionTmpColumnName, Some(inputDf.schema)) - - // Implode - val dfImploded = errorColumn match { - case None => - decDf - .orderBy(orderByRecordCol, orderByInsideArray) - .groupBy(groupByColumns: _*) - .agg(collect_list(deconstructedField).as(tmpColName)) - case Some(errorCol) => - // Implode taking into account the error column - // Errors should be collected, flattened and made distinct - decDf.orderBy(orderByRecordCol, orderByInsideArray) - .groupBy(groupByColumns: _*) - .agg(collect_list(deconstructedField).as(tmpColName), - array_distinct(flatten(collect_list(col(errorCol)))).as(errorCol)) - } - - // Restore null values to yet another temporary field - val tmpColName2 = getUniqueName(nullRestoredTmpColumnName, Some(inputDf.schema)) - val nullsRestored = dfImploded - .withColumn(tmpColName2, when(col(explosion.sizeFieldName) > 0, col(tmpColName)) - .otherwise(when(col(explosion.sizeFieldName) === 0, typedLit(Array())).otherwise(null)) - ) - - val dfArraysRestored = nestedRenameReplace(nullsRestored, tmpColName2, explosion.arrayFieldName, - transientColumn) - - val dfTransientRestored = explosion.superTransientFieldName match { - case Some(transientField) => dfArraysRestored.nestedDropColumn(transientField) - case None => dfArraysRestored - } - - dfTransientRestored - // Drop the temporary column - .drop(col(tmpColName)) - // Drop the array size column - .drop(col(explosion.sizeFieldName)) - // restore original record order - .orderBy(orderByRecordCol) - // remove monotonic id created during explode - .drop(orderByRecordCol) - } - // scalastyle:on method.length - - /** - * Takes a field name nested in a struct and moves it out to the root level as a top level column - * - * @param inputDf A dataframe to process - * @param columnName A nested column to process - * @return A transformed dataframe - **/ - def deconstructNestedColumn(inputDf: DataFrame, columnName: String): DeconstructedNestedField = { - var transientColName: Option[String] = None - def processStruct(schema: StructType, path: Seq[String], parentCol: Option[Column]): Seq[Column] = { - val currentField = path.head - val isLeaf = path.lengthCompare(1) <= 0 - val newFields = schema.fields.flatMap(field => { - if (field.name != currentField) { - Seq(getFullFieldPath(parentCol, field.name).as(field.name)) - } else { - if (isLeaf) { - // Removing the field from the struct replacing it with a transient field - val name = getClosestUniqueName(transientColumnName, schema) - transientColName = Some(name) - Seq(lit(0).as(name)) - } else { - field.dataType match { - case st: StructType => - Seq(struct(processStruct(st, path.tail, Some(getFullFieldPath(parentCol, field.name))): _*) - .as(field.name)) - case _ => - throw new IllegalArgumentException(s"$currentField is not a struct in $columnName") - } - } - } - }) - newFields - } - - val newFieldName = getClosestUniqueName(deconstructedColumnName, inputDf.schema) - val resultDf = inputDf.select(processStruct(inputDf.schema, columnName.split('.'), None) - :+ col(columnName).as(newFieldName): _*) - DeconstructedNestedField(resultDf, newFieldName, transientColName) - } - - /** - * Renames a column `columnFrom` to `columnTo` replacing the original column and putting the resulting column - * under the same struct level of nesting as `columnFrom`. - * - * @param inputDf A dataframe to process - * @param columnFrom A column name that needs to be put into a nested struct - * @param columnTo A column name that `columnFrom` should have after it is renamed - * @param positionColumn A column that should be replaced by contents of columnFrom. It makrs the position of - * the target column placement. - * @return A transformed dataframe - **/ - def nestedRenameReplace(inputDf: DataFrame, - columnFrom: String, - columnTo: String, - positionColumn: Option[String] = None): DataFrame = { - if (!columnTo.contains('.') && !columnFrom.contains('.')) { - var isColumnToFound = false - val newFields = inputDf.schema.fields.flatMap(field => - if (field.name == columnTo) { - isColumnToFound = true - Seq(col(columnFrom).as(columnTo)) - } else if (field.name == columnFrom) { - Nil - } else { - Seq(col(field.name)) - } - ) - val newFields2 = if (isColumnToFound) newFields else newFields :+ col(columnFrom).as(columnTo) - inputDf.select(newFields2: _*) - } else { - putFieldIntoNestedStruct(inputDf, columnFrom, columnTo.split('.'), positionColumn) - } - } - - private def putFieldIntoNestedStruct(df: DataFrame, - columnFrom: String, - pathTo: Seq[String], - placementCol: Option[String] = None): DataFrame = { - def processStruct(schema: StructType, path: Seq[String], parentCol: Option[Column]): Seq[Column] = { - val currentField = path.head - val isLeaf = path.lengthCompare(1) <= 0 - var isFound = false - - val newFields = schema.fields.flatMap(field => { - if (field.name == columnFrom) { - // This removes the original column name (if any) and the transient column - Nil - } else if (!isFound && isLeaf && placementCol.isDefined && placementCol.get == field.name) { - isFound = true - Seq(col(s"`$columnFrom`").as(currentField)) - } else if (!isFound && field.name == currentField) { - field.dataType match { - case _ if isLeaf => - isFound = true - Seq(col(s"`$columnFrom`").as(currentField)) - case st: StructType => - val newFields = processStruct(st, path.tail, Some(getFullFieldPath(parentCol, field.name))) - if (newFields.lengthCompare(1) == 0) { - // a struct that can be null - val fld = newFields.head - Seq(when(fld.isNotNull, struct(newFields: _*)).otherwise(null).as(field.name)) - } else { - // Normat struct - Seq(struct(newFields: _*).as(field.name)) - } - case _ => - throw new IllegalArgumentException(s"$currentField is not a struct in ${pathTo.mkString(".")}") - } - } else { - Seq(getFullFieldPath(parentCol, field.name).as(field.name)) - } - }) - if (!isFound && isLeaf) { - val c = col(s"`$columnFrom`") - newFields :+ c.as(currentField) - } else { - newFields - } - } - - df.select(processStruct(df.schema, pathTo, None): _*) - } - - private def addSuperTransientField(inputDf: DataFrame, arrayColPathName: String): (DataFrame, String) = { - val colName = SchemaUtils.getUniqueName(superTransientColumnName, Some(inputDf.schema)) - val nestedColName = (arrayColPathName.split('.').dropRight(1) :+ colName).mkString(".") - val df = inputDf.nestedWithColumn(nestedColName, lit(null)) - (df, nestedColName) - } - - private def getFullFieldPath(parentCol: Option[Column], fieldName: String): Column = { - parentCol match { - case None => col(fieldName) - case Some(parent) => parent.getField(fieldName) - } - } - - private def getRootLevelPrefix(fieldName: String, prefix: String, schema: StructType): String = { - getClosestUniqueName(s"${fieldName}_$prefix", schema) - .replaceAll("\\.", "_") - } - - private def validateArrayField(schema: StructType, fieldName: String): Unit = { - if (!SchemaUtils.isArray(schema, fieldName)) { - throw new IllegalArgumentException(s"$fieldName is not an array.") - } - - if (!SchemaUtils.isNonNestedArray(schema, fieldName)) { - throw new IllegalArgumentException( - s"$fieldName is an array that is nested in other arrays. Need to explode top level array first.") - } - } - - private def validateErrorColumnField(schema: StructType, fieldName: String): Unit = { - if (fieldName.contains('.')) { - throw new IllegalArgumentException(s"An error column $fieldName cannot be nested.") - } - if (!SchemaUtils.isArray(schema, fieldName)) { - throw new IllegalArgumentException(s"An error column $fieldName is not an array.") - } - } - - private val deconstructedColumnName = "electron" - private val explosionTmpColumnName = "proton" - private val nullRestoredTmpColumnName = "neutron" - private val transientColumnName = "quark" - private val superTransientColumnName = "higgs" -} diff --git a/utils/src/main/scala/za/co/absa/enceladus/utils/explode/Explosion.scala b/utils/src/main/scala/za/co/absa/enceladus/utils/explode/Explosion.scala deleted file mode 100644 index 9dd49dc52..000000000 --- a/utils/src/main/scala/za/co/absa/enceladus/utils/explode/Explosion.scala +++ /dev/null @@ -1,27 +0,0 @@ -/* - * Copyright 2018 ABSA Group Limited - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package za.co.absa.enceladus.utils.explode - -/** - * Stores all info needed to revert a single array explosion. - */ -case class Explosion( - arrayFieldName: String, - idFieldName: String, - indexFieldName: String, - sizeFieldName: String, - superTransientFieldName: Option[String] - ) diff --git a/utils/src/main/scala/za/co/absa/enceladus/utils/explode/ExplosionContext.scala b/utils/src/main/scala/za/co/absa/enceladus/utils/explode/ExplosionContext.scala deleted file mode 100644 index 5c82d7bbd..000000000 --- a/utils/src/main/scala/za/co/absa/enceladus/utils/explode/ExplosionContext.scala +++ /dev/null @@ -1,49 +0,0 @@ -/* - * Copyright 2018 ABSA Group Limited - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package za.co.absa.enceladus.utils.explode - -import org.apache.spark.sql.Column -import org.apache.spark.sql.functions._ - -/** - * Stores a context of several array explosions to they can be reverted in the proper order. - */ -case class ExplosionContext(explosions: Seq[Explosion] = Nil) { - - /** Given a column name generates a condition that should hold if an error is to be generated - * when joining against any array in the column path */ - def getArrayErrorCondition(columnName: String): Column = { - explosions.foldLeft(lit(true))((expr, explosion) => { - if (columnName.startsWith(explosion.arrayFieldName)) { - val arrayIsNull = col(explosion.sizeFieldName) === lit(-1) - val arrayIsEmpty = col(explosion.sizeFieldName) === lit(0) - expr and !arrayIsEmpty and !arrayIsNull - } else { - expr - } - }) - } - - - /** Generates a condition filter for the exploded dataset so control measurements can - * be used for non-array elements. */ - def getControlFrameworkFilter: Column = { - explosions.foldLeft(lit(true))((cond, explosion) => { - cond.and(coalesce(col(explosion.indexFieldName), lit(0)) === 0) - }) - } - -} diff --git a/utils/src/main/scala/za/co/absa/enceladus/utils/general/JsonUtils.scala b/utils/src/main/scala/za/co/absa/enceladus/utils/general/JsonUtils.scala deleted file mode 100644 index bfc9378c1..000000000 --- a/utils/src/main/scala/za/co/absa/enceladus/utils/general/JsonUtils.scala +++ /dev/null @@ -1,60 +0,0 @@ -/* - * Copyright 2018 ABSA Group Limited - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package za.co.absa.enceladus.utils.general - -import com.fasterxml.jackson.databind.ObjectMapper -import org.apache.spark.sql.{DataFrame, SparkSession} - -object JsonUtils { - - /** - * Formats a JSON string so it looks pretty. - * - * @param jsonIn A JSON string - * @return A pretty formatted JSON string - */ - def prettyJSON(jsonIn: String): String = { - val mapper = new ObjectMapper() - - val jsonUnindented = mapper.readValue(jsonIn, classOf[Any]) - val indented = mapper.writerWithDefaultPrettyPrinter.writeValueAsString(jsonUnindented) - indented.replace("\r\n", "\n") - } - - /** - * Formats a Spark-generated JSON strings that are returned by - * applying `.toJSON.collect()` to a DataFrame. - * - * @param jsons A list of JSON documents - * @return A pretty formatted JSON string - */ - def prettySparkJSON(jsons: Seq[String]): String = { - //val properJson = "[" + "}\n".r.replaceAllIn(jsonIn, "},\n") + "]" - val singleJSON = jsons.mkString("[", ",", "]") - prettyJSON(singleJSON) - } - - /** - * Creates a Spark DataFrame from a JSON document(s). - * - * @param json A json string to convert to a DataFrame - * @return A data frame - */ - def getDataFrameFromJson(spark: SparkSession, json: Seq[String]): DataFrame = { - import spark.implicits._ - spark.read.json(json.toDS) - } -} diff --git a/utils/src/main/scala/za/co/absa/enceladus/utils/implicits/DataFrameImplicits.scala b/utils/src/main/scala/za/co/absa/enceladus/utils/implicits/DataFrameImplicits.scala deleted file mode 100644 index 41bfba157..000000000 --- a/utils/src/main/scala/za/co/absa/enceladus/utils/implicits/DataFrameImplicits.scala +++ /dev/null @@ -1,73 +0,0 @@ -/* - * Copyright 2018 ABSA Group Limited - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package za.co.absa.enceladus.utils.implicits - -import java.io.ByteArrayOutputStream - -import org.apache.spark.sql.{Column, DataFrame} -import za.co.absa.enceladus.utils.schema.{SchemaUtils, SparkUtils} - -object DataFrameImplicits { - implicit class DataFrameEnhancements(val df: DataFrame) { - - private def gatherData(showFnc: () => Unit): String = { - val outCapture = new ByteArrayOutputStream - Console.withOut(outCapture) { - showFnc() - } - val dfData = new String(outCapture.toByteArray).replace("\r\n", "\n") - dfData - } - - def dataAsString(): String = { - val showFnc: () => Unit = df.show - gatherData(showFnc) - } - - def dataAsString(truncate: Boolean): String = { - val showFnc: () => Unit = ()=>{df.show(truncate)} - gatherData(showFnc) - } - - def dataAsString(numRows: Int, truncate: Boolean): String = { - val showFnc: ()=>Unit = () => df.show(numRows, truncate) - gatherData(showFnc) - } - - def dataAsString(numRows: Int, truncate: Int): String = { - val showFnc: ()=>Unit = () => df.show(numRows, truncate) - gatherData(showFnc) - } - - def dataAsString(numRows: Int, truncate: Int, vertical: Boolean): String = { - val showFnc: ()=>Unit = () => df.show(numRows, truncate, vertical) - gatherData(showFnc) - } - - /** - * Adds a column to a dataframe if it does not exist - * - * @param colName A column to add if it does not exist already - * @param col An expression for the column to add - * @return a new dataframe with the new column - */ - def withColumnIfDoesNotExist(colName: String, col: Column): DataFrame = { - SparkUtils.withColumnIfDoesNotExist(df, colName, col) - } - - } - -} diff --git a/utils/src/main/scala/za/co/absa/enceladus/utils/implicits/ColumnImplicits.scala b/utils/src/main/scala/za/co/absa/enceladus/utils/implicits/EnceladusColumnImplicits.scala similarity index 55% rename from utils/src/main/scala/za/co/absa/enceladus/utils/implicits/ColumnImplicits.scala rename to utils/src/main/scala/za/co/absa/enceladus/utils/implicits/EnceladusColumnImplicits.scala index a0a9ab1ba..1c8447f33 100644 --- a/utils/src/main/scala/za/co/absa/enceladus/utils/implicits/ColumnImplicits.scala +++ b/utils/src/main/scala/za/co/absa/enceladus/utils/implicits/EnceladusColumnImplicits.scala @@ -18,47 +18,10 @@ package za.co.absa.enceladus.utils.implicits import org.apache.spark.sql.Column import org.apache.spark.sql.functions._ import za.co.absa.enceladus.utils.general.Section +import za.co.absa.spark.commons.implicits.ColumnImplicits.ColumnEnhancements -object ColumnImplicits { - implicit class ColumnEnhancements(column: Column) { - def isInfinite: Column = { - column.isin(Double.PositiveInfinity, Double.NegativeInfinity) - } - - /** - * Spark strings are base on 1 unlike scala. The function shifts the substring indexation to be in accordance with - * Scala/ Java. - * Another enhancement is, that the function allows a negative index, denoting counting of the index from back - * This version takes the substring from the startPos until the end. - * @param startPos the index (zero based) where to start the substring from, if negative it's counted from end - * @return column with requested substring - */ - def zeroBasedSubstr(startPos: Int): Column = { - if (startPos >= 0) { - zeroBasedSubstr(startPos, Int.MaxValue - startPos) - } else { - zeroBasedSubstr(startPos, -startPos) - } - } - - /** - * Spark strings are base on 1 unlike scala. The function shifts the substring indexation to be in accordance with - * Scala/ Java. - * Another enhancement is, that the function allows a negative index, denoting counting of the index from back - * This version takes the substring from the startPos and takes up to the given number of characters (less. - * @param startPos the index (zero based) where to start the substring from, if negative it's counted from end - * @param len length of the desired substring, if longer then the rest of the string, all the remaining characters are taken - * @return column with requested substring - */ - def zeroBasedSubstr(startPos: Int, len: Int): Column = { - if (startPos >= 0) { - column.substr(startPos + 1, len) - } else { - val startPosColumn = greatest(length(column) + startPos + 1, lit(1)) - val lenColumn = lit(len) + when(length(column) + startPos <= 0, length(column) + startPos).otherwise(0) - column.substr(startPosColumn,lenColumn) - } - } +object EnceladusColumnImplicits { + implicit class EnceladusColumnEnhancements(column: Column) { /** * Spark strings are base on 1 unlike scala. The function shifts the substring indexation to be in accordance with @@ -68,7 +31,7 @@ object ColumnImplicits { * @param section the start and requested length of the substring encoded within the Section object * @return column with requested substring */ - def zeroBasedSubstr(section: Section): Column = zeroBasedSubstr(section.start, section.length) + def zeroBasedSubstr(section: Section): Column = column.zeroBasedSubstr(section.start, section.length) /** * Removes part of a StringType column, defined by the provided section. A column containing the remaining part of @@ -100,10 +63,10 @@ object ColumnImplicits { section match { case Section(_, 0) => Left(column) - case Section(0, l) => Left(zeroBasedSubstr(l)) + case Section(0, l) => Left(column.zeroBasedSubstr(l)) case Section(s, l) if (s < 0) && (s + l >= 0) => Left(upToNegative(s)) //till the end - case Section(s, l) if s >= 0 => Right(zeroBasedSubstr(0, s), zeroBasedSubstr(s + l, Int.MaxValue)) - case Section(s, l) => Right(upToNegative(s), zeroBasedSubstr(s + l)) + case Section(s, l) if s >= 0 => Right(column.zeroBasedSubstr(0, s), column.zeroBasedSubstr(s + l, Int.MaxValue)) + case Section(s, l) => Right(upToNegative(s), column.zeroBasedSubstr(s + l)) } } diff --git a/utils/src/main/scala/za/co/absa/enceladus/utils/implicits/StructFieldImplicits.scala b/utils/src/main/scala/za/co/absa/enceladus/utils/implicits/StructFieldImplicits.scala deleted file mode 100644 index 151296349..000000000 --- a/utils/src/main/scala/za/co/absa/enceladus/utils/implicits/StructFieldImplicits.scala +++ /dev/null @@ -1,47 +0,0 @@ -/* - * Copyright 2018 ABSA Group Limited - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package za.co.absa.enceladus.utils.implicits - -import org.apache.spark.sql.types._ -import scala.util.Try - -object StructFieldImplicits { - implicit class StructFieldEnhancements(val structField: StructField) { - def getMetadataString(key: String): Option[String] = { - Try(structField.metadata.getString(key)).toOption - } - - def getMetadataChar(key: String): Option[Char] = { - val resultString = Try(structField.metadata.getString(key)).toOption - resultString.flatMap { s => - if (s.length == 1) { - Option(s(0)) - } else { - None - } - } - } - - def getMetadataStringAsBoolean(key: String): Option[Boolean] = { - Try(structField.metadata.getString(key).toBoolean).toOption - } - - - def hasMetadataKey(key: String): Boolean = { - structField.metadata.contains(key) - } - } -} diff --git a/utils/src/main/scala/za/co/absa/enceladus/utils/schema/SchemaUtils.scala b/utils/src/main/scala/za/co/absa/enceladus/utils/schema/SchemaUtils.scala index eed4c6c1c..627448da9 100644 --- a/utils/src/main/scala/za/co/absa/enceladus/utils/schema/SchemaUtils.scala +++ b/utils/src/main/scala/za/co/absa/enceladus/utils/schema/SchemaUtils.scala @@ -16,119 +16,13 @@ package za.co.absa.enceladus.utils.schema import org.apache.spark.sql.types._ +import za.co.absa.spark.commons.implicits.StructFieldImplicits.StructFieldMetadataEnhancements +import za.co.absa.spark.commons.utils.SchemaUtils.appendPath + import scala.annotation.tailrec -import scala.util.{Random, Try} object SchemaUtils { - /** - * Returns the parent path of a field. Returns an empty string if a root level field name is provided. - * - * @param columnName A fully qualified column name - * @return The parent column name or an empty string if the input column is a root level column - */ - def getParentPath(columnName: String): String = { - val index = columnName.lastIndexOf('.') - if (index > 0) { - columnName.substring(0, index) - } else { - "" - } - } - - /** - * Get a field from a text path and a given schema - * @param path The dot-separated path to the field - * @param schema The schema which should contain the specified path - * @return Some(the requested field) or None if the field does not exist - */ - def getField(path: String, schema: StructType): Option[StructField] = { - - @tailrec - def goThroughArrayDataType(dataType: DataType): DataType = { - dataType match { - case ArrayType(dt, _) => goThroughArrayDataType(dt) - case result => result - } - } - - @tailrec - def examineStructField(names: List[String], structField: StructField): Option[StructField] = { - if (names.isEmpty) { - Option(structField) - } else { - structField.dataType match { - case struct: StructType => examineStructField(names.tail, struct(names.head)) - case ArrayType(el: DataType, _) => - goThroughArrayDataType(el) match { - case struct: StructType => examineStructField(names.tail, struct(names.head)) - case _ => None - } - case _ => None - } - } - } - - val pathTokens = path.split('.').toList - Try{ - examineStructField(pathTokens.tail, schema(pathTokens.head)) - }.getOrElse(None) - } - - /** - * Get a type of a field from a text path and a given schema - * - * @param path The dot-separated path to the field - * @param schema The schema which should contain the specified path - * @return Some(the type of the field) or None if the field does not exist - */ - def getFieldType(path: String, schema: StructType): Option[DataType] = { - getField(path, schema).map(_.dataType) - } - - /** - * Checks if the specified path is an array of structs - * - * @param path The dot-separated path to the field - * @param schema The schema which should contain the specified path - * @return true if the field is an array of structs - */ - def isColumnArrayOfStruct(path: String, schema: StructType): Boolean = { - getFieldType(path, schema) match { - case Some(dt) => - dt match { - case arrayType: ArrayType => - arrayType.elementType match { - case _: StructType => true - case _ => false - } - case _ => false - } - case None => false - } - } - - /** - * Get nullability of a field from a text path and a given schema - * - * @param path The dot-separated path to the field - * @param schema The schema which should contain the specified path - * @return Some(nullable) or None if the field does not exist - */ - def getFieldNullability(path: String, schema: StructType): Option[Boolean] = { - getField(path, schema).map(_.nullable) - } - - /** - * Checks if a field specified by a path and a schema exists - * @param path The dot-separated path to the field - * @param schema The schema which should contain the specified path - * @return True if the field exists false otherwise - */ - def fieldExists(path: String, schema: StructType): Boolean = { - getField(path, schema).nonEmpty - } - /** * Returns all renames in the provided schema. * @param schema schema to examine @@ -144,11 +38,10 @@ object SchemaUtils { struct: StructType, renamesAcc: Map[String, String], predecessorChanged: Boolean): Map[String, String] = { - import za.co.absa.enceladus.utils.implicits.StructFieldImplicits.StructFieldEnhancements struct.fields.foldLeft(renamesAcc) { (renamesSoFar, field) => val fieldFullName = appendPath(path, field.name) - val fieldSourceName = field.getMetadataString(MetadataKeys.SourceColumn).getOrElse(field.name) + val fieldSourceName = field.metadata.getOptString(MetadataKeys.SourceColumn).getOrElse(field.name) val fieldFullSourceName = appendPath(sourcePath, fieldSourceName) val (renames, renameOnPath) = if ((fieldSourceName != field.name) || (predecessorChanged && includeIfPredecessorChanged)) { @@ -179,190 +72,6 @@ object SchemaUtils { getRenamesRecursively("", "", schema, Map.empty, predecessorChanged = false) } - /** - * Get first array column's path out of complete path. - * - * E.g if the path argument is "a.b.c.d.e" where b and d are arrays, "a.b" will be returned. - * - * @param path The path to the attribute - * @param schema The schema of the whole dataset - * @return The path of the first array field or "" if none were found - */ - def getFirstArrayPath(path: String, schema: StructType): String = { - @tailrec - def helper(remPath: Seq[String], pathAcc: Seq[String]): Seq[String] = { - if (remPath.isEmpty) Seq() else { - val currPath = (pathAcc :+ remPath.head).mkString(".") - val currType = getFieldType(currPath, schema) - currType match { - case Some(_: ArrayType) => pathAcc :+ remPath.head - case Some(_) => helper(remPath.tail, pathAcc :+ remPath.head) - case None => Seq() - } - } - } - - val pathToks = path.split('.') - helper(pathToks, Seq()).mkString(".") - } - - /** - * Get paths for all array subfields of this given datatype - */ - def getAllArraySubPaths(path: String, name: String, dt: DataType): Seq[String] = { - val currPath = appendPath(path, name) - dt match { - case s: StructType => s.fields.flatMap(f => getAllArraySubPaths(currPath, f.name, f.dataType)) - case _@ArrayType(elType, _) => getAllArraySubPaths(path, name, elType) :+ currPath - case _ => Seq() - } - } - - /** - * Get all array columns' paths out of complete path. - * - * E.g. if the path argument is "a.b.c.d.e" where b and d are arrays, "a.b" and "a.b.c.d" will be returned. - * - * @param path The path to the attribute - * @param schema The schema of the whole dataset - * @return Seq of dot-separated paths for all array fields in the provided path - */ - def getAllArraysInPath(path: String, schema: StructType): Seq[String] = { - @tailrec - def helper(remPath: Seq[String], pathAcc: Seq[String], arrayAcc: Seq[String]): Seq[String] = { - if (remPath.isEmpty) arrayAcc else { - val currPath = (pathAcc :+ remPath.head).mkString(".") - val currType = getFieldType(currPath, schema) - currType match { - case Some(_: ArrayType) => - val strings = pathAcc :+ remPath.head - helper(remPath.tail, strings, arrayAcc :+ strings.mkString(".")) - case Some(_) => helper(remPath.tail, pathAcc :+ remPath.head, arrayAcc) - case None => arrayAcc - } - } - } - - val pathToks = path.split("\\.") - helper(pathToks, Seq(), Seq()) - } - - /** - * For a given list of field paths determines the deepest common array path. - * - * For instance, if given 'a.b', 'a.b.c', 'a.b.c.d' where b and c are arrays the common deepest array - * path is 'a.b.c'. - * - * If any of the arrays are on diverging paths this function returns None. - * - * The purpose of the function is to determine the order of explosions to be made before the dataframe can be - * joined on a field inside an array. - * - * @param schema A Spark schema - * @param fieldPaths A list of paths to analyze - * @return Returns a common array path if there is one and None if any of the arrays are on diverging paths - */ - def getDeepestCommonArrayPath(schema: StructType, fieldPaths: Seq[String]): Option[String] = { - val arrayPaths = fieldPaths.flatMap(path => getAllArraysInPath(path, schema)).distinct - - if (arrayPaths.nonEmpty && isCommonSubPath(arrayPaths: _*)) { - Some(arrayPaths.maxBy(_.length)) - } else { - None - } - } - - /** - * For a field path determines the deepest array path. - * - * For instance, if given 'a.b.c.d' where b and c are arrays the deepest array is 'a.b.c'. - * - * @param schema A Spark schema - * @param fieldPath A path to analyze - * @return Returns a common array path if there is one and None if any of the arrays are on diverging paths - */ - def getDeepestArrayPath(schema: StructType, fieldPath: String): Option[String] = { - val arrayPaths = getAllArraysInPath(fieldPath, schema) - - if (arrayPaths.nonEmpty) { - Some(arrayPaths.maxBy(_.length)) - } else { - None - } - } - - /** - * For a given list of field paths determines if any path pair is a subset of one another. - * - * For instance, - * - 'a.b', 'a.b.c', 'a.b.c.d' have this property. - * - 'a.b', 'a.b.c', 'a.x.y' does NOT have it, since 'a.b.c' and 'a.x.y' have diverging subpaths. - * - * @param paths A list of paths to be analyzed - * @return true if for all pathe the above property holds - */ - def isCommonSubPath(paths: String*): Boolean = { - def sliceRoot(paths: Seq[Seq[String]]): Seq[Seq[String]] = { - paths.map(path => path.drop(1)).filter(_.nonEmpty) - } - - var isParentCommon = true // For Seq() the property holds by [my] convention - var restOfPaths: Seq[Seq[String]] = paths.map(_.split('.').toSeq).filter(_.nonEmpty) - while (isParentCommon && restOfPaths.nonEmpty) { - val parent = restOfPaths.head.head - isParentCommon = restOfPaths.forall(path => path.head == parent) - restOfPaths = sliceRoot(restOfPaths) - } - isParentCommon - } - - /** - * Get paths for all array fields in the schema - * - * @param schema The schema in which to look for array fields - * @return Seq of dot separated paths of fields in the schema, which are of type Array - */ - def getAllArrayPaths(schema: StructType): Seq[String] = { - schema.fields.flatMap(f => getAllArraySubPaths("", f.name, f.dataType)).toSeq - } - - /** - * Append a new attribute to path or empty string. - * - * @param path The dot-separated existing path - * @param fieldName Name of the field to be appended to the path - * @return The path with the new field appended or the field itself if path is empty - */ - def appendPath(path: String, fieldName: String): String = { - if (path.isEmpty) { - fieldName - } else if (fieldName.isEmpty) { - path - } else { - s"$path.$fieldName" - } - } - - /** - * Determine if a datatype is a primitive one - */ - def isPrimitive(dt: DataType): Boolean = dt match { - case _: BinaryType - | _: BooleanType - | _: ByteType - | _: DateType - | _: DecimalType - | _: DoubleType - | _: FloatType - | _: IntegerType - | _: LongType - | _: NullType - | _: ShortType - | _: StringType - | _: TimestampType => true - case _ => false - } - /** * Determine the name of a field * Will override to "sourcecolumn" in the Metadata if it exists @@ -378,212 +87,6 @@ object SchemaUtils { } } - /** - * For an array of arrays of arrays, ... get the final element type at the bottom of the array - * - * @param arrayType An array data type from a Spark dataframe schema - * @return A non-array data type at the bottom of array nesting - */ - @tailrec - def getDeepestArrayType(arrayType: ArrayType): DataType = { - arrayType.elementType match { - case a: ArrayType => getDeepestArrayType(a) - case b => b - } - } - - /** - * Generate a unique column name - * - * @param prefix A prefix to use for the column name - * @param schema An optional schema to validate if the column already exists (a very low probability) - * @return A name that can be used as a unique column name - */ - def getUniqueName(prefix: String, schema: Option[StructType]): String = { - schema match { - case None => - s"${prefix}_${Random.nextLong().abs}" - case Some(sch) => - var exists = true - var columnName = "" - while (exists) { - columnName = s"${prefix}_${Random.nextLong().abs}" - exists = sch.fields.exists(_.name.compareToIgnoreCase(columnName) == 0) - } - columnName - } - } - - /** - * Get a closest unique column name - * - * @param desiredName A prefix to use for the column name - * @param schema A schema to validate if the column already exists - * @return A name that can be used as a unique column name - */ - def getClosestUniqueName(desiredName: String, schema: StructType): String = { - var exists = true - var columnName = "" - var i = 0 - while (exists) { - columnName = if (i == 0) desiredName else s"${desiredName}_$i" - exists = schema.fields.exists(_.name.compareToIgnoreCase(columnName) == 0) - i += 1 - } - columnName - } - - /** - * Checks if a casting between types always succeeds - * - * @param sourceType A type to be casted - * @param targetType A type to be casted to - * @return true if casting never fails - */ - def isCastAlwaysSucceeds(sourceType: DataType, targetType: DataType): Boolean = { - (sourceType, targetType) match { - case (_: StructType, _) | (_: ArrayType, _) => false - case (a, b) if a == b => true - case (_, _: StringType) => true - case (_: ByteType, _: ShortType | _: IntegerType | _: LongType) => true - case (_: ShortType, _: IntegerType | _: LongType) => true - case (_: IntegerType, _: LongType) => true - case (_: DateType, _: TimestampType) => true - case _ => false - } - } - - /** - * Checks if a field is an array - * - * @param schema A schema - * @param fieldPathName A field to check - * @return true if the specified field is an array - */ - def isArray(schema: StructType, fieldPathName: String): Boolean = { - @tailrec - def arrayHelper(arrayField: ArrayType, path: Seq[String]): Boolean = { - val currentField = path.head - val isLeaf = path.lengthCompare(1) <= 0 - - arrayField.elementType match { - case st: StructType => structHelper(st, path.tail) - case ar: ArrayType => arrayHelper(ar, path) - case _ => - if (!isLeaf) { - throw new IllegalArgumentException( - s"Primitive fields cannot have child fields $currentField is a primitive in $fieldPathName") - } - false - } - } - - def structHelper(structField: StructType, path: Seq[String]): Boolean = { - val currentField = path.head - val isLeaf = path.lengthCompare(1) <= 0 - var isArray = false - structField.fields.foreach(field => - if (field.name == currentField) { - field.dataType match { - case st: StructType => - if (!isLeaf) { - isArray = structHelper(st, path.tail) - } - case ar: ArrayType => - if (isLeaf) { - isArray = true - } else { - isArray = arrayHelper(ar, path) - } - case _ => - if (!isLeaf) { - throw new IllegalArgumentException( - s"Primitive fields cannot have child fields $currentField is a primitive in $fieldPathName") - } - } - } - ) - isArray - } - - val path = fieldPathName.split('.') - structHelper(schema, path) - } - - /** - * Checks if a field is an array that is not nested in another array - * - * @param schema A schema - * @param fieldPathName A field to check - * @return true if a field is an array that is not nested in another array - */ - def isNonNestedArray(schema: StructType, fieldPathName: String): Boolean = { - def structHelper(structField: StructType, path: Seq[String]): Boolean = { - val currentField = path.head - val isLeaf = path.lengthCompare(1) <= 0 - var isArray = false - structField.fields.foreach(field => - if (field.name == currentField) { - field.dataType match { - case st: StructType => - if (!isLeaf) { - isArray = structHelper(st, path.tail) - } - case _: ArrayType => - if (isLeaf) { - isArray = true - } - case _ => - if (!isLeaf) { - throw new IllegalArgumentException( - s"Primitive fields cannot have child fields $currentField is a primitive in $fieldPathName") - } - } - } - ) - isArray - } - - val path = fieldPathName.split('.') - structHelper(schema, path) - } - - /** - * Checks if a field is the only field in a struct - * - * @param schema A schema - * @param column A column to check - * @return true if the column is the only column in a struct - */ - def isOnlyField(schema: StructType, column: String): Boolean = { - def structHelper(structField: StructType, path: Seq[String]): Boolean = { - val currentField = path.head - val isLeaf = path.lengthCompare(1) <= 0 - var isOnlyField = false - structField.fields.foreach(field => - if (field.name == currentField) { - if (isLeaf) { - isOnlyField = structField.fields.length == 1 - } else { - field.dataType match { - case st: StructType => - isOnlyField = structHelper(st, path.tail) - case _: ArrayType => - throw new IllegalArgumentException( - s"SchemaUtils.isOnlyField() does not support checking struct fields inside an array") - case _ => - throw new IllegalArgumentException( - s"Primitive fields cannot have child fields $currentField is a primitive in $column") - } - } - } - ) - isOnlyField - } - val path = column.split('.') - structHelper(schema, path) - } - /** * Converts a fully qualified field name (including its path, e.g. containing fields) to a unique field name without * dot notation diff --git a/utils/src/main/scala/za/co/absa/enceladus/utils/schema/SparkUtils.scala b/utils/src/main/scala/za/co/absa/enceladus/utils/schema/SparkUtils.scala index 1805ae944..4e8258457 100644 --- a/utils/src/main/scala/za/co/absa/enceladus/utils/schema/SparkUtils.scala +++ b/utils/src/main/scala/za/co/absa/enceladus/utils/schema/SparkUtils.scala @@ -21,6 +21,7 @@ import org.apache.spark.sql.types.StructType import org.apache.spark.sql.{Column, DataFrame, SparkSession} import za.co.absa.enceladus.utils.error.ErrorMessage import za.co.absa.enceladus.utils.udf.UDFLibrary +import za.co.absa.spark.commons.implicits.StructTypeImplicits.StructTypeEnhancements import za.co.absa.spark.hats.transformations.NestedArrayTransformations @@ -41,8 +42,8 @@ object SparkUtils { * @return the field name set */ def setUniqueColumnNameOfCorruptRecord(spark: SparkSession, schema: StructType): String = { - val result = if (SchemaUtils.fieldExists(DefaultColumnNameOfCorruptRecord, schema)) { - SchemaUtils.getClosestUniqueName(DefaultColumnNameOfCorruptRecord, schema) + val result = if (schema.fieldExists(DefaultColumnNameOfCorruptRecord)) { + schema.getClosestUniqueName(DefaultColumnNameOfCorruptRecord) } else { DefaultColumnNameOfCorruptRecord } @@ -82,8 +83,8 @@ object SparkUtils { implicit val udfLib: UDFLibrary = new UDFLibrary - val tmpColumn = SchemaUtils.getUniqueName("tmpColumn", Some(df.schema)) - val tmpErrColumn = SchemaUtils.getUniqueName("tmpErrColumn", Some(df.schema)) + val tmpColumn = df.schema.getClosestUniqueName("tmpColumn") + val tmpErrColumn = df.schema.getClosestUniqueName("tmpErrColumn") val litErrUdfCall = callUDF("confLitErr", lit(colName), col(tmpColumn)) // Rename the original column to a temporary name. We need it for comparison. diff --git a/utils/src/main/scala/za/co/absa/enceladus/utils/testUtils/LoggerTestBase.scala b/utils/src/main/scala/za/co/absa/enceladus/utils/testUtils/LoggerTestBase.scala index 71e88cac8..b5bb7661c 100644 --- a/utils/src/main/scala/za/co/absa/enceladus/utils/testUtils/LoggerTestBase.scala +++ b/utils/src/main/scala/za/co/absa/enceladus/utils/testUtils/LoggerTestBase.scala @@ -15,12 +15,11 @@ package za.co.absa.enceladus.utils.testUtils -import java.io.ByteArrayOutputStream - import org.apache.spark.sql.DataFrame import org.slf4j.{Logger, LoggerFactory} import org.slf4j.event.Level import org.slf4j.event.Level._ +import za.co.absa.spark.commons.implicits.DataFrameImplicits.DataFrameEnhancements trait LoggerTestBase { @@ -37,7 +36,6 @@ trait LoggerTestBase { } protected def logDataFrameContent(df: DataFrame, logLevel: Level = DEBUG): Unit = { - import za.co.absa.enceladus.utils.implicits.DataFrameImplicits.DataFrameEnhancements val logFnc = logLevelToLogFunction(logLevel) logFnc(df.schema.treeString) diff --git a/utils/src/main/scala/za/co/absa/enceladus/utils/testUtils/SparkLocalMaster.scala b/utils/src/main/scala/za/co/absa/enceladus/utils/testUtils/SparkLocalMaster.scala deleted file mode 100644 index 50063b0f9..000000000 --- a/utils/src/main/scala/za/co/absa/enceladus/utils/testUtils/SparkLocalMaster.scala +++ /dev/null @@ -1,20 +0,0 @@ -/* - * Copyright 2018 ABSA Group Limited - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package za.co.absa.enceladus.utils.testUtils - -trait SparkLocalMaster { - System.getProperties.setProperty("spark.master", "local[4]") -} diff --git a/utils/src/main/scala/za/co/absa/enceladus/utils/testUtils/SparkTestBase.scala b/utils/src/main/scala/za/co/absa/enceladus/utils/testUtils/SparkTestBase.scala deleted file mode 100644 index aceebe64e..000000000 --- a/utils/src/main/scala/za/co/absa/enceladus/utils/testUtils/SparkTestBase.scala +++ /dev/null @@ -1,136 +0,0 @@ -/* - * Copyright 2018 ABSA Group Limited - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package za.co.absa.enceladus.utils.testUtils - -import org.apache.log4j.{Level, Logger} -import org.apache.spark.sql.SparkSession -import org.apache.spark.SparkConf -import com.typesafe.config.ConfigFactory -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.{FileSystem, Path} -import scala.collection.JavaConversions._ -import java.io.File -import za.co.absa.enceladus.utils.time.TimeZoneNormalizer -import com.typesafe.config.Config - -trait SparkTestBase extends HasSparkSession { self => - TimeZoneNormalizer.normalizeJVMTimeZone() - - val config: Config = ConfigFactory.load() - val sparkMaster: String = config.getString("enceladus.utils.testUtils.sparkTestBaseMaster") - - val sparkBuilder: SparkSession.Builder = SparkSession.builder() - .master(sparkMaster) - .appName(s"Enceladus test - ${self.getClass.getName}") - .config("spark.ui.enabled", "false") - .config("spark.debug.maxToStringFields", 100) // scalastyle:ignore magic.number - // ^- default value is insufficient for some tests, 100 is a compromise between resource consumption and expected need - - implicit val spark: SparkSession = if (sparkMaster == "yarn") { - val confDir = config.getString("enceladus.utils.testUtils.hadoop.conf.dir") - val distJarsDir = config.getString("enceladus.utils.testUtils.spark.distJars.dir") - val sparkHomeDir = config.getString("enceladus.utils.testUtils.spark.home.dir") - - val hadoopConfigs = SparkTestBase.getHadoopConfigurationForSpark(confDir) - val sparkConfigs = SparkTestBase.loadSparkDefaults(sparkHomeDir) - val allConfigs = hadoopConfigs ++ sparkConfigs - - //get a list of all dist jars - val distJars = FileSystem.get(SparkTestBase.getHadoopConfiguration(confDir)).listStatus(new Path(distJarsDir)).map(_.getPath) - val localJars = SparkTestBase.getDepsFromClassPath("absa") - val currentJars = SparkTestBase.getCurrentProjectJars - val deps = (distJars ++ localJars ++currentJars).mkString(",") - - sparkBuilder.config(new SparkConf().setAll(allConfigs)) - .config("spark.yarn.jars", deps) - .config("spark.deploy.mode", "client") - .getOrCreate() - - } else { - sparkBuilder - .config("spark.driver.bindAddress", "127.0.0.1") - .config("spark.driver.host", "127.0.0.1") - .getOrCreate() - } - TimeZoneNormalizer.normalizeSessionTimeZone(spark) - - // Do not display INFO entries for tests - Logger.getLogger("org").setLevel(Level.WARN) - Logger.getLogger("akka").setLevel(Level.WARN) -} - -object SparkTestBase { - /** - * Gets a Hadoop configuration object from the specified hadoopConfDir parameter - * - * @param hadoopConfDir string representation of HADOOP_CONF_DIR - */ - def getHadoopConfiguration(hadoopConfDir: String): Configuration = { - val hadoopConf = new Configuration() - hadoopConf.addResource(new Path(s"$hadoopConfDir/hdfs-site.xml")) - hadoopConf.addResource(new Path(s"$hadoopConfDir/yarn-site.xml")) - hadoopConf.addResource(new Path(s"$hadoopConfDir/core-site.xml")) - - hadoopConf - } - - /** - * Converts all entries from a Hadoop configuration to Map, which can be consumed by SparkConf - * - * @param hadoopConf Hadoop Configuration object to be converted into Spark configs - */ - def hadoopConfToSparkMap(hadoopConf: Configuration): Map[String, String] = { - hadoopConf.iterator().map(entry => (s"spark.hadoop.${entry.getKey}", entry.getValue)).toMap - } - - /** - * Get Hadoop configuration consumable by SparkConf - */ - def getHadoopConfigurationForSpark(hadoopConfDir: String): Map[String, String] = { - hadoopConfToSparkMap(getHadoopConfiguration(hadoopConfDir)) - } - - /** - * Loads spark defaults from the specified SPARK_HOME directory - */ - def loadSparkDefaults(sparkHome: String): Map[String, String] = { - val sparkConfigIn = ConfigFactory.empty().atPath(s"$sparkHome/conf/spark-defaults.conf") - sparkConfigIn - .entrySet() - .filter(_.getKey != "spark.yarn.jars") - .map(entry => (entry.getKey, entry.getValue.unwrapped().toString)) - .toMap - } - - /** - * Gets the list of jars, which are currently loaded in the classpath and contain the given inclPattern in the file name - */ - def getDepsFromClassPath(inclPattern: String): Seq[String] = { - val cl = this.getClass.getClassLoader - cl.asInstanceOf[java.net.URLClassLoader].getURLs.filter(c => c.toString.contains(inclPattern)).map(_.toString()) - } - - /** - * Get the list of jar(s) of the current project - */ - def getCurrentProjectJars: Seq[String] = { - val targetDir = new File(s"${System.getProperty("user.dir")}/target") - targetDir - .listFiles() - .filter(f => f.getName.split("\\.").last.toLowerCase() == "jar" && f.getName.contains("original")) - .map(_.getAbsolutePath) - } -} diff --git a/utils/src/main/scala/za/co/absa/enceladus/utils/transformations/ArrayTransformations.scala b/utils/src/main/scala/za/co/absa/enceladus/utils/transformations/ArrayTransformations.scala index d51668f7f..51f17eb54 100644 --- a/utils/src/main/scala/za/co/absa/enceladus/utils/transformations/ArrayTransformations.scala +++ b/utils/src/main/scala/za/co/absa/enceladus/utils/transformations/ArrayTransformations.scala @@ -18,11 +18,11 @@ package za.co.absa.enceladus.utils.transformations import org.apache.spark.sql._ import org.apache.spark.sql.functions._ import org.apache.spark.sql.api.java.UDF1 -import za.co.absa.enceladus.utils.schema.SchemaUtils import org.apache.spark.sql.types._ import org.apache.spark.sql.api.java.UDF2 import org.slf4j.LoggerFactory import org.apache.spark.storage.StorageLevel +import za.co.absa.spark.commons.implicits.StructTypeImplicits.StructTypeEnhancementsArrays object ArrayTransformations { @@ -59,7 +59,7 @@ object ArrayTransformations { def helper(tokens: List[String], pathAcc: Seq[String]): Column = { val currPath = (pathAcc :+ tokens.head).mkString(".") - val topType = SchemaUtils.getFieldType(currPath, ds.schema) + val topType = ds.schema.getFieldType(currPath) // got a match if (currPath == columnName) { @@ -90,7 +90,7 @@ object ArrayTransformations { } private def getArraySchema(field: String, schema: StructType): ArrayType = { - val arrType = SchemaUtils.getFieldType(field, schema) + val arrType = schema.getFieldType(field) if (arrType.isEmpty || !arrType.get.isInstanceOf[ArrayType]) { throw new IllegalStateException(s"Column $field either does not exist or is not of type ArrayType") } else { @@ -151,8 +151,8 @@ object ArrayTransformations { val list = withInd.groupBy(arrCol(groupField)).agg(collect_list(col(arrayCol)) as arrayCol) val tmp2 = if (tokens.length > 1) nestedWithColumn(list)(arrayCol, col(s"`$arrayCol`")).drop(col(s"`$arrayCol`")) else list - val origArraySchema = SchemaUtils.getFieldType(arrayCol, ds.schema).getOrElse(throw new IllegalStateException(s"The field $arrayCol not found in the transformed schema.")).asInstanceOf[ArrayType] - val arrayChildSchema = SchemaUtils.getFieldType(arrayCol, transformed.schema).getOrElse(throw new IllegalStateException(s"The field $arrayCol not found in the transformed schema.")) + val origArraySchema = ds.schema.getFieldType(arrayCol).getOrElse(throw new IllegalStateException(s"The field $arrayCol not found in the transformed schema.")).asInstanceOf[ArrayType] + val arrayChildSchema = transformed.schema.getFieldType(arrayCol).getOrElse(throw new IllegalStateException(s"The field $arrayCol not found in the transformed schema.")) val arraySchema = ArrayType.apply(arrayChildSchema, origArraySchema.containsNull) spark.udf.register(s"${groupField}_handleNullAndEmpty", new UDF2[Int, Seq[Row], Seq[Row]] { @@ -180,10 +180,10 @@ object ArrayTransformations { def nestedDrop(df: Dataset[Row], colName: String): Dataset[Row] = { val toks = colName.split("\\.") if (toks.size == 1) df.drop(colName) else { - if (SchemaUtils.getFirstArrayPath(colName, df.schema) != "") throw new IllegalStateException(s"Array Type fields in the path of $colName - dropping arrays children is not supported") + if (df.schema.getFirstArrayPath(colName) != "") throw new IllegalStateException(s"Array Type fields in the path of $colName - dropping arrays children is not supported") val parentPath = toks.init.mkString(".") logger.info(s"Nested Drop: parent path $parentPath") - val parentType = SchemaUtils.getFieldType(parentPath, df.schema) + val parentType = df.schema.getFieldType(parentPath) logger.info(s"Nested Drop: parent type $parentType") val parentCols = if (parentType.isEmpty) throw new IllegalStateException(s"Field $colName does not exist in \n ${df.printSchema()}") else parentType.get.asInstanceOf[StructType].fields val replace = struct(parentCols.filter(_.name != toks.last).map(x => arrCol(s"$parentPath.${x.name}") as x.name): _*) @@ -192,7 +192,7 @@ object ArrayTransformations { } def flattenArrays(df: Dataset[Row], colName: String)(implicit spark: SparkSession): Dataset[Row] = { - val typ = SchemaUtils.getFieldType(colName, df.schema).getOrElse(throw new Error(s"Field $colName does not exist in ${df.schema.printTreeString()}")) + val typ = df.schema.getFieldType(colName).getOrElse(throw new Error(s"Field $colName does not exist in ${df.schema.printTreeString()}")) if (!typ.isInstanceOf[ArrayType]) { logger.info(s"Field $colName is not an ArrayType, returning the original dataset!") df @@ -216,7 +216,7 @@ object ArrayTransformations { def handleArrays(targetColumn: String, df: Dataset[Row])(fn: Dataset[Row] => Dataset[Row])(implicit spark: SparkSession): Dataset[Row] = { logger.info(s"handleArrays: Finding first array for $targetColumn") - val firstArr = SchemaUtils.getFirstArrayPath(targetColumn, df.schema) + val firstArr = df.schema.getFirstArrayPath(targetColumn) logger.info(s"handleArrays: First array field $firstArr") firstArr match { case "" => fn(df) diff --git a/utils/src/main/scala/za/co/absa/enceladus/utils/types/TypedStructField.scala b/utils/src/main/scala/za/co/absa/enceladus/utils/types/TypedStructField.scala index 9098717ff..bdab0ae81 100644 --- a/utils/src/main/scala/za/co/absa/enceladus/utils/types/TypedStructField.scala +++ b/utils/src/main/scala/za/co/absa/enceladus/utils/types/TypedStructField.scala @@ -19,7 +19,6 @@ import java.sql.{Date, Timestamp} import java.util.Base64 import org.apache.spark.sql.types._ -import za.co.absa.enceladus.utils.implicits.StructFieldImplicits.StructFieldEnhancements import za.co.absa.enceladus.utils.numeric._ import za.co.absa.enceladus.utils.schema.{MetadataKeys, MetadataValues} import za.co.absa.enceladus.utils.time.DateTimePattern @@ -27,6 +26,8 @@ import za.co.absa.enceladus.utils.typeClasses.{DoubleLike, LongLike} import za.co.absa.enceladus.utils.types.parsers._ import za.co.absa.enceladus.utils.validation.ValidationIssue import za.co.absa.enceladus.utils.validation.field._ +import za.co.absa.spark.commons.implicits.StructFieldImplicits.StructFieldEnhancements +import za.co.absa.spark.commons.implicits.StructFieldImplicits.StructFieldMetadataEnhancements import scala.util.{Failure, Success, Try} @@ -76,7 +77,7 @@ sealed abstract class TypedStructField(structField: StructField)(implicit defaul * inner Option - the actual default value or None in case the default is null */ def ownDefaultValue: Try[Option[Option[BaseType]]] = { - if (hasMetadataKey(MetadataKeys.DefaultValue)) { + if (structField.metadata.hasKey(MetadataKeys.DefaultValue)) { for { defaultValueString <- Try{structField.metadata.getString(MetadataKeys.DefaultValue)} defaultValueTyped <- stringToTyped(defaultValueString) @@ -187,7 +188,7 @@ object TypedStructField { final class BinaryTypeStructField private[TypedStructField](structField: StructField) (implicit defaults: Defaults) extends TypedStructFieldTagged[Array[Byte]](structField) { - val normalizedEncoding: Option[String] = structField.getMetadataString(MetadataKeys.Encoding).map(_.toLowerCase) + val normalizedEncoding: Option[String] = structField.metadata.getOptString(MetadataKeys.Encoding).map(_.toLowerCase) // used to convert the default value from metadata's [[MetadataKeys.DefaultValue]] override protected def convertString(string: String): Try[Array[Byte]] = { @@ -238,7 +239,7 @@ object TypedStructField { } private def readNumericPatternFromMetadata: Option[NumericPattern] = { - val stringPatternOpt = getMetadataString(MetadataKeys.Pattern) + val stringPatternOpt = structField.metadata.getOptString(MetadataKeys.Pattern) val decimalSymbolsOpt = readDecimalSymbolsFromMetadata() if (stringPatternOpt.nonEmpty) { @@ -250,9 +251,9 @@ object TypedStructField { private def readDecimalSymbolsFromMetadata(): Option[DecimalSymbols] = { val ds = defaults.getDecimalSymbols - val minusSign = getMetadataChar(MetadataKeys.MinusSign).getOrElse(ds.minusSign) - val decimalSeparator = getMetadataChar(MetadataKeys.DecimalSeparator).getOrElse(ds.decimalSeparator) - val groupingSeparator = getMetadataChar(MetadataKeys.GroupingSeparator).getOrElse(ds.groupingSeparator) + val minusSign = structField.metadata.getOptChar(MetadataKeys.MinusSign).getOrElse(ds.minusSign) + val decimalSeparator = structField.metadata.getOptChar(MetadataKeys.DecimalSeparator).getOrElse(ds.decimalSeparator) + val groupingSeparator = structField.metadata.getOptChar(MetadataKeys.GroupingSeparator).getOrElse(ds.groupingSeparator) if ((ds.minusSign != minusSign) || (ds.decimalSeparator != decimalSeparator) || (ds.groupingSeparator != groupingSeparator)) { Option(ds.copy(minusSign = minusSign, decimalSeparator = decimalSeparator, groupingSeparator = groupingSeparator)) @@ -291,7 +292,7 @@ object TypedStructField { } private def readRadixFromMetadata:Radix = { - Try(getMetadataString(MetadataKeys.Radix).map(Radix(_))).toOption.flatten.getOrElse(Radix.DefaultRadix) + Try(structField.metadata.getOptString(MetadataKeys.Radix).map(Radix(_))).toOption.flatten.getOrElse(Radix.DefaultRadix) } } @@ -314,7 +315,7 @@ object TypedStructField { (implicit defaults: Defaults) extends NumericTypeStructField[D](structField, typeMin, typeMax) { - override val allowInfinity: Boolean = getMetadataStringAsBoolean(MetadataKeys.AllowInfinity).getOrElse(false) + override val allowInfinity: Boolean = structField.metadata.getOptStringAsBoolean(MetadataKeys.AllowInfinity).getOrElse(false) override val parser: Try[NumericParser[D]] = { pattern.map {patternOpt => @@ -349,7 +350,7 @@ object TypedStructField { DecimalTypeStructField.minPossible(dataType), DecimalTypeStructField.maxPossible(dataType) ){ - val strictParsing: Boolean = getMetadataStringAsBoolean(MetadataKeys.StrictParsing).getOrElse(false) + val strictParsing: Boolean = structField.metadata.getOptStringAsBoolean(MetadataKeys.StrictParsing).getOrElse(false) override val parser: Try[DecimalParser] = { val maxScale = if(strictParsing) Some(scale) else None @@ -400,7 +401,7 @@ object TypedStructField { } def defaultTimeZone: Option[String] = { - getMetadataString(MetadataKeys.DefaultTimeZone) + structField.metadata.getOptString(MetadataKeys.DefaultTimeZone) } override def validate(): Seq[ValidationIssue] = { @@ -408,8 +409,8 @@ object TypedStructField { } private def readDateTimePattern: DateTimePattern = { - getMetadataString(MetadataKeys.Pattern).map { pattern => - val timeZoneOpt = getMetadataString(MetadataKeys.DefaultTimeZone) + structField.metadata.getOptString(MetadataKeys.Pattern).map { pattern => + val timeZoneOpt = structField.metadata.getOptString(MetadataKeys.DefaultTimeZone) DateTimePattern(pattern, timeZoneOpt) }.getOrElse( DateTimePattern.asDefault(defaults.getStringPattern(structField.dataType), None) diff --git a/utils/src/main/scala/za/co/absa/enceladus/utils/validation/field/BinaryFieldValidator.scala b/utils/src/main/scala/za/co/absa/enceladus/utils/validation/field/BinaryFieldValidator.scala index bd8362644..3d72d7882 100644 --- a/utils/src/main/scala/za/co/absa/enceladus/utils/validation/field/BinaryFieldValidator.scala +++ b/utils/src/main/scala/za/co/absa/enceladus/utils/validation/field/BinaryFieldValidator.scala @@ -17,11 +17,11 @@ package za.co.absa.enceladus.utils.validation.field import java.util.Base64 -import za.co.absa.enceladus.utils.implicits.StructFieldImplicits._ import za.co.absa.enceladus.utils.schema.{MetadataKeys, MetadataValues} import za.co.absa.enceladus.utils.types.TypedStructField import za.co.absa.enceladus.utils.types.TypedStructField.BinaryTypeStructField import za.co.absa.enceladus.utils.validation.{ValidationError, ValidationIssue, ValidationWarning} +import za.co.absa.spark.commons.implicits.StructFieldImplicits.StructFieldMetadataEnhancements import scala.util.{Failure, Success, Try} @@ -32,7 +32,7 @@ object BinaryFieldValidator extends FieldValidator { } private def validateExplicitBase64DefaultValue(field: BinaryTypeStructField): Seq[ValidationIssue] = { - val defaultValue: Option[String] = field.structField.getMetadataString(MetadataKeys.DefaultValue) + val defaultValue: Option[String] = field.structField.metadata.getOptString(MetadataKeys.DefaultValue) (field.normalizedEncoding, defaultValue) match { case (None, Some(encodedDefault)) => diff --git a/utils/src/main/scala/za/co/absa/enceladus/utils/validation/field/DateTimeFieldValidator.scala b/utils/src/main/scala/za/co/absa/enceladus/utils/validation/field/DateTimeFieldValidator.scala index dab476b9e..71541de95 100644 --- a/utils/src/main/scala/za/co/absa/enceladus/utils/validation/field/DateTimeFieldValidator.scala +++ b/utils/src/main/scala/za/co/absa/enceladus/utils/validation/field/DateTimeFieldValidator.scala @@ -25,6 +25,7 @@ import za.co.absa.enceladus.utils.time.DateTimePattern import za.co.absa.enceladus.utils.types.TypedStructField import za.co.absa.enceladus.utils.types.TypedStructField.DateTimeTypeStructField import za.co.absa.enceladus.utils.types.parsers.DateTimeParser +import za.co.absa.spark.commons.implicits.StructFieldImplicits.StructFieldMetadataEnhancements import scala.util.control.NonFatal @@ -40,8 +41,8 @@ abstract class DateTimeFieldValidator extends FieldValidator { private def validateDateTimeTypeStructField(field: DateTimeTypeStructField[_]): Seq[ValidationIssue] = { val result = for { parser <- field.parser - defaultValue: Option[String] = field.getMetadataString(MetadataKeys.DefaultValue) - defaultTimeZone: Option[String] = field.getMetadataString(MetadataKeys.DefaultTimeZone) + defaultValue: Option[String] = field.structField.metadata.getOptString(MetadataKeys.DefaultValue) + defaultTimeZone: Option[String] = field.structField.metadata.getOptString(MetadataKeys.DefaultTimeZone) } yield patternConversionIssues(field, parser).toSeq ++ defaultTimeZoneIssues(defaultTimeZone) ++ patternAnalysisIssues(parser.pattern, defaultValue, defaultTimeZone) diff --git a/utils/src/main/scala/za/co/absa/enceladus/utils/validation/field/FieldValidator.scala b/utils/src/main/scala/za/co/absa/enceladus/utils/validation/field/FieldValidator.scala index fcaa0eb9d..97c7213f0 100644 --- a/utils/src/main/scala/za/co/absa/enceladus/utils/validation/field/FieldValidator.scala +++ b/utils/src/main/scala/za/co/absa/enceladus/utils/validation/field/FieldValidator.scala @@ -17,6 +17,8 @@ package za.co.absa.enceladus.utils.validation.field import za.co.absa.enceladus.utils.types.TypedStructField import za.co.absa.enceladus.utils.validation.{ValidationError, ValidationIssue} +import za.co.absa.spark.commons.implicits.StructFieldImplicits.StructFieldMetadataEnhancements + import scala.util.{Failure, Success, Try} import scala.reflect.runtime.universe._ @@ -52,11 +54,11 @@ class FieldValidator { ) } - if (field.hasMetadataKey(metadataKey)) { + if (field.structField.metadata.hasKey(metadataKey)) { typeOf[T] match { - case t if t =:= typeOf[String] => optionToValidationIssueSeq(field.getMetadataString(metadataKey), t.toString) - case t if t =:= typeOf[Boolean] => optionToValidationIssueSeq(field.getMetadataStringAsBoolean(metadataKey), t.toString) - case t if t =:= typeOf[Char] => optionToValidationIssueSeq(field.getMetadataChar(metadataKey), t.toString) + case t if t =:= typeOf[String] => optionToValidationIssueSeq(field.structField.metadata.getOptString(metadataKey), t.toString) + case t if t =:= typeOf[Boolean] => optionToValidationIssueSeq(field.structField.metadata.getOptStringAsBoolean(metadataKey), t.toString) + case t if t =:= typeOf[Char] => optionToValidationIssueSeq(field.structField.metadata.getOptChar(metadataKey), t.toString) case _ => Seq(ValidationError(s"Unsupported metadata validation type for key '$metadataKey' of field '${field.name}'")) } } else { diff --git a/utils/src/main/scala/za/co/absa/enceladus/utils/validation/field/IntegralFieldValidator.scala b/utils/src/main/scala/za/co/absa/enceladus/utils/validation/field/IntegralFieldValidator.scala index 78d8287b3..da49ef141 100644 --- a/utils/src/main/scala/za/co/absa/enceladus/utils/validation/field/IntegralFieldValidator.scala +++ b/utils/src/main/scala/za/co/absa/enceladus/utils/validation/field/IntegralFieldValidator.scala @@ -18,13 +18,14 @@ import za.co.absa.enceladus.utils.numeric.Radix import za.co.absa.enceladus.utils.schema.MetadataKeys import za.co.absa.enceladus.utils.types.TypedStructField import za.co.absa.enceladus.utils.validation.{ValidationIssue, ValidationWarning} +import za.co.absa.spark.commons.implicits.StructFieldImplicits.StructFieldMetadataEnhancements import scala.util.Try object IntegralFieldValidator extends NumericFieldValidator { private def radixIssues(field: TypedStructField): Seq[ValidationIssue] = { - field.getMetadataString(MetadataKeys.Radix).map { radixString => + field.structField.metadata.getOptString(MetadataKeys.Radix).map { radixString => val result = for { radix <- Try(Radix(radixString)) pattern <- field.pattern diff --git a/utils/src/test/scala/za/co/absa/enceladus/utils/ExplosionSuite.scala b/utils/src/test/scala/za/co/absa/enceladus/utils/ExplosionSuite.scala deleted file mode 100644 index 7d65b0b58..000000000 --- a/utils/src/test/scala/za/co/absa/enceladus/utils/ExplosionSuite.scala +++ /dev/null @@ -1,808 +0,0 @@ -/* - * Copyright 2018 ABSA Group Limited - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package za.co.absa.enceladus.utils - -import org.apache.spark.sql.DataFrame -import org.apache.spark.sql.functions._ -import org.scalatest.funsuite.AnyFunSuite -import org.slf4j.LoggerFactory -import za.co.absa.spark.hats.Extensions._ -import za.co.absa.enceladus.utils.explode.ExplodeTools -import za.co.absa.enceladus.utils.general.JsonUtils -import za.co.absa.enceladus.utils.schema.SchemaUtils -import za.co.absa.enceladus.utils.testUtils.SparkTestBase - -class ExplosionSuite extends AnyFunSuite with SparkTestBase { - - private val logger = LoggerFactory.getLogger(this.getClass) - - import spark.implicits._ - - test("Test explosion of a simple array") { - // An array of 5 elements each having 10 elements - val sampleArray = Range(1, 6).map(a => Range(a, 10 + a).toList).toList - val df = sampleArray.toDF() - - val expectedSchema = """root - | |-- value: integer (nullable = true) - | |-- value_id: long (nullable = false) - | |-- value_size: integer (nullable = false) - | |-- value_idx: integer (nullable = true) - |""".stripMargin.replace("\r\n", "\n") - - val expectedResults = - """+-----+--------+----------+---------+ - ||value|value_id|value_size|value_idx| - |+-----+--------+----------+---------+ - ||1 |0 |10 |0 | - ||2 |0 |10 |1 | - ||3 |0 |10 |2 | - ||4 |0 |10 |3 | - ||5 |0 |10 |4 | - ||6 |0 |10 |5 | - ||7 |0 |10 |6 | - ||8 |0 |10 |7 | - ||9 |0 |10 |8 | - ||10 |0 |10 |9 | - ||2 |1 |10 |0 | - ||3 |1 |10 |1 | - ||4 |1 |10 |2 | - ||5 |1 |10 |3 | - ||6 |1 |10 |4 | - ||7 |1 |10 |5 | - ||8 |1 |10 |6 | - ||9 |1 |10 |7 | - ||10 |1 |10 |8 | - ||11 |1 |10 |9 | - |+-----+--------+----------+---------+ - |only showing top 20 rows - |""".stripMargin.replace("\r\n", "\n") - - - val (explodedDf, explodeContext) = ExplodeTools.explodeArray("value", df) - val actualResults = showString(explodedDf) - - assert(explodeContext.explosions.nonEmpty) - assertSchema(explodedDf.schema.treeString, expectedSchema) - assertResults(actualResults, expectedResults) - } - - test("Test a simple array reconstruction") { - // An array of 5 elements each having 10 elements - val sampleArray = Range(1, 6).map(a => Range(a, 10 + a).toList).toList - val df = sampleArray.toDF().withColumn("static", lit(1)) - - val expectedExplodedSchema = - """root - | |-- value: integer (nullable = true) - | |-- static: integer (nullable = false) - | |-- value_id: long (nullable = false) - | |-- value_size: integer (nullable = false) - | |-- value_idx: integer (nullable = true) - |""".stripMargin.replace("\r\n", "\n") - - val expectedExplodedResults = - """+-----+------+--------+----------+---------+ - ||value|static|value_id|value_size|value_idx| - |+-----+------+--------+----------+---------+ - ||1 |1 |0 |10 |0 | - ||2 |1 |0 |10 |1 | - ||3 |1 |0 |10 |2 | - ||4 |1 |0 |10 |3 | - ||5 |1 |0 |10 |4 | - ||6 |1 |0 |10 |5 | - ||7 |1 |0 |10 |6 | - ||8 |1 |0 |10 |7 | - ||9 |1 |0 |10 |8 | - ||10 |1 |0 |10 |9 | - ||2 |1 |1 |10 |0 | - ||3 |1 |1 |10 |1 | - ||4 |1 |1 |10 |2 | - ||5 |1 |1 |10 |3 | - ||6 |1 |1 |10 |4 | - ||7 |1 |1 |10 |5 | - ||8 |1 |1 |10 |6 | - ||9 |1 |1 |10 |7 | - ||10 |1 |1 |10 |8 | - ||11 |1 |1 |10 |9 | - |+-----+------+--------+----------+---------+ - |only showing top 20 rows - |""".stripMargin.replace("\r\n", "\n") - - val expectedRestoredSchema = - """root - | |-- static: integer (nullable = false) - | |-- value: array (nullable = true) - | | |-- element: integer (containsNull = true) - |""".stripMargin.replace("\r\n", "\n") - - val expectedRestoredResults = - """+------+-----------------------------------+ - ||static|value | - |+------+-----------------------------------+ - ||1 |[1, 2, 3, 4, 5, 6, 7, 8, 9, 10] | - ||1 |[2, 3, 4, 5, 6, 7, 8, 9, 10, 11] | - ||1 |[3, 4, 5, 6, 7, 8, 9, 10, 11, 12] | - ||1 |[4, 5, 6, 7, 8, 9, 10, 11, 12, 13] | - ||1 |[5, 6, 7, 8, 9, 10, 11, 12, 13, 14]| - |+------+-----------------------------------+ - |""".stripMargin.replace("\r\n", "\n") - - - val (explodedDf, explodeContext) = ExplodeTools.explodeArray("value", df) - - val restoredDf = ExplodeTools.revertAllExplosions(explodedDf, explodeContext) - - val actualExplodedResults = showString(explodedDf) - val actualRestoredResults = showString(restoredDf) - - // Checking if explosion has been done correctly - assert(explodeContext.explosions.nonEmpty) - assertSchema(explodedDf.schema.treeString, expectedExplodedSchema) - assertResults(actualExplodedResults, expectedExplodedResults) - - // Checking if restoration has been done correctly - assertSchema(restoredDf.schema.treeString, expectedRestoredSchema) - assertResults(actualRestoredResults, expectedRestoredResults) - } - - test("Test a array of array sequence of explosions") { - // Example provided by Saša Zejnilović - val sampleMatrix = List( - List( - List(1, 2, 3, 4, 5, 6), - List(7, 8, 9, 10, 11, 12, 13) - ), List( - List(201, 202, 203, 204, 205, 206), - List(207, 208, 209, 210, 211, 212, 213) - ), List( - List(301, 302, 303, 304, 305, 306), - List(307, 308, 309, 310, 311, 312, 313) - ), List( - List(401, 402, 403, 404, 405, 406), - List(407, 408, 409, 410, 411, 412, 413) - ) - ) - val df = sampleMatrix.toDF().withColumn("static", lit(1)) - - val expectedExplodedSchema = - """root - | |-- value: integer (nullable = true) - | |-- static: integer (nullable = false) - | |-- value_id: long (nullable = false) - | |-- value_size: integer (nullable = false) - | |-- value_idx: integer (nullable = true) - | |-- value_id_1: long (nullable = false) - | |-- value_size_1: integer (nullable = false) - | |-- value_idx_1: integer (nullable = true) - |""".stripMargin.replace("\r\n", "\n") - - val expectedExplodedResults = - """+-----+------+--------+----------+---------+----------+------------+-----------+ - ||value|static|value_id|value_size|value_idx|value_id_1|value_size_1|value_idx_1| - |+-----+------+--------+----------+---------+----------+------------+-----------+ - ||1 |1 |0 |2 |0 |0 |6 |0 | - ||2 |1 |0 |2 |0 |0 |6 |1 | - ||3 |1 |0 |2 |0 |0 |6 |2 | - ||4 |1 |0 |2 |0 |0 |6 |3 | - ||5 |1 |0 |2 |0 |0 |6 |4 | - ||6 |1 |0 |2 |0 |0 |6 |5 | - ||7 |1 |0 |2 |1 |1 |7 |0 | - ||8 |1 |0 |2 |1 |1 |7 |1 | - ||9 |1 |0 |2 |1 |1 |7 |2 | - ||10 |1 |0 |2 |1 |1 |7 |3 | - |+-----+------+--------+----------+---------+----------+------------+-----------+ - |only showing top 10 rows - |""".stripMargin.replace("\r\n", "\n") - - val expectedRestoredSchema = - """root - | |-- static: integer (nullable = false) - | |-- value: array (nullable = true) - | | |-- element: array (containsNull = true) - | | | |-- element: integer (containsNull = true) - |""".stripMargin.replace("\r\n", "\n") - - val expectedRestoredResults = - """+------+---------------------------------------------------------------------+ - ||static|value | - |+------+---------------------------------------------------------------------+ - ||1 |[[1, 2, 3, 4, 5, 6], [7, 8, 9, 10, 11, 12, 13]] | - ||1 |[[201, 202, 203, 204, 205, 206], [207, 208, 209, 210, 211, 212, 213]]| - ||1 |[[301, 302, 303, 304, 305, 306], [307, 308, 309, 310, 311, 312, 313]]| - ||1 |[[401, 402, 403, 404, 405, 406], [407, 408, 409, 410, 411, 412, 413]]| - |+------+---------------------------------------------------------------------+ - |""".stripMargin.replace("\r\n", "\n") - - val (explodedDf1, explodeContext1) = ExplodeTools.explodeArray("value", df) - val (explodedDf2, explodeContext2) = ExplodeTools.explodeArray("value", explodedDf1, explodeContext1) - - val restoredDf = ExplodeTools.revertAllExplosions(explodedDf2, explodeContext2) - - val actualExplodedResults = showString(explodedDf2, 10) - val actualRestoredResults = showString(restoredDf) - - // Checking if explosion has been done correctly - assert(explodeContext2.explosions.size == 2) - assertSchema(explodedDf2.schema.treeString, expectedExplodedSchema) - assertResults(actualExplodedResults, expectedExplodedResults) - - // Checking if restoration has been done correctly - assertSchema(restoredDf.schema.treeString, expectedRestoredSchema) - assertResults(actualRestoredResults, expectedRestoredResults) - } - - test("Test handling of empty and null arrays") { - val sample = Seq("""{"value":[1,2,3,4,5,6,7,8,9,10],"static":1}""", - """{"value":[2,3,4,5,6,7,8,9,10,11],"static":2}""", - """{"value":[],"static":3}""", - """{"static":4}""") - val df = JsonUtils.getDataFrameFromJson(spark, sample) - - val expectedExplodedSchema = - """root - | |-- static: long (nullable = true) - | |-- value: long (nullable = true) - | |-- value_id: long (nullable = false) - | |-- value_size: integer (nullable = false) - | |-- value_idx: integer (nullable = true) - |""".stripMargin.replace("\r\n", "\n") - - val expectedExplodedResults = - """+------+----------+---------+-----+ - ||static|value_size|value_idx|value| - |+------+----------+---------+-----+ - ||4 |-1 |null |null | - ||3 |0 |null |null | - ||1 |10 |0 |1 | - ||2 |10 |0 |2 | - ||1 |10 |1 |2 | - |+------+----------+---------+-----+ - |only showing top 5 rows - |""".stripMargin.replace("\r\n", "\n") - - val expectedRestoredSchema = - """root - | |-- static: long (nullable = true) - | |-- value: array (nullable = true) - | | |-- element: long (containsNull = true) - |""".stripMargin.replace("\r\n", "\n") - - val expectedRestoredResults = - """+------+--------------------------------+ - ||static|value | - |+------+--------------------------------+ - ||1 |[1, 2, 3, 4, 5, 6, 7, 8, 9, 10] | - ||2 |[2, 3, 4, 5, 6, 7, 8, 9, 10, 11]| - ||3 |[] | - ||4 |null | - |+------+--------------------------------+ - |""".stripMargin.replace("\r\n", "\n") - - - val (explodedDf, explodeContext) = ExplodeTools.explodeArray("value", df) - - val restoredDf = ExplodeTools.revertAllExplosions(explodedDf, explodeContext) - - val actualExplodedResults = showString(explodedDf - .select($"static", $"value_size", $"value_idx", $"value") - .orderBy($"value_size", $"value_idx", $"static"), 5) - val actualRestoredResults = showString(restoredDf) - - // Checking if explosion has been done correctly - assert(explodeContext.explosions.nonEmpty) - assertSchema(explodedDf.schema.treeString, expectedExplodedSchema) - assertResults(actualExplodedResults, expectedExplodedResults) - - // Checking if restoration has been done correctly - assertSchema(restoredDf.schema.treeString, expectedRestoredSchema) - assertResults(actualRestoredResults, expectedRestoredResults) - } - - test("Test deconstruct()") { - val sample = """{"id":1,"leg":{"legid":100,"conditions":[{"check":"a","action":"b"},{"check":"c","action":"d"},{"check":"e","action":"f"}]}}""" :: - """{"id":2,"leg":{"legid":200,"conditions":[{"check":"g","action":"h"},{"check":"i","action":"j"},{"check":"k","action":"l"}]}}""" :: - """{"id":3,"leg":{"legid":300,"conditions":[]}}""" :: - """{"id":4,"leg":{"legid":400}}""" :: Nil - - val df = JsonUtils.getDataFrameFromJson(spark, sample) - - val expectedDeconstructedSchema = - """root - | |-- id: long (nullable = true) - | |-- leg: struct (nullable = false) - | | |-- quark: integer (nullable = false) - | | |-- legid: long (nullable = true) - | |-- electron: array (nullable = true) - | | |-- element: struct (containsNull = true) - | | | |-- action: string (nullable = true) - | | | |-- check: string (nullable = true) - |""".stripMargin.replace("\r\n", "\n") - - val expectedDeconstructedData = - """+---+--------+------------------------+ - ||id |leg |electron | - |+---+--------+------------------------+ - ||1 |[0, 100]|[[b, a], [d, c], [f, e]]| - ||2 |[0, 200]|[[h, g], [j, i], [l, k]]| - ||3 |[0, 300]|[] | - ||4 |[0, 400]|null | - |+---+--------+------------------------+ - |""".stripMargin.replace("\r\n", "\n") - - val expectedRestoredSchema = - """root - | |-- id: long (nullable = true) - | |-- leg: struct (nullable = false) - | | |-- conditions: array (nullable = true) - | | | |-- element: struct (containsNull = true) - | | | | |-- action: string (nullable = true) - | | | | |-- check: string (nullable = true) - | | |-- legid: long (nullable = true) - |""".stripMargin.replace("\r\n", "\n") - - val expectedRestoredData = - """+---+-------------------------------+ - ||id |leg | - |+---+-------------------------------+ - ||1 |[[[b, a], [d, c], [f, e]], 100]| - ||2 |[[[h, g], [j, i], [l, k]], 200]| - ||3 |[[], 300] | - ||4 |[, 400] | - |+---+-------------------------------+ - |""".stripMargin.replace("\r\n", "\n") - - val d = ExplodeTools.deconstructNestedColumn(df, "leg.conditions") - val (df2, deconstructedCol, transientCol) = ExplodeTools.DeconstructedNestedField.unapply(d).get - - val df3 = ExplodeTools.nestedRenameReplace(df2, deconstructedCol, "leg.conditions", transientCol) - - val actualDeconstructedResults = showString(df2, 5) - val actualRestoredResults = showString(df3, 5) - - assertSchema(df2.schema.treeString, expectedDeconstructedSchema) - assertResults(actualDeconstructedResults, expectedDeconstructedData) - - assertSchema(df3.schema.treeString, expectedRestoredSchema) - assertResults(actualRestoredResults, expectedRestoredData) - } - - test ("Test multiple nesting of arrays and structs") { - val sample = """{"id":1,"legs":[{"legid":100,"conditions":[{"checks":[{"checkNums":["1","2","3b","4","5c","6"]}],"amount":100}]}]}""" :: - """{"id":2,"legs":[{"legid":200,"conditions":[{"checks":[{"checkNums":["8","9","10b","11","12c","13"]}],"amount":200}]}]}""" :: - """{"id":3,"legs":[{"legid":300,"conditions":[{"checks":[],"amount": 300}]}]}""" :: - """{"id":4,"legs":[{"legid":400,"conditions":[{"checks":null,"amount": 400}]}]}""" :: - """{"id":5,"legs":[{"legid":500,"conditions":[]}]}""" :: - """{"id":6,"legs":[]}""" :: - """{"id":7}""" :: Nil - - val df = JsonUtils.getDataFrameFromJson(spark, sample) - - val expectedOriginalSchema = - """root - | |-- id: long (nullable = true) - | |-- legs: array (nullable = true) - | | |-- element: struct (containsNull = true) - | | | |-- conditions: array (nullable = true) - | | | | |-- element: struct (containsNull = true) - | | | | | |-- amount: long (nullable = true) - | | | | | |-- checks: array (nullable = true) - | | | | | | |-- element: struct (containsNull = true) - | | | | | | | |-- checkNums: array (nullable = true) - | | | | | | | | |-- element: string (containsNull = true) - | | | |-- legid: long (nullable = true) - |""".stripMargin.replace("\r\n", "\n") - - val expectedOriginalResults = - """+---+----------------------------------------------+ - ||id |legs | - |+---+----------------------------------------------+ - ||1 |[[[[100, [[[1, 2, 3b, 4, 5c, 6]]]]], 100]] | - ||2 |[[[[200, [[[8, 9, 10b, 11, 12c, 13]]]]], 200]]| - ||3 |[[[[300, []]], 300]] | - ||4 |[[[[400,]], 400]] | - ||5 |[[[], 500]] | - ||6 |[] | - ||7 |null | - |+---+----------------------------------------------+ - |""".stripMargin.replace("\r\n", "\n") - - val expectedExplodedSchema = - """root - | |-- id: long (nullable = true) - | |-- legs: struct (nullable = false) - | | |-- conditions: struct (nullable = false) - | | | |-- amount: long (nullable = true) - | | | |-- checks: struct (nullable = false) - | | | | |-- checkNums: string (nullable = true) - | | | | |-- higgs: null (nullable = true) - | | |-- legid: long (nullable = true) - | |-- legs_id: long (nullable = false) - | |-- legs_size: integer (nullable = false) - | |-- legs_idx: integer (nullable = true) - | |-- legs_conditions_id: long (nullable = false) - | |-- legs_conditions_size: integer (nullable = false) - | |-- legs_conditions_idx: integer (nullable = true) - | |-- legs_conditions_checks_id: long (nullable = false) - | |-- legs_conditions_checks_size: integer (nullable = false) - | |-- legs_conditions_checks_idx: integer (nullable = true) - | |-- legs_conditions_checks_checkNums_id: long (nullable = false) - | |-- legs_conditions_checks_checkNums_size: integer (nullable = false) - | |-- legs_conditions_checks_checkNums_idx: integer (nullable = true) - |""".stripMargin.replace("\r\n", "\n") - - val expectedExplodedResults = - """+------+----------+---------+-----+ - ||static|value_size|value_idx|value| - |+------+----------+---------+-----+ - ||4 |-1 |null |null | - ||3 |0 |null |null | - ||1 |10 |0 |1 | - ||2 |10 |0 |2 | - ||1 |10 |1 |2 | - |+------+----------+---------+-----+ - |only showing top 5 rows - |""".stripMargin.replace("\r\n", "\n") - - val expectedRestoredSchema = - """root - | |-- id: long (nullable = true) - | |-- legs: array (nullable = true) - | | |-- element: struct (containsNull = true) - | | | |-- conditions: array (nullable = true) - | | | | |-- element: struct (containsNull = true) - | | | | | |-- amount: long (nullable = true) - | | | | | |-- checks: array (nullable = true) - | | | | | | |-- element: struct (containsNull = true) - | | | | | | | |-- checkNums: array (nullable = true) - | | | | | | | | |-- element: string (containsNull = true) - | | | |-- legid: long (nullable = true) - |""".stripMargin.replace("\r\n", "\n") - - val expectedRestoredResults = - """+---+----------------------------------------------+ - ||id |legs | - |+---+----------------------------------------------+ - ||1 |[[[[100, [[[1, 2, 3b, 4, 5c, 6]]]]], 100]] | - ||2 |[[[[200, [[[8, 9, 10b, 11, 12c, 13]]]]], 200]]| - ||3 |[[[[300, []]], 300]] | - ||4 |[[[[400,]], 400]] | - ||5 |[[[], 500]] | - ||6 |[] | - ||7 |null | - |+---+----------------------------------------------+ - |""".stripMargin.replace("\r\n", "\n") - - - - val (explodedDf1, explodeContext1) = ExplodeTools.explodeArray("legs", df) - val (explodedDf2, explodeContext2) = ExplodeTools.explodeArray("legs.conditions", explodedDf1, explodeContext1) - val (explodedDf3, explodeContext3) = ExplodeTools.explodeArray("legs.conditions.checks", explodedDf2, explodeContext2) - val (explodedDf4, explodeContext4) = ExplodeTools.explodeArray("legs.conditions.checks.checkNums", explodedDf3, explodeContext3) - - val explodeConditionFilter = explodeContext4.getControlFrameworkFilter - val expectedExplodeFilter = "((((true AND (coalesce(legs_conditions_checks_checkNums_idx, 0) = 0)) AND (coalesce(legs_conditions_checks_idx, 0) = 0)) AND (coalesce(legs_conditions_idx, 0) = 0)) AND (coalesce(legs_idx, 0) = 0))" - - val restoredDf = ExplodeTools.revertAllExplosions(explodedDf4, explodeContext4) - - val actualOriginalResults = showString(df) - val actualRestoredResults = showString(restoredDf) - - assert(SchemaUtils.isNonNestedArray(df.schema, "legs")) - assert(!SchemaUtils.isNonNestedArray(df.schema, "legs.conditions")) - assert(!SchemaUtils.isNonNestedArray(df.schema, "legs.conditions.checks")) - assert(!SchemaUtils.isNonNestedArray(df.schema, "legs.conditions.checks.checkNums")) - assert(!SchemaUtils.isNonNestedArray(df.schema, "id")) - assert(!SchemaUtils.isNonNestedArray(df.schema, "legs.legid")) - - assertSchema(df.schema.treeString, expectedOriginalSchema) - assertResults(actualOriginalResults, expectedOriginalResults) - - val actualExplodedSchema = explodedDf4.schema.treeString.replaceAll("higgs_\\d+","higgs") - assertSchema(actualExplodedSchema, expectedExplodedSchema) - assert(explodedDf4.count() == 17) - - assertSchema(restoredDf.schema.treeString, expectedRestoredSchema) - assertResults(actualRestoredResults, expectedRestoredResults) - - // Check the filter generator as well - assert(explodeConditionFilter.toString == expectedExplodeFilter) - } - - test ("Test exploding a nested array that is the only element of a struct") { - val sample = """{"id":1,"leg":{"conditions":[{"check":"a","action":"b"},{"check":"c","action":"d"},{"check":"e","action":"f"}]}}""" :: - """{"id":2,"leg":{"conditions":[{"check":"g","action":"h"},{"check":"i","action":"j"},{"check":"k","action":"l"}]}}""" :: - """{"id":3,"leg":{"conditions":[]}}""" :: - """{"id":4}""" :: Nil - - val df = JsonUtils.getDataFrameFromJson(spark, sample) - - val expectedOriginalSchema = - """root - | |-- id: long (nullable = true) - | |-- leg: struct (nullable = true) - | | |-- conditions: array (nullable = true) - | | | |-- element: struct (containsNull = true) - | | | | |-- action: string (nullable = true) - | | | | |-- check: string (nullable = true) - |""".stripMargin.replace("\r\n", "\n") - - val expectedOriginalResults = - """+---+--------------------------+ - ||id |leg | - |+---+--------------------------+ - ||1 |[[[b, a], [d, c], [f, e]]]| - ||2 |[[[h, g], [j, i], [l, k]]]| - ||3 |[[]] | - ||4 |null | - |+---+--------------------------+ - |""".stripMargin.replace("\r\n", "\n") - - val expectedExplodedSchema = - """root - | |-- id: long (nullable = true) - | |-- leg: struct (nullable = false) - | | |-- conditions: struct (nullable = true) - | | | |-- action: string (nullable = true) - | | | |-- check: string (nullable = true) - | | |-- higgs: null (nullable = true) - | |-- leg_conditions_id: long (nullable = false) - | |-- leg_conditions_size: integer (nullable = false) - | |-- leg_conditions_idx: integer (nullable = true) - |""".stripMargin.replace("\r\n", "\n") - - val expectedRestoredSchema = - """root - | |-- id: long (nullable = true) - | |-- leg: struct (nullable = false) - | | |-- conditions: array (nullable = true) - | | | |-- element: struct (containsNull = true) - | | | | |-- action: string (nullable = true) - | | | | |-- check: string (nullable = true) - |""".stripMargin.replace("\r\n", "\n") - - val expectedRestoredResults = - """+---+--------------------------+ - ||id |leg | - |+---+--------------------------+ - ||1 |[[[b, a], [d, c], [f, e]]]| - ||2 |[[[h, g], [j, i], [l, k]]]| - ||3 |[[]] | - ||4 |[] | - |+---+--------------------------+ - |""".stripMargin.replace("\r\n", "\n") - - - val (explodedDf, explodeContext) = ExplodeTools.explodeArray("leg.conditions", df) - val restoredDf = ExplodeTools.revertAllExplosions(explodedDf, explodeContext) - - val actualOriginalResults = showString(df) - val actualRestoredResults = showString(restoredDf) - - assertSchema(df.schema.treeString, expectedOriginalSchema) - assertResults(actualOriginalResults, expectedOriginalResults) - - val actualExplodedSchema = explodedDf.schema.treeString.replaceAll("higgs_\\d+","higgs") - assertSchema(actualExplodedSchema, expectedExplodedSchema) - assert(explodedDf.count() == 8) - - assertSchema(restoredDf.schema.treeString, expectedRestoredSchema) - assertResults(actualRestoredResults, expectedRestoredResults) - } - - test ("Test explosion of an array field inside a struct") { - val sample = """{"id":1,"leg":{"legid":100,"conditions":[{"check":"a","action":"b"},{"check":"c","action":"d"},{"check":"e","action":"f"}]}}""" :: - """{"id":2,"leg":{"legid":200,"conditions":[{"check":"g","action":"h"},{"check":"i","action":"j"},{"check":"k","action":"l"}]}}""" :: - """{"id":3,"leg":{"legid":300,"conditions":[]}}""" :: - """{"id":4,"leg":{"legid":400}}""" :: Nil - - val df = JsonUtils.getDataFrameFromJson(spark, sample) - - val (explodedDf, explodeContext) = ExplodeTools.explodeArray("leg.conditions", df) - val restoredDf = ExplodeTools.revertAllExplosions(explodedDf, explodeContext) - - val expectedSchema = - """root - | |-- id: long (nullable = true) - | |-- leg: struct (nullable = false) - | | |-- conditions: array (nullable = true) - | | | |-- element: struct (containsNull = true) - | | | | |-- action: string (nullable = true) - | | | | |-- check: string (nullable = true) - | | |-- legid: long (nullable = true) - |""".stripMargin.replace("\r\n", "\n") - - val expectedData = - """+---+-------------------------------+ - ||id |leg | - |+---+-------------------------------+ - ||1 |[[[b, a], [d, c], [f, e]], 100]| - ||2 |[[[h, g], [j, i], [l, k]], 200]| - ||3 |[[], 300] | - ||4 |[, 400] | - |+---+-------------------------------+ - |""".stripMargin.replace("\r\n", "\n") - - val actualResults = showString(restoredDf, 5) - - assertSchema(restoredDf.schema.treeString, expectedSchema) - assertResults(actualResults, expectedData) - } - - test ("Test explosion with an error column") { - val sample = """{"id":1,"errors":["Error 1","Error 2"],"leg":{"legid":100,"conditions":[{"check":"1","action":"b"},{"check":"2","action":"d"},{"check":"3","action":"f"}]}}""" :: - """{"id":2,"errors":[],"leg":{"legid":200,"conditions":[{"check":"0","action":"b"}]}}""" :: - """{"id":3,"errors":[],"leg":{"legid":300}}""" :: Nil - - val df = JsonUtils.getDataFrameFromJson(spark, sample) - - val (explodedDf, explodeContext) = ExplodeTools.explodeArray("leg.conditions", df) - - // Manupilate error column - val changedDf = explodedDf.select(concat($"errors", array($"leg.conditions.check")).as("errors"), - $"id", $"leg", $"leg_conditions_id", $"leg_conditions_size", $"leg_conditions_idx") - - val restoredDf = ExplodeTools.revertAllExplosions(changedDf, explodeContext, Some("errors")) - - val expectedSchema = - """root - | |-- id: long (nullable = true) - | |-- leg: struct (nullable = false) - | | |-- conditions: array (nullable = true) - | | | |-- element: struct (containsNull = true) - | | | | |-- action: string (nullable = true) - | | | | |-- check: string (nullable = true) - | | |-- legid: long (nullable = true) - | |-- errors: array (nullable = true) - | | |-- element: string (containsNull = true) - |""".stripMargin.replace("\r\n", "\n") - - val expectedData = - """+---+-------------------------------+---------------------------+ - ||id |leg |errors | - |+---+-------------------------------+---------------------------+ - ||1 |[[[b, 1], [d, 2], [f, 3]], 100]|[Error 1, Error 2, 1, 2, 3]| - ||2 |[[[b, 0]], 200] |[0] | - ||3 |[, 300] |[] | - |+---+-------------------------------+---------------------------+ - |""".stripMargin.replace("\r\n", "\n") - - val actualResults = showString(restoredDf, 5) - - assertSchema(restoredDf.schema.treeString, expectedSchema) - assertResults(actualResults, expectedData) - } - - test ("Test empty struct inside an array") { - val sample = """{"order":1,"a":[{"b":"H1","c":[{"d":1,"toDrop": "drop me"}]}],"myFlag":true}""" :: - """{"order":2,"a":[{"b":"H2","c":[]}],"myFlag":true}""" :: - """{"order":3,"a":[{"b":"H3"}],"myFlag":true}""" :: - """{"order":4,"a":[{}],"myFlag":true}""" :: - """{"order":5,"a":[],"myFlag":true}""" :: - """{"order":6,"myFlag":true}""" :: Nil - - val df = JsonUtils.getDataFrameFromJson(spark, sample) - - val (explodedDf1, explodeContext1) = ExplodeTools.explodeArray("a", df) - val (explodedDf2, explodeContext2) = ExplodeTools.explodeArray("a.c", explodedDf1, explodeContext1) - - // Manipulate the exploded structs - val changedDf = explodedDf2.nestedDropColumn("a.c.toDrop") - - val restoredDf = ExplodeTools.revertAllExplosions(changedDf, explodeContext2) - - val expectedSchema = - """root - | |-- myFlag: boolean (nullable = true) - | |-- order: long (nullable = true) - | |-- a: array (nullable = true) - | | |-- element: struct (containsNull = true) - | | | |-- b: string (nullable = true) - | | | |-- c: array (nullable = true) - | | | | |-- element: struct (containsNull = true) - | | | | | |-- d: long (nullable = true) - |""".stripMargin.replace("\r\n", "\n") - - val expectedData = - """+------+-----+-------------+ - ||myFlag|order|a | - |+------+-----+-------------+ - ||true |1 |[[H1, [[1]]]]| - ||true |2 |[[H2, []]] | - ||true |3 |[[H3,]] | - ||true |4 |[[,]] | - ||true |5 |[] | - ||true |6 |null | - |+------+-----+-------------+ - |""".stripMargin.replace("\r\n", "\n") - - val actualResults = showString(restoredDf, 10) - - assertSchema(restoredDf.schema.treeString, expectedSchema) - assertResults(actualResults, expectedData) - } - - test ("Test empty struct inside an array with the only array field") { - val sample = """{"order":1,"a":[{"c":[{"d":1}]}],"myFlag":true}""" :: - """{"order":2,"a":[{"c":[]}],"myFlag":true}""" :: - """{"order":3,"a":[{}],"myFlag":true}""" :: - """{"order":4,"a":[],"myFlag":true}""" :: - """{"order":5,"myFlag":true}""" :: Nil - - val df = JsonUtils.getDataFrameFromJson(spark, sample) - - val (explodedDf1, explodeContext1) = ExplodeTools.explodeArray("a", df) - - val (explodedDf2, explodeContext2) = ExplodeTools.explodeArray("a.c", explodedDf1, explodeContext1) - - val restoredDf = ExplodeTools.revertAllExplosions(explodedDf2, explodeContext2) - - val expectedSchema = - """root - | |-- myFlag: boolean (nullable = true) - | |-- order: long (nullable = true) - | |-- a: array (nullable = true) - | | |-- element: struct (containsNull = true) - | | | |-- c: array (nullable = true) - | | | | |-- element: struct (containsNull = true) - | | | | | |-- d: long (nullable = true) - |""".stripMargin.replace("\r\n", "\n") - - val expectedData = - """+------+-----+---------+ - ||myFlag|order|a | - |+------+-----+---------+ - ||true |1 |[[[[1]]]]| - ||true |2 |[[[]]] | - ||true |3 |[[]] | - ||true |4 |[] | - ||true |5 |null | - |+------+-----+---------+ - |""".stripMargin.replace("\r\n", "\n") - - val actualResults = showString(restoredDf, 10) - - assertSchema(restoredDf.schema.treeString, expectedSchema) - assertResults(actualResults, expectedData) - } - - // Call showString() by reflection since it is private - // Thanks https://stackoverflow.com/a/51218800/1038282 - private def showString(df: DataFrame, numRows: Int = 20): String = { - val showString = classOf[org.apache.spark.sql.DataFrame].getDeclaredMethod("showString", - classOf[Int], classOf[Int], classOf[Boolean]) - showString.setAccessible(true) - showString.invoke(df, numRows.asInstanceOf[Object], 0.asInstanceOf[Object], - false.asInstanceOf[Object]).asInstanceOf[String] - } - - private def assertSchema(actualSchema: String, expectedSchema: String): Unit = { - if (actualSchema != expectedSchema) { - logger.error(s"EXPECTED:\n$expectedSchema") - logger.error(s"ACTUAL:\n$actualSchema") - fail("Actual conformed schema does not match the expected schema (see above).") - } - } - - private def assertResults(actualResults: String, expectedResults: String): Unit = { - if (actualResults != expectedResults) { - logger.error(s"EXPECTED:\n$expectedResults") - logger.error(s"ACTUAL:\n$actualResults") - fail("Actual conformed dataset data does not match the expected data (see above).") - } - } - -} diff --git a/utils/src/test/scala/za/co/absa/enceladus/utils/broadcast/BroadcastUtilsSuite.scala b/utils/src/test/scala/za/co/absa/enceladus/utils/broadcast/BroadcastUtilsSuite.scala index 00b45faff..e8329a480 100644 --- a/utils/src/test/scala/za/co/absa/enceladus/utils/broadcast/BroadcastUtilsSuite.scala +++ b/utils/src/test/scala/za/co/absa/enceladus/utils/broadcast/BroadcastUtilsSuite.scala @@ -19,7 +19,8 @@ import org.apache.spark.sql.functions._ import org.apache.spark.sql.{DataFrame, Row} import org.scalatest.wordspec.AnyWordSpec import za.co.absa.enceladus.utils.error.Mapping -import za.co.absa.enceladus.utils.testUtils.{LoggerTestBase, SparkTestBase} +import za.co.absa.enceladus.utils.testUtils.LoggerTestBase +import za.co.absa.spark.commons.test.SparkTestBase import scala.collection.mutable diff --git a/utils/src/test/scala/za/co/absa/enceladus/utils/broadcast/LocalMappingTableSuite.scala b/utils/src/test/scala/za/co/absa/enceladus/utils/broadcast/LocalMappingTableSuite.scala index 250d96a21..c1992b830 100644 --- a/utils/src/test/scala/za/co/absa/enceladus/utils/broadcast/LocalMappingTableSuite.scala +++ b/utils/src/test/scala/za/co/absa/enceladus/utils/broadcast/LocalMappingTableSuite.scala @@ -18,8 +18,8 @@ package za.co.absa.enceladus.utils.broadcast import org.apache.spark.sql.Row import org.apache.spark.sql.types.NumericType import org.scalatest.wordspec.AnyWordSpec -import za.co.absa.enceladus.utils.general.JsonUtils -import za.co.absa.enceladus.utils.testUtils.SparkTestBase +import za.co.absa.spark.commons.utils.JsonUtils +import za.co.absa.spark.commons.test.SparkTestBase class LocalMappingTableSuite extends AnyWordSpec with SparkTestBase { diff --git a/utils/src/test/scala/za/co/absa/enceladus/utils/fs/HdfsUtilsSpec.scala b/utils/src/test/scala/za/co/absa/enceladus/utils/fs/HdfsUtilsSpec.scala index 5e32622a0..dd401727a 100644 --- a/utils/src/test/scala/za/co/absa/enceladus/utils/fs/HdfsUtilsSpec.scala +++ b/utils/src/test/scala/za/co/absa/enceladus/utils/fs/HdfsUtilsSpec.scala @@ -20,7 +20,8 @@ import java.io.FileNotFoundException import org.apache.hadoop.fs.Path import org.scalatest.matchers.should.Matchers import org.scalatest.wordspec.AnyWordSpec -import za.co.absa.enceladus.utils.testUtils.{HadoopFsTestBase, SparkTestBase} +import za.co.absa.enceladus.utils.testUtils.HadoopFsTestBase +import za.co.absa.spark.commons.test.SparkTestBase /** * Unit tests for File system utils diff --git a/utils/src/test/scala/za/co/absa/enceladus/utils/general/JsonUtilsSuite.scala b/utils/src/test/scala/za/co/absa/enceladus/utils/general/JsonUtilsSuite.scala deleted file mode 100644 index 5204f398a..000000000 --- a/utils/src/test/scala/za/co/absa/enceladus/utils/general/JsonUtilsSuite.scala +++ /dev/null @@ -1,63 +0,0 @@ -/* - * Copyright 2018 ABSA Group Limited - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package za.co.absa.enceladus.utils.general - -import org.scalatest.funsuite.AnyFunSuite -import za.co.absa.enceladus.utils.testUtils.SparkTestBase - -class JsonUtilsSuite extends AnyFunSuite with SparkTestBase { - test("Test JSON pretty formatting from a JSON string") { - val inputJson = """[{"id":1,"items":[{"itemid":100,"subitems":[{"elems":[{"numbers":["1","2","3b","4","5c","6"]}],"code":100}]}]}]""" - val expected = """[ { - | "id" : 1, - | "items" : [ { - | "itemid" : 100, - | "subitems" : [ { - | "elems" : [ { - | "numbers" : [ "1", "2", "3b", "4", "5c", "6" ] - | } ], - | "code" : 100 - | } ] - | } ] - |} ]""".stripMargin.replace("\r\n", "\n") - - val actual = JsonUtils.prettyJSON(inputJson) - - assert(expected == actual) - } - - test("Test JSON pretty formatting from a Spark JSON string") { - val inputJsons = Seq("""{"value": 1}""", """{"value": 2}""") - val expected = "[ {\n \"value\" : 1\n}, {\n \"value\" : 2\n} ]" - - val actual = JsonUtils.prettySparkJSON(inputJsons) - - assert(expected == actual) - } - - test("Test a dataframe created from a JSON") { - val inputJson = Seq("""{"value":1}""", """{"value":2}""") - - val df = JsonUtils.getDataFrameFromJson(spark, inputJson) - - val expectedSchema = """root - | |-- value: long (nullable = true) - |""".stripMargin.replace("\r\n", "\n") - val actualSchema = df.schema.treeString - - assert(expectedSchema == actualSchema) - } -} diff --git a/utils/src/test/scala/za/co/absa/enceladus/utils/implicits/DataFrameImplicitsSuite.scala b/utils/src/test/scala/za/co/absa/enceladus/utils/implicits/DataFrameImplicitsSuite.scala index 9b944fe35..9a4fab21e 100644 --- a/utils/src/test/scala/za/co/absa/enceladus/utils/implicits/DataFrameImplicitsSuite.scala +++ b/utils/src/test/scala/za/co/absa/enceladus/utils/implicits/DataFrameImplicitsSuite.scala @@ -16,8 +16,8 @@ package za.co.absa.enceladus.utils.implicits import org.scalatest.funsuite.AnyFunSuite -import za.co.absa.enceladus.utils.implicits.DataFrameImplicits.DataFrameEnhancements -import za.co.absa.enceladus.utils.testUtils.SparkTestBase +import za.co.absa.spark.commons.implicits.DataFrameImplicits.DataFrameEnhancements +import za.co.absa.spark.commons.test.SparkTestBase class DataFrameImplicitsSuite extends AnyFunSuite with SparkTestBase { import spark.implicits._ diff --git a/utils/src/test/scala/za/co/absa/enceladus/utils/schema/SchemaUtilsSuite.scala b/utils/src/test/scala/za/co/absa/enceladus/utils/schema/SchemaUtilsSuite.scala deleted file mode 100644 index bd41f998e..000000000 --- a/utils/src/test/scala/za/co/absa/enceladus/utils/schema/SchemaUtilsSuite.scala +++ /dev/null @@ -1,476 +0,0 @@ -/* - * Copyright 2018 ABSA Group Limited - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package za.co.absa.enceladus.utils.schema - -import org.apache.spark.sql.types._ -import org.scalatest.funsuite.AnyFunSuite -import org.scalatest.matchers.should.Matchers -import za.co.absa.enceladus.utils.schema.SchemaUtils._ - -class SchemaUtilsSuite extends AnyFunSuite with Matchers { - // scalastyle:off magic.number - - private val schema = StructType(Seq( - StructField("a", IntegerType, nullable = false), - StructField("b", StructType(Seq( - StructField("c", IntegerType), - StructField("d", StructType(Seq( - StructField("e", IntegerType))), nullable = true)))), - StructField("f", StructType(Seq( - StructField("g", ArrayType.apply(StructType(Seq( - StructField("h", IntegerType)))))))))) - - private val nestedSchema = StructType(Seq( - StructField("a", IntegerType), - StructField("b", ArrayType(StructType(Seq( - StructField("c", StructType(Seq( - StructField("d", ArrayType(StructType(Seq( - StructField("e", IntegerType)))))))))))))) - - private val arrayOfArraysSchema = StructType(Seq( - StructField("a", ArrayType(ArrayType(IntegerType)), nullable = false), - StructField("b", ArrayType(ArrayType(StructType(Seq( - StructField("c", StringType, nullable = false) - )) - )), nullable = true) - )) - - private val structFieldNoMetadata = StructField("a", IntegerType) - - private val structFieldWithMetadataNotSourceColumn = StructField("a", IntegerType, nullable = false, new MetadataBuilder().putString("meta", "data").build) - private val structFieldWithMetadataSourceColumn = StructField("a", IntegerType, nullable = false, new MetadataBuilder().putString("sourcecolumn", "override_a").build) - - test("Testing getFieldType") { - - val a = getFieldType("a", schema) - val b = getFieldType("b", schema) - val c = getFieldType("b.c", schema) - val d = getFieldType("b.d", schema) - val e = getFieldType("b.d.e", schema) - val f = getFieldType("f", schema) - val g = getFieldType("f.g", schema) - val h = getFieldType("f.g.h", schema) - - assert(a.get.isInstanceOf[IntegerType]) - assert(b.get.isInstanceOf[StructType]) - assert(c.get.isInstanceOf[IntegerType]) - assert(d.get.isInstanceOf[StructType]) - assert(e.get.isInstanceOf[IntegerType]) - assert(f.get.isInstanceOf[StructType]) - assert(g.get.isInstanceOf[ArrayType]) - assert(h.get.isInstanceOf[IntegerType]) - assert(getFieldType("z", schema).isEmpty) - assert(getFieldType("x.y.z", schema).isEmpty) - assert(getFieldType("f.g.h.a", schema).isEmpty) - } - - test("Testing fieldExists") { - assert(fieldExists("a", schema)) - assert(fieldExists("b", schema)) - assert(fieldExists("b.c", schema)) - assert(fieldExists("b.d", schema)) - assert(fieldExists("b.d.e", schema)) - assert(fieldExists("f", schema)) - assert(fieldExists("f.g", schema)) - assert(fieldExists("f.g.h", schema)) - assert(!fieldExists("z", schema)) - assert(!fieldExists("x.y.z", schema)) - assert(!fieldExists("f.g.h.a", schema)) - } - - test ("Test isColumnArrayOfStruct") { - assert(!isColumnArrayOfStruct("a", schema)) - assert(!isColumnArrayOfStruct("b", schema)) - assert(!isColumnArrayOfStruct("b.c", schema)) - assert(!isColumnArrayOfStruct("b.d", schema)) - assert(!isColumnArrayOfStruct("b.d.e", schema)) - assert(!isColumnArrayOfStruct("f", schema)) - assert(isColumnArrayOfStruct("f.g", schema)) - assert(!isColumnArrayOfStruct("f.g.h", schema)) - assert(!isColumnArrayOfStruct("a", nestedSchema)) - assert(isColumnArrayOfStruct("b", nestedSchema)) - assert(isColumnArrayOfStruct("b.c.d", nestedSchema)) - } - - test("getRenamesInSchema - no renames") { - val result = getRenamesInSchema(StructType(Seq( - structFieldNoMetadata, - structFieldWithMetadataNotSourceColumn))) - assert(result.isEmpty) - } - - test("getRenamesInSchema - simple rename") { - val result = getRenamesInSchema(StructType(Seq(structFieldWithMetadataSourceColumn))) - assert(result == Map("a" -> "override_a")) - - } - - test("getRenamesInSchema - complex with includeIfPredecessorChanged set") { - val sub = StructType(Seq( - StructField("d", IntegerType, nullable = false, new MetadataBuilder().putString("sourcecolumn", "o").build), - StructField("e", IntegerType, nullable = false, new MetadataBuilder().putString("sourcecolumn", "e").build), - StructField("f", IntegerType) - )) - val schema = StructType(Seq( - StructField("a", sub, nullable = false, new MetadataBuilder().putString("sourcecolumn", "x").build), - StructField("b", sub, nullable = false, new MetadataBuilder().putString("sourcecolumn", "b").build), - StructField("c", sub) - )) - - val includeIfPredecessorChanged = true - val result = getRenamesInSchema(schema, includeIfPredecessorChanged) - val expected = Map( - "a" -> "x" , - "a.d" -> "x.o", - "a.e" -> "x.e", - "a.f" -> "x.f", - "b.d" -> "b.o", - "c.d" -> "c.o" - ) - - assert(result == expected) - } - - test("getRenamesInSchema - complex with includeIfPredecessorChanged not set") { - val sub = StructType(Seq( - StructField("d", IntegerType, nullable = false, new MetadataBuilder().putString("sourcecolumn", "o").build), - StructField("e", IntegerType, nullable = false, new MetadataBuilder().putString("sourcecolumn", "e").build), - StructField("f", IntegerType) - )) - val schema = StructType(Seq( - StructField("a", sub, nullable = false, new MetadataBuilder().putString("sourcecolumn", "x").build), - StructField("b", sub, nullable = false, new MetadataBuilder().putString("sourcecolumn", "b").build), - StructField("c", sub) - )) - - val includeIfPredecessorChanged = false - val result = getRenamesInSchema(schema, includeIfPredecessorChanged) - val expected = Map( - "a" -> "x", - "a.d" -> "x.o", - "b.d" -> "b.o", - "c.d" -> "c.o" - ) - - assert(result == expected) - } - - - test("getRenamesInSchema - array") { - val sub = StructType(Seq( - StructField("renamed", IntegerType, nullable = false, new MetadataBuilder().putString("sourcecolumn", "rename source").build), - StructField("same", IntegerType, nullable = false, new MetadataBuilder().putString("sourcecolumn", "same").build), - StructField("f", IntegerType) - )) - val schema = StructType(Seq( - StructField("array1", ArrayType(sub)), - StructField("array2", ArrayType(ArrayType(ArrayType(sub)))), - StructField("array3", ArrayType(IntegerType), nullable = false, new MetadataBuilder().putString("sourcecolumn", "array source").build) - )) - - val includeIfPredecessorChanged = false - val result = getRenamesInSchema(schema, includeIfPredecessorChanged) - val expected = Map( - "array1.renamed" -> "array1.rename source", - "array2.renamed" -> "array2.rename source", - "array3" -> "array source" - ) - - assert(result == expected) - } - - - test("getRenamesInSchema - source column used multiple times") { - val sub = StructType(Seq( - StructField("x", IntegerType, nullable = false, new MetadataBuilder().putString("sourcecolumn", "src").build), - StructField("y", IntegerType, nullable = false, new MetadataBuilder().putString("sourcecolumn", "src").build) - )) - val schema = StructType(Seq( - StructField("a", sub), - StructField("b", IntegerType, nullable = false, new MetadataBuilder().putString("sourcecolumn", "src").build) - )) - - val result = getRenamesInSchema(schema) - val expected = Map( - "a.x" -> "a.src", - "a.y" -> "a.src", - "b" -> "src" - ) - - assert(result == expected) - } - - test("Testing getFirstArrayPath") { - assertResult("f.g")(getFirstArrayPath("f.g.h", schema)) - assertResult("f.g")(getFirstArrayPath("f.g", schema)) - assertResult("")(getFirstArrayPath("z.x.y", schema)) - assertResult("")(getFirstArrayPath("b.c.d.e", schema)) - } - - test("Testing getAllArrayPaths") { - assertResult(Seq("f.g"))(getAllArrayPaths(schema)) - assertResult(Seq())(getAllArrayPaths(schema("b").dataType.asInstanceOf[StructType])) - } - - test("Testing getAllArraysInPath") { - assertResult(Seq("b", "b.c.d"))(getAllArraysInPath("b.c.d.e", nestedSchema)) - } - - test("Testing getFieldNameOverriddenByMetadata") { - assertResult("a")(getFieldNameOverriddenByMetadata(structFieldNoMetadata)) - assertResult("a")(getFieldNameOverriddenByMetadata(structFieldWithMetadataNotSourceColumn)) - assertResult("override_a")(getFieldNameOverriddenByMetadata(structFieldWithMetadataSourceColumn)) - } - - test("Testing getFieldNullability") { - assert(!getFieldNullability("a", schema).get) - assert(getFieldNullability("b.d", schema).get) - assert(getFieldNullability("x.y.z", schema).isEmpty) - } - - test ("Test isCastAlwaysSucceeds()") { - assert(!isCastAlwaysSucceeds(StructType(Seq()), StringType)) - assert(!isCastAlwaysSucceeds(ArrayType(StringType), StringType)) - assert(!isCastAlwaysSucceeds(StringType, ByteType)) - assert(!isCastAlwaysSucceeds(StringType, ShortType)) - assert(!isCastAlwaysSucceeds(StringType, IntegerType)) - assert(!isCastAlwaysSucceeds(StringType, LongType)) - assert(!isCastAlwaysSucceeds(StringType, DecimalType(10,10))) - assert(!isCastAlwaysSucceeds(StringType, DateType)) - assert(!isCastAlwaysSucceeds(StringType, TimestampType)) - assert(!isCastAlwaysSucceeds(StructType(Seq()), StructType(Seq()))) - assert(!isCastAlwaysSucceeds(ArrayType(StringType), ArrayType(StringType))) - - assert(!isCastAlwaysSucceeds(ShortType, ByteType)) - assert(!isCastAlwaysSucceeds(IntegerType, ByteType)) - assert(!isCastAlwaysSucceeds(IntegerType, ShortType)) - assert(!isCastAlwaysSucceeds(LongType, ByteType)) - assert(!isCastAlwaysSucceeds(LongType, ShortType)) - assert(!isCastAlwaysSucceeds(LongType, IntegerType)) - - assert(isCastAlwaysSucceeds(StringType, StringType)) - assert(isCastAlwaysSucceeds(ByteType, StringType)) - assert(isCastAlwaysSucceeds(ShortType, StringType)) - assert(isCastAlwaysSucceeds(IntegerType, StringType)) - assert(isCastAlwaysSucceeds(LongType, StringType)) - assert(isCastAlwaysSucceeds(DecimalType(10,10), StringType)) - assert(isCastAlwaysSucceeds(DateType, StringType)) - assert(isCastAlwaysSucceeds(TimestampType, StringType)) - assert(isCastAlwaysSucceeds(StringType, StringType)) - - assert(isCastAlwaysSucceeds(ByteType, ByteType)) - assert(isCastAlwaysSucceeds(ByteType, ShortType)) - assert(isCastAlwaysSucceeds(ByteType, IntegerType)) - assert(isCastAlwaysSucceeds(ByteType, LongType)) - assert(isCastAlwaysSucceeds(ShortType, ShortType)) - assert(isCastAlwaysSucceeds(ShortType, IntegerType)) - assert(isCastAlwaysSucceeds(ShortType, LongType)) - assert(isCastAlwaysSucceeds(IntegerType, IntegerType)) - assert(isCastAlwaysSucceeds(IntegerType, LongType)) - assert(isCastAlwaysSucceeds(LongType, LongType)) - assert(isCastAlwaysSucceeds(DateType, TimestampType)) - } - - test("Test isCommonSubPath()") { - assert (isCommonSubPath()) - assert (isCommonSubPath("a")) - assert (isCommonSubPath("a.b.c.d.e.f", "a.b.c.d", "a.b.c", "a.b", "a")) - assert (!isCommonSubPath("a.b.c.d.e.f", "a.b.c.x", "a.b.c", "a.b", "a")) - } - - test("Test getDeepestCommonArrayPath() for a path without an array") { - val schema = StructType(Seq[StructField]( - StructField("a", - StructType(Seq[StructField]( - StructField("b", StringType)) - )))) - - assert (getDeepestCommonArrayPath(schema, Seq("a", "a.b")).isEmpty) - } - - test("Test getDeepestCommonArrayPath() for a path with a single array at top level") { - val schema = StructType(Seq[StructField]( - StructField("a", ArrayType(StructType(Seq[StructField]( - StructField("b", StringType))) - )))) - - val deepestPath = getDeepestCommonArrayPath(schema, Seq("a", "a.b")) - - assert (deepestPath.nonEmpty) - assert (deepestPath.get == "a") - } - - test("Test getDeepestCommonArrayPath() for a path with a single array at nested level") { - val schema = StructType(Seq[StructField]( - StructField("a", StructType(Seq[StructField]( - StructField("b", ArrayType(StringType)))) - ))) - - val deepestPath = getDeepestCommonArrayPath(schema, Seq("a", "a.b")) - - assert (deepestPath.nonEmpty) - assert (deepestPath.get == "a.b") - } - - test("Test getDeepestCommonArrayPath() for a path with several nested arrays of struct") { - val schema = StructType(Seq[StructField]( - StructField("a", ArrayType(StructType(Seq[StructField]( - StructField("b", StructType(Seq[StructField]( - StructField("c", ArrayType(StructType(Seq[StructField]( - StructField("d", StructType(Seq[StructField]( - StructField("e", StringType)) - ))) - )))) - ))) - ))))) - - val deepestPath = getDeepestCommonArrayPath(schema, Seq("a", "a.b", "a.b.c.d.e", "a.b.c.d")) - - assert (deepestPath.nonEmpty) - assert (deepestPath.get == "a.b.c") - } - - test("Test getDeepestArrayPath() for a path without an array") { - val schema = StructType(Seq[StructField]( - StructField("a", - StructType(Seq[StructField]( - StructField("b", StringType)) - )))) - - assert (getDeepestArrayPath(schema, "a.b").isEmpty) - } - - test("Test getDeepestArrayPath() for a path with a single array at top level") { - val schema = StructType(Seq[StructField]( - StructField("a", ArrayType(StructType(Seq[StructField]( - StructField("b", StringType))) - )))) - - val deepestPath = getDeepestArrayPath(schema, "a.b") - - assert (deepestPath.nonEmpty) - assert (deepestPath.get == "a") - } - - test("Test getDeepestArrayPath() for a path with a single array at nested level") { - val schema = StructType(Seq[StructField]( - StructField("a", StructType(Seq[StructField]( - StructField("b", ArrayType(StringType)))) - ))) - - val deepestPath = getDeepestArrayPath(schema, "a.b") - val deepestPath2 = getDeepestArrayPath(schema, "a") - - assert (deepestPath.nonEmpty) - assert (deepestPath.get == "a.b") - assert (deepestPath2.isEmpty) - } - - test("Test getDeepestArrayPath() for a path with several nested arrays of struct") { - val schema = StructType(Seq[StructField]( - StructField("a", ArrayType(StructType(Seq[StructField]( - StructField("b", StructType(Seq[StructField]( - StructField("c", ArrayType(StructType(Seq[StructField]( - StructField("d", StructType(Seq[StructField]( - StructField("e", StringType)) - ))) - )))) - ))) - ))))) - - val deepestPath = getDeepestArrayPath(schema, "a.b.c.d.e") - - assert (deepestPath.nonEmpty) - assert (deepestPath.get == "a.b.c") - } - - - test("Test getClosestUniqueName() is working properly") { - val schema = StructType(Seq[StructField]( - StructField("value", StringType))) - - // A column name that does not exist - val name1 = SchemaUtils.getClosestUniqueName("v", schema) - // A column that exists - val name2 = SchemaUtils.getClosestUniqueName("value", schema) - - assert(name1 == "v") - assert(name2 == "value_1") - } - - test("Test isOnlyField()") { - val schema = StructType(Seq[StructField]( - StructField("a", StringType), - StructField("b", StructType(Seq[StructField]( - StructField("e", StringType), - StructField("f", StringType) - ))), - StructField("c", StructType(Seq[StructField]( - StructField("d", StringType) - ))) - )) - - assert(!isOnlyField(schema, "a")) - assert(!isOnlyField(schema, "b.e")) - assert(!isOnlyField(schema, "b.f")) - assert(isOnlyField(schema, "c.d")) - } - - test("Test getStructField on array of arrays") { - assert(getField("a", arrayOfArraysSchema).contains(StructField("a",ArrayType(ArrayType(IntegerType)),nullable = false))) - assert(getField("b", arrayOfArraysSchema).contains(StructField("b",ArrayType(ArrayType(StructType(Seq(StructField("c",StringType,nullable = false))))), nullable = true))) - assert(getField("b.c", arrayOfArraysSchema).contains(StructField("c",StringType,nullable = false))) - assert(getField("b.d", arrayOfArraysSchema).isEmpty) - } - - test("Test fieldExists") { - assert(fieldExists("a", schema)) - assert(fieldExists("b", schema)) - assert(fieldExists("b.c", schema)) - assert(fieldExists("b.d", schema)) - assert(fieldExists("b.d.e", schema)) - assert(fieldExists("f", schema)) - assert(fieldExists("f.g", schema)) - assert(fieldExists("f.g.h", schema)) - assert(!fieldExists("z", schema)) - assert(!fieldExists("x.y.z", schema)) - assert(!fieldExists("f.g.h.a", schema)) - - assert(fieldExists("a", arrayOfArraysSchema)) - assert(fieldExists("b", arrayOfArraysSchema)) - assert(fieldExists("b.c", arrayOfArraysSchema)) - assert(!fieldExists("b.d", arrayOfArraysSchema)) - } - - test("unpath - empty string remains empty") { - val result = unpath("") - val expected = "" - assert(result == expected) - } - - test("unpath - underscores get doubled") { - val result = unpath("one_two__three") - val expected = "one__two____three" - assert(result == expected) - } - - test("unpath - dot notation conversion") { - val result = unpath("grand_parent.parent.first_child") - val expected = "grand__parent_parent_first__child" - assert(result == expected) - } - -} diff --git a/utils/src/test/scala/za/co/absa/enceladus/utils/schema/SparkUtilsSuite.scala b/utils/src/test/scala/za/co/absa/enceladus/utils/schema/SparkUtilsSuite.scala index a5b82b03c..781c499cb 100644 --- a/utils/src/test/scala/za/co/absa/enceladus/utils/schema/SparkUtilsSuite.scala +++ b/utils/src/test/scala/za/co/absa/enceladus/utils/schema/SparkUtilsSuite.scala @@ -19,11 +19,11 @@ import org.apache.spark.sql.DataFrame import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.{BooleanType, LongType, StructField, StructType} import org.scalatest.funsuite.AnyFunSuite -import za.co.absa.enceladus.utils.testUtils.SparkTestBase +import za.co.absa.spark.commons.test.SparkTestBase class SparkUtilsSuite extends AnyFunSuite with SparkTestBase { - import za.co.absa.enceladus.utils.implicits.DataFrameImplicits.DataFrameEnhancements + import za.co.absa.spark.commons.implicits.DataFrameImplicits.DataFrameEnhancements private def getDummyDataFrame: DataFrame = { import spark.implicits._ diff --git a/utils/src/test/scala/za/co/absa/enceladus/utils/transformations/ArrayTransformationsSuite.scala b/utils/src/test/scala/za/co/absa/enceladus/utils/transformations/ArrayTransformationsSuite.scala index 2384238da..0a4c2c192 100644 --- a/utils/src/test/scala/za/co/absa/enceladus/utils/transformations/ArrayTransformationsSuite.scala +++ b/utils/src/test/scala/za/co/absa/enceladus/utils/transformations/ArrayTransformationsSuite.scala @@ -16,9 +16,10 @@ package za.co.absa.enceladus.utils.transformations import org.scalatest.funsuite.AnyFunSuite -import za.co.absa.enceladus.utils.testUtils.SparkTestBase + import scala.util.Random import org.apache.spark.sql.functions._ +import za.co.absa.spark.commons.test.SparkTestBase case class InnerStruct(a: Int, b: String = null) case class OuterStruct(id: Int, vals: Seq[InnerStruct]) From 6d1b88f98c2a03407056b4c8032ae6274f5ef20d Mon Sep 17 00:00:00 2001 From: Adrian Olosutean Date: Thu, 17 Feb 2022 12:13:13 +0100 Subject: [PATCH 2/6] #2022 dependencies --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index b89a90a0b..c7e961d7e 100644 --- a/pom.xml +++ b/pom.xml @@ -141,7 +141,7 @@ 3.1.1 1.0.0 - 0.2.0-SNAPSHOT + 0.2.0 3.7.0 2.7.3 3.5.4 From 773846824ab41a5c448fa6fee2413837ce39123f Mon Sep 17 00:00:00 2001 From: Adrian Olosutean Date: Fri, 25 Feb 2022 17:13:04 +0200 Subject: [PATCH 3/6] Update spark-jobs/src/main/scala/za/co/absa/enceladus/standardization/interpreter/stages/TypeParser.scala Co-authored-by: Daniel K --- .../standardization/interpreter/stages/TypeParser.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/spark-jobs/src/main/scala/za/co/absa/enceladus/standardization/interpreter/stages/TypeParser.scala b/spark-jobs/src/main/scala/za/co/absa/enceladus/standardization/interpreter/stages/TypeParser.scala index d97fbf16a..bccaa68cb 100644 --- a/spark-jobs/src/main/scala/za/co/absa/enceladus/standardization/interpreter/stages/TypeParser.scala +++ b/spark-jobs/src/main/scala/za/co/absa/enceladus/standardization/interpreter/stages/TypeParser.scala @@ -197,7 +197,8 @@ object TypeParser { logger.info(s"Creating standardization plan for Array $inputFullPathName") val origArrayType = origType.asInstanceOf[ArrayType] // this should never throw an exception because of `checkSetupForFailure` val arrayField = StructField(fieldInputName, fieldType.elementType, fieldType.containsNull, field.structField.metadata) - val lambdaVariableName = s"${za.co.absa.enceladus.utils.schema.SchemaUtils.unpath(inputFullPathName)}_${Random.nextLong().abs}" + import za.co.absa.enceladus.utils.schema.{SchemaUtil => EnceladusSchemautils} // import rename + val lambdaVariableName = s"${EnceladusSchemautils.unpath(inputFullPathName)}_${Random.nextLong().abs}" val lambda = (forCol: Column) => TypeParser(arrayField, path, forCol, origArrayType.elementType, failOnInputNotPerSchema, isArrayElement = true) .standardize() From 85b452b24876c82e4d7c1f9074ae32d98817cc58 Mon Sep 17 00:00:00 2001 From: Adrian Olosutean Date: Sat, 26 Feb 2022 10:49:55 +0100 Subject: [PATCH 4/6] #2022 feedback --- .../za/co/absa/enceladus/common/CommonJobExecution.scala | 8 ++++---- .../standardization/interpreter/stages/TypeParser.scala | 2 +- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/spark-jobs/src/main/scala/za/co/absa/enceladus/common/CommonJobExecution.scala b/spark-jobs/src/main/scala/za/co/absa/enceladus/common/CommonJobExecution.scala index 61dc9ccd0..89f34c937 100644 --- a/spark-jobs/src/main/scala/za/co/absa/enceladus/common/CommonJobExecution.scala +++ b/spark-jobs/src/main/scala/za/co/absa/enceladus/common/CommonJobExecution.scala @@ -327,12 +327,12 @@ trait CommonJobExecution extends ProjectMetadata { protected def addInfoColumns(intoDf: DataFrame, reportDate: String, reportVersion: Int): DataFrame = { import za.co.absa.spark.commons.implicits.DataFrameImplicits.DataFrameEnhancements - val function: (DataFrame, String) => DataFrame = (df: DataFrame, _) => + val ifExistsFunc: (DataFrame, String) => DataFrame = (df: DataFrame, _) => df.withColumn("errCol", lit(Array.emptyIntArray)) intoDf - .withColumnIfDoesNotExist(function)(InfoDateColumn, to_date(lit(reportDate), ReportDateFormat)) - .withColumnIfDoesNotExist(function)(InfoDateColumnString, lit(reportDate)) - .withColumnIfDoesNotExist(function)(InfoVersionColumn, lit(reportVersion)) + .withColumnIfDoesNotExist(ifExistsFunc)(InfoDateColumn, to_date(lit(reportDate), ReportDateFormat)) + .withColumnIfDoesNotExist(ifExistsFunc)(InfoDateColumnString, lit(reportDate)) + .withColumnIfDoesNotExist(ifExistsFunc)(InfoVersionColumn, lit(reportVersion)) } private def getReportVersion[T](jobConfig: JobConfigParser[T], dataset: Dataset)(implicit hadoopConf: Configuration): Int = { diff --git a/spark-jobs/src/main/scala/za/co/absa/enceladus/standardization/interpreter/stages/TypeParser.scala b/spark-jobs/src/main/scala/za/co/absa/enceladus/standardization/interpreter/stages/TypeParser.scala index bccaa68cb..cfabcb863 100644 --- a/spark-jobs/src/main/scala/za/co/absa/enceladus/standardization/interpreter/stages/TypeParser.scala +++ b/spark-jobs/src/main/scala/za/co/absa/enceladus/standardization/interpreter/stages/TypeParser.scala @@ -38,6 +38,7 @@ import za.co.absa.spark.commons.implicits.ColumnImplicits.ColumnEnhancements import za.co.absa.spark.commons.implicits.StructTypeImplicits.StructTypeEnhancements import za.co.absa.spark.commons.utils.SchemaUtils import za.co.absa.spark.hofs.transform +import za.co.absa.enceladus.utils.schema.{SchemaUtils => EnceladusSchemautils} import scala.reflect.runtime.universe._ import scala.util.{Random, Try} @@ -197,7 +198,6 @@ object TypeParser { logger.info(s"Creating standardization plan for Array $inputFullPathName") val origArrayType = origType.asInstanceOf[ArrayType] // this should never throw an exception because of `checkSetupForFailure` val arrayField = StructField(fieldInputName, fieldType.elementType, fieldType.containsNull, field.structField.metadata) - import za.co.absa.enceladus.utils.schema.{SchemaUtil => EnceladusSchemautils} // import rename val lambdaVariableName = s"${EnceladusSchemautils.unpath(inputFullPathName)}_${Random.nextLong().abs}" val lambda = (forCol: Column) => TypeParser(arrayField, path, forCol, origArrayType.elementType, failOnInputNotPerSchema, isArrayElement = true) .standardize() From 3b34913109f301c9142f6933f80b13f233da8d68 Mon Sep 17 00:00:00 2001 From: Adrian Olosutean Date: Mon, 28 Feb 2022 12:17:10 +0100 Subject: [PATCH 5/6] #2022 withColumnIfDoesNotExist --- .../enceladus/common/CommonJobExecution.scala | 13 +++++------ .../conformance/HyperConformance.scala | 13 ++++++----- .../enceladus/utils/schema/SparkUtils.scala | 22 +++---------------- .../utils/schema/SparkUtilsSuite.scala | 10 +++++---- 4 files changed, 21 insertions(+), 37 deletions(-) diff --git a/spark-jobs/src/main/scala/za/co/absa/enceladus/common/CommonJobExecution.scala b/spark-jobs/src/main/scala/za/co/absa/enceladus/common/CommonJobExecution.scala index 89f34c937..1c5b1359d 100644 --- a/spark-jobs/src/main/scala/za/co/absa/enceladus/common/CommonJobExecution.scala +++ b/spark-jobs/src/main/scala/za/co/absa/enceladus/common/CommonJobExecution.scala @@ -20,14 +20,11 @@ import java.time.Instant import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.FileSystem import org.apache.spark.SPARK_VERSION -import org.apache.spark.sql.{DataFrame, SparkSession} import org.apache.spark.sql.functions.{lit, to_date} import org.apache.spark.sql.{DataFrame, SparkSession} import org.slf4j.{Logger, LoggerFactory} import za.co.absa.atum.AtumImplicits._ import za.co.absa.atum.core.{Atum, ControlType} -import za.co.absa.enceladus.common.Constants.{InfoDateColumn, InfoVersionColumn} -import za.co.absa.enceladus.common.config.{CommonConfConstants, JobConfigParser, PathConfig} import za.co.absa.enceladus.common.Constants.{InfoDateColumn, InfoDateColumnString, InfoVersionColumn, ReportDateFormat} import za.co.absa.enceladus.common.config.{CommonConfConstants, JobConfigParser, PathConfig} import za.co.absa.enceladus.common.plugin.PostProcessingService @@ -43,6 +40,7 @@ import za.co.absa.enceladus.utils.fs.{FileSystemUtils, HadoopFsUtils} import za.co.absa.enceladus.utils.modules.SourcePhase import za.co.absa.enceladus.utils.modules.SourcePhase.Standardization import za.co.absa.enceladus.common.performance.PerformanceMeasurer +import za.co.absa.enceladus.utils.schema.SparkUtils.ifExistsErrorFunction import za.co.absa.enceladus.utils.time.TimeZoneNormalizer import za.co.absa.enceladus.utils.validation.ValidationLevel import scala.util.control.NonFatal @@ -327,12 +325,11 @@ trait CommonJobExecution extends ProjectMetadata { protected def addInfoColumns(intoDf: DataFrame, reportDate: String, reportVersion: Int): DataFrame = { import za.co.absa.spark.commons.implicits.DataFrameImplicits.DataFrameEnhancements - val ifExistsFunc: (DataFrame, String) => DataFrame = (df: DataFrame, _) => - df.withColumn("errCol", lit(Array.emptyIntArray)) + val dateLitWithFormat = to_date(lit(reportDate), ReportDateFormat) intoDf - .withColumnIfDoesNotExist(ifExistsFunc)(InfoDateColumn, to_date(lit(reportDate), ReportDateFormat)) - .withColumnIfDoesNotExist(ifExistsFunc)(InfoDateColumnString, lit(reportDate)) - .withColumnIfDoesNotExist(ifExistsFunc)(InfoVersionColumn, lit(reportVersion)) + .withColumnIfDoesNotExist(ifExistsErrorFunction(dateLitWithFormat))(InfoDateColumn, dateLitWithFormat) + .withColumnIfDoesNotExist(ifExistsErrorFunction(lit(reportDate)))(InfoDateColumnString, lit(reportDate)) + .withColumnIfDoesNotExist(ifExistsErrorFunction(lit(reportVersion)))(InfoVersionColumn, lit(reportVersion)) } private def getReportVersion[T](jobConfig: JobConfigParser[T], dataset: Dataset)(implicit hadoopConf: Configuration): Int = { diff --git a/spark-jobs/src/main/scala/za/co/absa/enceladus/conformance/HyperConformance.scala b/spark-jobs/src/main/scala/za/co/absa/enceladus/conformance/HyperConformance.scala index 410155bb4..69cc63e02 100644 --- a/spark-jobs/src/main/scala/za/co/absa/enceladus/conformance/HyperConformance.scala +++ b/spark-jobs/src/main/scala/za/co/absa/enceladus/conformance/HyperConformance.scala @@ -17,6 +17,7 @@ package za.co.absa.enceladus.conformance import java.text.SimpleDateFormat import java.util.Date + import org.apache.commons.configuration2.Configuration import org.apache.hadoop.fs.FileSystem import org.apache.spark.SPARK_VERSION @@ -35,6 +36,7 @@ import za.co.absa.enceladus.dao.rest.RestDaoFactory.AvailabilitySetup import za.co.absa.enceladus.dao.rest.{MenasConnectionStringParser, RestDaoFactory} import za.co.absa.enceladus.model.{ConformedSchema, Dataset} import za.co.absa.enceladus.utils.fs.HadoopFsUtils +import za.co.absa.enceladus.utils.schema.SparkUtils.ifExistsErrorFunction import za.co.absa.enceladus.utils.validation.ValidationLevel import za.co.absa.hyperdrive.ingestor.api.transformer.{StreamTransformer, StreamTransformerFactory} @@ -78,13 +80,12 @@ class HyperConformance (menasBaseUrls: List[String], // using HDFS implementation until HyperConformance is S3-ready implicit val hdfs: FileSystem = FileSystem.get(sparkSession.sparkContext.hadoopConfiguration) implicit val hdfsUtils: HadoopFsUtils = HadoopFsUtils.getOrCreate(hdfs) - val function: (DataFrame, String) => DataFrame = (df: DataFrame, _) => - df.withColumn("errCol", lit(Array.emptyIntArray)) - + val dataFormat = coalesce(date_format(infoDateColumn, "yyyy-MM-dd"), lit("")) + val currentDateColumn = current_date() val conformedDf = DynamicInterpreter().interpret(conformance, rawDf) - .withColumnIfDoesNotExist(function)(InfoDateColumn, coalesce(infoDateColumn, current_date())) - .withColumnIfDoesNotExist(function)(InfoDateColumnString, coalesce(date_format(infoDateColumn,"yyyy-MM-dd"), lit(""))) - .withColumnIfDoesNotExist(function)(InfoVersionColumn, infoVersionColumn) + .withColumnIfDoesNotExist(ifExistsErrorFunction(currentDateColumn))(InfoDateColumn, coalesce(infoDateColumn, currentDateColumn)) + .withColumnIfDoesNotExist(ifExistsErrorFunction(dataFormat))(InfoDateColumnString, dataFormat) + .withColumnIfDoesNotExist(ifExistsErrorFunction(infoVersionColumn))(InfoVersionColumn, infoVersionColumn) conformedDf } diff --git a/utils/src/main/scala/za/co/absa/enceladus/utils/schema/SparkUtils.scala b/utils/src/main/scala/za/co/absa/enceladus/utils/schema/SparkUtils.scala index 4e8258457..5f70bc42c 100644 --- a/utils/src/main/scala/za/co/absa/enceladus/utils/schema/SparkUtils.scala +++ b/utils/src/main/scala/za/co/absa/enceladus/utils/schema/SparkUtils.scala @@ -15,7 +15,6 @@ package za.co.absa.enceladus.utils.schema -import org.apache.log4j.{LogManager, Logger} import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.StructType import org.apache.spark.sql.{Column, DataFrame, SparkSession} @@ -29,7 +28,6 @@ import za.co.absa.spark.hats.transformations.NestedArrayTransformations * General Spark utils */ object SparkUtils { - private val log: Logger = LogManager.getLogger(this.getClass) private final val DefaultColumnNameOfCorruptRecord = "_corrupt_record" final val ColumnNameOfCorruptRecordConf = "spark.sql.columnNameOfCorruptRecord" @@ -51,23 +49,6 @@ object SparkUtils { result } - /** - * Adds a column to a dataframe if it does not exist - * - * @param df A dataframe - * @param colName A column to add if it does not exist already - * @param colExpr An expression for the column to add - * @return a new dataframe with the new column - */ - def withColumnIfDoesNotExist(df: DataFrame, colName: String, colExpr: Column): DataFrame = { - if (df.schema.exists(field => field.name.equalsIgnoreCase(colName))) { - log.warn(s"Column '$colName' already exists. The content of the column will be overwritten.") - overwriteWithErrorColumn(df, colName, colExpr) - } else { - df.withColumn(colName, colExpr) - } - } - /** * Overwrites a column with a value provided by an expression. * If the value in the column does not match the one provided by the expression, an error will be @@ -105,4 +86,7 @@ object SparkUtils { dfWithAggregatedErrColumn.drop(tmpColumn) } + def ifExistsErrorFunction(colExpr: Column): (DataFrame, String) => DataFrame = + (df: DataFrame, colName: String) => overwriteWithErrorColumn(df, colName, colExpr) + } diff --git a/utils/src/test/scala/za/co/absa/enceladus/utils/schema/SparkUtilsSuite.scala b/utils/src/test/scala/za/co/absa/enceladus/utils/schema/SparkUtilsSuite.scala index 781c499cb..101d8ce72 100644 --- a/utils/src/test/scala/za/co/absa/enceladus/utils/schema/SparkUtilsSuite.scala +++ b/utils/src/test/scala/za/co/absa/enceladus/utils/schema/SparkUtilsSuite.scala @@ -15,7 +15,7 @@ package za.co.absa.enceladus.utils.schema -import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.{Column, DataFrame} import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.{BooleanType, LongType, StructField, StructType} import org.scalatest.funsuite.AnyFunSuite @@ -45,6 +45,7 @@ class SparkUtilsSuite extends AnyFunSuite with SparkTestBase { assert(spark.conf.get(SparkUtils.ColumnNameOfCorruptRecordConf) == expected2) } + private val colExpression: Column = lit(1) test("Test withColumnIfNotExist() when the column does not exist") { val expectedOutput = """+-----+---+ @@ -60,7 +61,8 @@ class SparkUtilsSuite extends AnyFunSuite with SparkTestBase { |""".stripMargin.replace("\r\n", "\n") val dfIn = getDummyDataFrame - val dfOut = SparkUtils.withColumnIfDoesNotExist(dfIn, "foo", lit(1)) + + val dfOut = dfIn.withColumnIfDoesNotExist(SparkUtils.ifExistsErrorFunction(colExpression))("foo", colExpression) val actualOutput = dfOut.dataAsString(truncate = false) assert(dfOut.schema.length == 2) @@ -84,7 +86,7 @@ class SparkUtilsSuite extends AnyFunSuite with SparkTestBase { |""".stripMargin.replace("\r\n", "\n") val dfIn = getDummyDataFrame - val dfOut = SparkUtils.withColumnIfDoesNotExist(dfIn, "value", lit(1)) + val dfOut = dfIn.withColumnIfDoesNotExist(SparkUtils.ifExistsErrorFunction(colExpression))("value", colExpression) val actualOutput = dfOut.dataAsString(truncate = false) assert(dfIn.schema.length == 1) @@ -107,7 +109,7 @@ class SparkUtilsSuite extends AnyFunSuite with SparkTestBase { |""".stripMargin.replace("\r\n", "\n") val dfIn = getDummyDataFrame - val dfOut = SparkUtils.withColumnIfDoesNotExist(dfIn, "vAlUe", lit(1)) + val dfOut = dfIn.withColumnIfDoesNotExist(SparkUtils.ifExistsErrorFunction(colExpression))("vAlUe", colExpression) val actualOutput = dfOut.dataAsString(truncate = false) assert(dfIn.schema.length == 1) From ca144ec3fdfb7853ec644e4b6d7d8215a286325e Mon Sep 17 00:00:00 2001 From: Adrian Olosutean Date: Tue, 8 Mar 2022 13:32:21 +0100 Subject: [PATCH 6/6] #2022 feedback --- .../absa/enceladus/common/CommonJobExecution.scala | 10 ++++------ .../enceladus/conformance/HyperConformance.scala | 8 ++++---- .../co/absa/enceladus/utils/schema/SparkUtils.scala | 9 +++++++-- .../enceladus/utils/schema/SparkUtilsSuite.scala | 13 +++++++------ 4 files changed, 22 insertions(+), 18 deletions(-) diff --git a/spark-jobs/src/main/scala/za/co/absa/enceladus/common/CommonJobExecution.scala b/spark-jobs/src/main/scala/za/co/absa/enceladus/common/CommonJobExecution.scala index 1c5b1359d..67943e38a 100644 --- a/spark-jobs/src/main/scala/za/co/absa/enceladus/common/CommonJobExecution.scala +++ b/spark-jobs/src/main/scala/za/co/absa/enceladus/common/CommonJobExecution.scala @@ -40,7 +40,6 @@ import za.co.absa.enceladus.utils.fs.{FileSystemUtils, HadoopFsUtils} import za.co.absa.enceladus.utils.modules.SourcePhase import za.co.absa.enceladus.utils.modules.SourcePhase.Standardization import za.co.absa.enceladus.common.performance.PerformanceMeasurer -import za.co.absa.enceladus.utils.schema.SparkUtils.ifExistsErrorFunction import za.co.absa.enceladus.utils.time.TimeZoneNormalizer import za.co.absa.enceladus.utils.validation.ValidationLevel import scala.util.control.NonFatal @@ -324,12 +323,11 @@ trait CommonJobExecution extends ProjectMetadata { } protected def addInfoColumns(intoDf: DataFrame, reportDate: String, reportVersion: Int): DataFrame = { - import za.co.absa.spark.commons.implicits.DataFrameImplicits.DataFrameEnhancements - val dateLitWithFormat = to_date(lit(reportDate), ReportDateFormat) + import za.co.absa.enceladus.utils.schema.SparkUtils.DataFrameWithEnhancements intoDf - .withColumnIfDoesNotExist(ifExistsErrorFunction(dateLitWithFormat))(InfoDateColumn, dateLitWithFormat) - .withColumnIfDoesNotExist(ifExistsErrorFunction(lit(reportDate)))(InfoDateColumnString, lit(reportDate)) - .withColumnIfDoesNotExist(ifExistsErrorFunction(lit(reportVersion)))(InfoVersionColumn, lit(reportVersion)) + .withColumnOverwriteIfExists(InfoDateColumn, to_date(lit(reportDate), ReportDateFormat)) + .withColumnOverwriteIfExists(InfoDateColumnString, lit(reportDate)) + .withColumnOverwriteIfExists(InfoVersionColumn, lit(reportVersion)) } private def getReportVersion[T](jobConfig: JobConfigParser[T], dataset: Dataset)(implicit hadoopConf: Configuration): Int = { diff --git a/spark-jobs/src/main/scala/za/co/absa/enceladus/conformance/HyperConformance.scala b/spark-jobs/src/main/scala/za/co/absa/enceladus/conformance/HyperConformance.scala index 69cc63e02..5ea2591c6 100644 --- a/spark-jobs/src/main/scala/za/co/absa/enceladus/conformance/HyperConformance.scala +++ b/spark-jobs/src/main/scala/za/co/absa/enceladus/conformance/HyperConformance.scala @@ -36,7 +36,6 @@ import za.co.absa.enceladus.dao.rest.RestDaoFactory.AvailabilitySetup import za.co.absa.enceladus.dao.rest.{MenasConnectionStringParser, RestDaoFactory} import za.co.absa.enceladus.model.{ConformedSchema, Dataset} import za.co.absa.enceladus.utils.fs.HadoopFsUtils -import za.co.absa.enceladus.utils.schema.SparkUtils.ifExistsErrorFunction import za.co.absa.enceladus.utils.validation.ValidationLevel import za.co.absa.hyperdrive.ingestor.api.transformer.{StreamTransformer, StreamTransformerFactory} @@ -82,10 +81,11 @@ class HyperConformance (menasBaseUrls: List[String], implicit val hdfsUtils: HadoopFsUtils = HadoopFsUtils.getOrCreate(hdfs) val dataFormat = coalesce(date_format(infoDateColumn, "yyyy-MM-dd"), lit("")) val currentDateColumn = current_date() + import za.co.absa.enceladus.utils.schema.SparkUtils.DataFrameWithEnhancements val conformedDf = DynamicInterpreter().interpret(conformance, rawDf) - .withColumnIfDoesNotExist(ifExistsErrorFunction(currentDateColumn))(InfoDateColumn, coalesce(infoDateColumn, currentDateColumn)) - .withColumnIfDoesNotExist(ifExistsErrorFunction(dataFormat))(InfoDateColumnString, dataFormat) - .withColumnIfDoesNotExist(ifExistsErrorFunction(infoVersionColumn))(InfoVersionColumn, infoVersionColumn) + .withColumnOverwriteIfExists(InfoDateColumn, coalesce(infoDateColumn, currentDateColumn)) + .withColumnOverwriteIfExists(InfoDateColumnString, dataFormat) + .withColumnOverwriteIfExists(InfoVersionColumn, infoVersionColumn) conformedDf } diff --git a/utils/src/main/scala/za/co/absa/enceladus/utils/schema/SparkUtils.scala b/utils/src/main/scala/za/co/absa/enceladus/utils/schema/SparkUtils.scala index 5f70bc42c..224c367af 100644 --- a/utils/src/main/scala/za/co/absa/enceladus/utils/schema/SparkUtils.scala +++ b/utils/src/main/scala/za/co/absa/enceladus/utils/schema/SparkUtils.scala @@ -21,6 +21,7 @@ import org.apache.spark.sql.{Column, DataFrame, SparkSession} import za.co.absa.enceladus.utils.error.ErrorMessage import za.co.absa.enceladus.utils.udf.UDFLibrary import za.co.absa.spark.commons.implicits.StructTypeImplicits.StructTypeEnhancements +import za.co.absa.spark.commons.implicits.DataFrameImplicits.DataFrameEnhancements import za.co.absa.spark.hats.transformations.NestedArrayTransformations @@ -86,7 +87,11 @@ object SparkUtils { dfWithAggregatedErrColumn.drop(tmpColumn) } - def ifExistsErrorFunction(colExpr: Column): (DataFrame, String) => DataFrame = - (df: DataFrame, colName: String) => overwriteWithErrorColumn(df, colName, colExpr) + implicit class DataFrameWithEnhancements(val df: DataFrame) { + def withColumnOverwriteIfExists(colName: String, colExpr: Column): DataFrame = { + val overwrite: (DataFrame, String) => DataFrame = overwriteWithErrorColumn(_, _, colExpr) + df.withColumnIfDoesNotExist(overwrite)(colName, colExpr) + } + } } diff --git a/utils/src/test/scala/za/co/absa/enceladus/utils/schema/SparkUtilsSuite.scala b/utils/src/test/scala/za/co/absa/enceladus/utils/schema/SparkUtilsSuite.scala index 101d8ce72..dfe3ef20f 100644 --- a/utils/src/test/scala/za/co/absa/enceladus/utils/schema/SparkUtilsSuite.scala +++ b/utils/src/test/scala/za/co/absa/enceladus/utils/schema/SparkUtilsSuite.scala @@ -20,6 +20,7 @@ import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.{BooleanType, LongType, StructField, StructType} import org.scalatest.funsuite.AnyFunSuite import za.co.absa.spark.commons.test.SparkTestBase +import za.co.absa.enceladus.utils.schema.SparkUtils.DataFrameWithEnhancements class SparkUtilsSuite extends AnyFunSuite with SparkTestBase { @@ -46,7 +47,7 @@ class SparkUtilsSuite extends AnyFunSuite with SparkTestBase { } private val colExpression: Column = lit(1) - test("Test withColumnIfNotExist() when the column does not exist") { + test("Test withColumnOverwriteIfExists() when the column does not exist") { val expectedOutput = """+-----+---+ ||value|foo| @@ -62,7 +63,7 @@ class SparkUtilsSuite extends AnyFunSuite with SparkTestBase { val dfIn = getDummyDataFrame - val dfOut = dfIn.withColumnIfDoesNotExist(SparkUtils.ifExistsErrorFunction(colExpression))("foo", colExpression) + val dfOut = dfIn.withColumnOverwriteIfExists("foo", colExpression) val actualOutput = dfOut.dataAsString(truncate = false) assert(dfOut.schema.length == 2) @@ -71,7 +72,7 @@ class SparkUtilsSuite extends AnyFunSuite with SparkTestBase { assert(actualOutput == expectedOutput) } - test("Test withColumnIfNotExist() when the column exists") { + test("Test withColumnOverwriteIfExists() when the column exists") { val expectedOutput = """+-----+----------------------------------------------------------------------------------------------+ ||value|errCol | @@ -86,7 +87,7 @@ class SparkUtilsSuite extends AnyFunSuite with SparkTestBase { |""".stripMargin.replace("\r\n", "\n") val dfIn = getDummyDataFrame - val dfOut = dfIn.withColumnIfDoesNotExist(SparkUtils.ifExistsErrorFunction(colExpression))("value", colExpression) + val dfOut = dfIn.withColumnOverwriteIfExists("value", colExpression) val actualOutput = dfOut.dataAsString(truncate = false) assert(dfIn.schema.length == 1) @@ -94,7 +95,7 @@ class SparkUtilsSuite extends AnyFunSuite with SparkTestBase { assert(actualOutput == expectedOutput) } - test("Test withColumnIfNotExist() when the column exists, but has a different case") { + test("Test withColumnOverwriteIfExists() when the column exists, but has a different case") { val expectedOutput = """+-----+----------------------------------------------------------------------------------------------+ ||vAlUe|errCol | @@ -109,7 +110,7 @@ class SparkUtilsSuite extends AnyFunSuite with SparkTestBase { |""".stripMargin.replace("\r\n", "\n") val dfIn = getDummyDataFrame - val dfOut = dfIn.withColumnIfDoesNotExist(SparkUtils.ifExistsErrorFunction(colExpression))("vAlUe", colExpression) + val dfOut = dfIn.withColumnOverwriteIfExists("vAlUe", colExpression) val actualOutput = dfOut.dataAsString(truncate = false) assert(dfIn.schema.length == 1)