From a26b4c97bc8aa21adb7dfaf5d0a329d36870d748 Mon Sep 17 00:00:00 2001 From: Daniel Kavan Date: Mon, 26 Oct 2020 14:08:17 +0100 Subject: [PATCH 01/12] s3 using hadoop fs api - first attempt to run --- .../enceladus/dao/auth/MenasCredentials.scala | 11 +-- .../examples/CustomRuleSample1.scala | 7 +- .../examples/CustomRuleSample2.scala | 6 +- .../examples/CustomRuleSample3.scala | 5 +- .../examples/CustomRuleSample4.scala | 8 +-- .../UppercaseCustomConformanceRuleSuite.scala | 8 +-- .../XPadCustomConformanceRuleSuite.scala | 10 ++- pom.xml | 2 +- spark-jobs/src/main/resources/reference.conf | 6 -- .../enceladus/common/CommonJobExecution.scala | 48 ++++++------- .../enceladus/common/config/FileSystems.scala | 57 +++++++++++++++ .../conformance/ConformanceExecution.scala | 56 +++++++-------- .../conformance/DynamicConformanceJob.scala | 1 - .../conformance/HyperConformance.scala | 7 +- .../interpreter/DynamicInterpreter.scala | 13 ++-- .../StandardizationExecution.scala | 62 ++++++++--------- .../standardization/StandardizationJob.scala | 3 +- ...andardizationAndConformanceExecution.scala | 13 ++-- .../StandardizationAndConformanceJob.scala | 3 +- .../interpreter/ArrayConformanceSuite.scala | 10 ++- .../interpreter/ChorusMockSuite.scala | 7 +- .../interpreter/InterpreterSuite.scala | 18 +++-- .../LiteralJoinMappingRuleTest.scala | 7 +- .../interpreter/NestedStructSuite.scala | 7 +- .../interpreter/rules/CastingRuleSuite.scala | 7 +- .../rules/MappingRuleBroadcastSuite.scala | 7 +- .../interpreter/rules/MappingRuleSuite.scala | 7 +- .../interpreter/rules/NegationRuleSuite.scala | 7 +- .../rules/RuleOptimizationSuite.scala | 8 +-- .../interpreter/rules/TestRuleBehaviors.scala | 8 +-- .../rules/custom/CustomRuleSuite.scala | 7 +- .../NestedTestCaseFactory.scala | 7 +- .../SimpleTestCaseFactory.scala | 7 +- .../utils/fs/DistributedFsUtils.scala | 4 -- .../enceladus/utils/fs/FileSystemUtils.scala | 69 +++++++++++++++++++ .../{HdfsUtils.scala => HadoopFsUtils.scala} | 9 +-- .../{S3FsUtils.scala => S3SdkFsUtils.scala} | 14 ++-- .../utils/fs/example/S3FsUtilsTestJob.scala | 4 +- .../performance/PerformanceMetricTools.scala | 16 ++--- .../utils/testUtils/HadoopFsTestBase.scala | 20 +++++- .../utils/testUtils/SparkTestBase.scala | 4 +- .../enceladus/utils/fs/HdfsUtilsSpec.scala | 6 +- ...ilsSuite.scala => S3SdkFsUtilsSuite.scala} | 4 +- 43 files changed, 333 insertions(+), 257 deletions(-) create mode 100644 spark-jobs/src/main/scala/za/co/absa/enceladus/common/config/FileSystems.scala create mode 100644 utils/src/main/scala/za/co/absa/enceladus/utils/fs/FileSystemUtils.scala rename utils/src/main/scala/za/co/absa/enceladus/utils/fs/{HdfsUtils.scala => HadoopFsUtils.scala} (96%) rename utils/src/main/scala/za/co/absa/enceladus/utils/fs/{S3FsUtils.scala => S3SdkFsUtils.scala} (95%) rename spark-jobs/src/main/scala/za/co/absa/enceladus/common/config/S3Config.scala => utils/src/main/scala/za/co/absa/enceladus/utils/testUtils/HadoopFsTestBase.scala (59%) rename utils/src/test/scala/za/co/absa/enceladus/utils/fs/{S3FsUtilsSuite.scala => S3SdkFsUtilsSuite.scala} (99%) diff --git a/dao/src/main/scala/za/co/absa/enceladus/dao/auth/MenasCredentials.scala b/dao/src/main/scala/za/co/absa/enceladus/dao/auth/MenasCredentials.scala index beeb69089..63a430899 100644 --- a/dao/src/main/scala/za/co/absa/enceladus/dao/auth/MenasCredentials.scala +++ b/dao/src/main/scala/za/co/absa/enceladus/dao/auth/MenasCredentials.scala @@ -18,7 +18,8 @@ package za.co.absa.enceladus.dao.auth import com.typesafe.config.ConfigFactory import org.apache.spark.sql.SparkSession import sun.security.krb5.internal.ktab.KeyTab -import za.co.absa.enceladus.utils.fs.HdfsUtils +import za.co.absa.enceladus.utils.fs.{DistributedFsUtils, FileSystemUtils} +import FileSystemUtils.FileSystemExt sealed abstract class MenasCredentials { val username: String @@ -40,9 +41,9 @@ object MenasPlainCredentials { * @return An instance of Menas Credentials. */ def fromFile(path: String)(implicit spark: SparkSession): MenasPlainCredentials = { - val fsUtils = new HdfsUtils(spark.sparkContext.hadoopConfiguration) + val fs = FileSystemUtils.getFileSystemFromPath(path)(spark.sparkContext.hadoopConfiguration) - val conf = ConfigFactory.parseString(fsUtils.getLocalOrDistributedFileContent(path)) + val conf = ConfigFactory.parseString(fs.toFsUtils.getLocalOrDistributedFileContent(path)) MenasPlainCredentials(conf.getString("username"), conf.getString("password")) } } @@ -55,9 +56,9 @@ object MenasKerberosCredentials { * @return An instance of Menas Credentials. */ def fromFile(path: String)(implicit spark: SparkSession): MenasKerberosCredentials = { - val fsUtils = new HdfsUtils(spark.sparkContext.hadoopConfiguration) + val fs = FileSystemUtils.getFileSystemFromPath(path)(spark.sparkContext.hadoopConfiguration) - val localKeyTabPath = fsUtils.getLocalPathToFileOrCopyToLocal(path) + val localKeyTabPath = fs.toFsUtils.getLocalPathToFileOrCopyToLocal(path) val keytab = KeyTab.getInstance(localKeyTabPath) val username = keytab.getOneName.getName diff --git a/examples/src/main/scala/za/co/absa/enceladus/examples/CustomRuleSample1.scala b/examples/src/main/scala/za/co/absa/enceladus/examples/CustomRuleSample1.scala index eb46b94a4..df13f7625 100644 --- a/examples/src/main/scala/za/co/absa/enceladus/examples/CustomRuleSample1.scala +++ b/examples/src/main/scala/za/co/absa/enceladus/examples/CustomRuleSample1.scala @@ -23,10 +23,11 @@ import za.co.absa.enceladus.dao.auth.MenasKerberosCredentials import za.co.absa.enceladus.dao.rest.RestDaoFactory import za.co.absa.enceladus.examples.interpreter.rules.custom.UppercaseCustomConformanceRule import za.co.absa.enceladus.model.Dataset -import za.co.absa.enceladus.utils.fs.HdfsUtils +import za.co.absa.enceladus.utils.fs.HadoopFsUtils +import za.co.absa.enceladus.utils.testUtils.HadoopFsTestBase import za.co.absa.enceladus.utils.time.TimeZoneNormalizer -object CustomRuleSample1 { +object CustomRuleSample1 extends HadoopFsTestBase { case class ExampleRow(id: Int, makeUpper: String, leave: String) case class OutputRow(id: Int, makeUpper: String, leave: String, doneUpper: String) @@ -38,8 +39,6 @@ object CustomRuleSample1 { .getOrCreate() TimeZoneNormalizer.normalizeAll(spark) //normalize the timezone of JVM and the spark session - implicit val fsUtils: HdfsUtils = new HdfsUtils(spark.sparkContext.hadoopConfiguration) - def main(args: Array[String]) { // scalastyle:off magic.number val menasBaseUrls = List("http://localhost:8080/menas") diff --git a/examples/src/main/scala/za/co/absa/enceladus/examples/CustomRuleSample2.scala b/examples/src/main/scala/za/co/absa/enceladus/examples/CustomRuleSample2.scala index 5ce905cde..ff1460a0f 100644 --- a/examples/src/main/scala/za/co/absa/enceladus/examples/CustomRuleSample2.scala +++ b/examples/src/main/scala/za/co/absa/enceladus/examples/CustomRuleSample2.scala @@ -24,10 +24,10 @@ import za.co.absa.enceladus.dao.auth.MenasKerberosCredentials import za.co.absa.enceladus.dao.rest.{MenasConnectionStringParser, RestDaoFactory} import za.co.absa.enceladus.examples.interpreter.rules.custom.LPadCustomConformanceRule import za.co.absa.enceladus.model.Dataset -import za.co.absa.enceladus.utils.fs.HdfsUtils +import za.co.absa.enceladus.utils.testUtils.HadoopFsTestBase import za.co.absa.enceladus.utils.time.TimeZoneNormalizer -object CustomRuleSample2 { +object CustomRuleSample2 extends HadoopFsTestBase { case class ExampleRow(id: Int, addPad: String, leave: String) case class OutputRow(id: Int, addPad: String, leave: String, donePad: String) @@ -39,8 +39,6 @@ object CustomRuleSample2 { .getOrCreate() TimeZoneNormalizer.normalizeAll(spark) //normalize the timezone of JVM and the spark session - implicit val fsUtils: HdfsUtils = new HdfsUtils(spark.sparkContext.hadoopConfiguration) - def main(args: Array[String]) { // scalastyle:off magic.number val conf = ConfigFactory.load() diff --git a/examples/src/main/scala/za/co/absa/enceladus/examples/CustomRuleSample3.scala b/examples/src/main/scala/za/co/absa/enceladus/examples/CustomRuleSample3.scala index 59fe41b7a..c22d5d433 100644 --- a/examples/src/main/scala/za/co/absa/enceladus/examples/CustomRuleSample3.scala +++ b/examples/src/main/scala/za/co/absa/enceladus/examples/CustomRuleSample3.scala @@ -24,17 +24,16 @@ import za.co.absa.enceladus.dao.auth.MenasKerberosCredentials import za.co.absa.enceladus.dao.rest.{MenasConnectionStringParser, RestDaoFactory} import za.co.absa.enceladus.examples.interpreter.rules.custom.{LPadCustomConformanceRule, UppercaseCustomConformanceRule} import za.co.absa.enceladus.model.Dataset -import za.co.absa.enceladus.utils.fs.HdfsUtils +import za.co.absa.enceladus.utils.testUtils.HadoopFsTestBase import za.co.absa.enceladus.utils.time.TimeZoneNormalizer -object CustomRuleSample3 { +object CustomRuleSample3 extends HadoopFsTestBase { implicit val spark: SparkSession = SparkSession.builder .master("local[*]") .appName("CustomRuleSample3") .config("spark.sql.codegen.wholeStage", value = false) .getOrCreate() TimeZoneNormalizer.normalizeAll(spark) //normalize the timezone of JVM and the spark session - implicit val fsUtils: HdfsUtils = new HdfsUtils(spark.sparkContext.hadoopConfiguration) def main(args: Array[String]): Unit = { val conf = ConfigFactory.load() diff --git a/examples/src/main/scala/za/co/absa/enceladus/examples/CustomRuleSample4.scala b/examples/src/main/scala/za/co/absa/enceladus/examples/CustomRuleSample4.scala index 47e8dd649..3a23dcdbf 100644 --- a/examples/src/main/scala/za/co/absa/enceladus/examples/CustomRuleSample4.scala +++ b/examples/src/main/scala/za/co/absa/enceladus/examples/CustomRuleSample4.scala @@ -26,10 +26,10 @@ import za.co.absa.enceladus.dao.auth.MenasKerberosCredentials import za.co.absa.enceladus.dao.rest.{MenasConnectionStringParser, RestDaoFactory} import za.co.absa.enceladus.examples.interpreter.rules.custom.{LPadCustomConformanceRule, UppercaseCustomConformanceRule} import za.co.absa.enceladus.model.Dataset -import za.co.absa.enceladus.utils.fs.HdfsUtils +import za.co.absa.enceladus.utils.testUtils.HadoopFsTestBase import za.co.absa.enceladus.utils.time.TimeZoneNormalizer -object CustomRuleSample4 { +object CustomRuleSample4 extends HadoopFsTestBase { TimeZoneNormalizer.normalizeJVMTimeZone() //normalize JVM time zone as soon as possible /** @@ -136,10 +136,10 @@ object CustomRuleSample4 { result } + implicit val spark: SparkSession = buildSparkSession() + def main(args: Array[String]): Unit = { val cmd: CmdConfigLocal = getCmdLineArguments(args) - implicit val spark: SparkSession = buildSparkSession() - implicit val fsUtils: HdfsUtils = new HdfsUtils(spark.sparkContext.hadoopConfiguration) val conf = ConfigFactory.load() val menasBaseUrls = MenasConnectionStringParser.parse(conf.getString("menas.rest.uri")) 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 5009aa7c8..5c0ba1fbd 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 @@ -23,8 +23,8 @@ 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.fs.HdfsUtils -import za.co.absa.enceladus.utils.testUtils.SparkTestBase +import za.co.absa.enceladus.utils.fs.HadoopFsUtils +import za.co.absa.enceladus.utils.testUtils.{HadoopFsTestBase, SparkTestBase} case class TestInputRow(id: Int, mandatoryString: String, nullableString: Option[String]) @@ -33,12 +33,12 @@ object TestOutputRow { def apply(input: TestInputRow, doneUpper: String): TestOutputRow = TestOutputRow(input.id, input.mandatoryString, input.nullableString, doneUpper) } -class UppercaseCustomConformanceRuleSuite extends AnyFunSuite with SparkTestBase with MockitoSugar { +class UppercaseCustomConformanceRuleSuite extends AnyFunSuite with SparkTestBase with MockitoSugar with HadoopFsTestBase { import spark.implicits._ implicit val progArgs: ConformanceConfig = ConformanceConfig() // here we may need to specify some parameters (for certain rules) implicit val dao: MenasDAO = mock[MenasDAO] // you may have to hard-code your own implementation here (if not working with menas) - implicit val fsUtils: HdfsUtils = new HdfsUtils(spark.sparkContext.hadoopConfiguration) + val experimentalMR = true val isCatalystWorkaroundEnabled = true 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 c3f656b9b..78daedc01 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.HdfsUtils -import za.co.absa.enceladus.utils.testUtils.SparkTestBase +import za.co.absa.enceladus.utils.fs.HadoopFsUtils +import za.co.absa.enceladus.utils.testUtils.{HadoopFsTestBase, SparkTestBase} case class XPadTestInputRow(intField: Int, stringField: Option[String]) case class XPadTestOutputRow(intField: Int, stringField: Option[String], targetField: String) @@ -35,12 +35,11 @@ object XPadTestOutputRow { def apply(input: XPadTestInputRow, targetField: String): XPadTestOutputRow = XPadTestOutputRow(input.intField, input.stringField, targetField) } -class LpadCustomConformanceRuleSuite extends AnyFunSuite with SparkTestBase with MockitoSugar { +class LpadCustomConformanceRuleSuite extends AnyFunSuite with SparkTestBase with MockitoSugar with HadoopFsTestBase { import spark.implicits._ implicit val progArgs: ConformanceConfig = ConformanceConfig() // here we may need to specify some parameters (for certain rules) implicit val dao: MenasDAO = mock[MenasDAO] // you may have to hard-code your own implementation here (if not working with menas) - implicit val fsUtils: HdfsUtils = new HdfsUtils(spark.sparkContext.hadoopConfiguration) val experimentalMR = true val isCatalystWorkaroundEnabled = true @@ -180,7 +179,7 @@ class LpadCustomConformanceRuleSuite extends AnyFunSuite with SparkTestBase with } -class RpadCustomConformanceRuleSuite extends AnyFunSuite with SparkTestBase { +class RpadCustomConformanceRuleSuite extends AnyFunSuite with SparkTestBase with HadoopFsTestBase { import spark.implicits._ @@ -189,7 +188,6 @@ class RpadCustomConformanceRuleSuite extends AnyFunSuite with SparkTestBase { private val meansCredentials = MenasKerberosCredentials("user@EXAMPLE.COM", "src/test/resources/user.keytab.example") implicit val progArgs: ConformanceConfig = ConformanceConfig() // here we may need to specify some parameters (for certain rules) implicit val dao: MenasDAO = RestDaoFactory.getInstance(meansCredentials, menasBaseUrls) // you may have to hard-code your own implementation here (if not working with menas) - implicit val fsUtils: HdfsUtils = new HdfsUtils(spark.sparkContext.hadoopConfiguration) val experimentalMR = true val isCatalystWorkaroundEnabled = true diff --git a/pom.xml b/pom.xml index 0b9cb1333..3d1788a9a 100644 --- a/pom.xml +++ b/pom.xml @@ -144,7 +144,7 @@ 1.6 3.1.1 - 3.0.0 + 3.0.1-SNAPSHOT 2.13.65 2.7.3 3.5.4 diff --git a/spark-jobs/src/main/resources/reference.conf b/spark-jobs/src/main/resources/reference.conf index 48efb6d48..2591e9c77 100644 --- a/spark-jobs/src/main/resources/reference.conf +++ b/spark-jobs/src/main/resources/reference.conf @@ -96,9 +96,3 @@ timezone="UTC" #kafka.security.protocol="SASL_SSL" #kafka.sasl.mechanism="GSSAPI" -# S3 specific settings: -s3.region = "eu-west-1" # default region, overridable - -# s3.kmsKeyId is recommended to set externally only: -# s3.kmsKeyId = "arn:aws:kms:eu-west-1:XXXX:key/YYYY" - 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 b11a1c143..d7096ed8e 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 @@ -19,16 +19,15 @@ import java.text.MessageFormat import java.time.Instant import com.typesafe.config.{Config, ConfigFactory} +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.FileSystem import org.apache.spark.SPARK_VERSION import org.apache.spark.sql.SparkSession import org.slf4j.{Logger, LoggerFactory} -import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider -import software.amazon.awssdk.regions.Region import za.co.absa.atum.AtumImplicits import za.co.absa.atum.core.{Atum, ControlType} -import za.co.absa.atum.persistence.S3KmsSettings import za.co.absa.enceladus.common.Constants.{InfoDateColumn, InfoVersionColumn} -import za.co.absa.enceladus.common.config.{JobConfigParser, PathConfig, S3Config} +import za.co.absa.enceladus.common.config.{FileSystems, JobConfigParser, PathConfig} import za.co.absa.enceladus.common.plugin.PostProcessingService import za.co.absa.enceladus.common.plugin.menas.{MenasPlugin, MenasRunUrl} import za.co.absa.enceladus.common.version.SparkVersionGuard @@ -37,7 +36,7 @@ import za.co.absa.enceladus.dao.rest.MenasConnectionStringParser import za.co.absa.enceladus.model.Dataset import za.co.absa.enceladus.plugins.builtin.errorsender.params.ErrorSenderPluginParams import za.co.absa.enceladus.utils.config.{ConfigReader, SecureConfig} -import za.co.absa.enceladus.utils.fs.{DistributedFsUtils, S3FsUtils} +import za.co.absa.enceladus.utils.fs.{DistributedFsUtils, FileSystemUtils, HadoopFsUtils, S3SdkFsUtils} import za.co.absa.enceladus.utils.general.ProjectMetadataTools import za.co.absa.enceladus.utils.modules.SourcePhase import za.co.absa.enceladus.utils.modules.SourcePhase.Standardization @@ -52,7 +51,7 @@ trait CommonJobExecution { protected case class PreparationResult(dataset: Dataset, reportVersion: Int, pathCfg: PathConfig, - s3Config: S3Config, + fileSystems: FileSystems, performance: PerformanceMeasurer) TimeZoneNormalizer.normalizeJVMTimeZone() @@ -82,18 +81,19 @@ trait CommonJobExecution { protected def prepareJob[T]() (implicit dao: MenasDAO, cmd: JobConfigParser[T], - fsUtils: DistributedFsUtils, spark: SparkSession): PreparationResult = { val confReader: ConfigReader = new ConfigReader(conf) confReader.logEffectiveConfigProps(Constants.ConfigKeysToRedact) - dao.authenticate() + + implicit val hadoopConf = spark.sparkContext.hadoopConfiguration + val dataset = dao.getDataset(cmd.datasetName, cmd.datasetVersion) val reportVersion = getReportVersion(cmd, dataset) val pathCfg: PathConfig = getPathConfig(cmd, dataset, reportVersion) - val s3Config: S3Config = getS3Config + val fileSystems: FileSystems = FileSystems.fromPathConfig(pathCfg) - validateOutputPath(s3Config, pathCfg) + validateOutputPath(pathCfg)(fileSystems) // Enable Spline import za.co.absa.spline.harvester.SparkLineageInitializer._ @@ -102,12 +102,12 @@ trait CommonJobExecution { // Enable non-default persistence storage level if provided in the command line cmd.persistStorageLevel.foreach(Atum.setCachingStorageLevel) - PreparationResult(dataset, reportVersion, pathCfg, s3Config, new PerformanceMeasurer(spark.sparkContext.appName)) + PreparationResult(dataset, reportVersion, pathCfg, fileSystems, new PerformanceMeasurer(spark.sparkContext.appName)) } - protected def validateOutputPath(s3Config: S3Config, pathConfig: PathConfig)(implicit fsUtils: DistributedFsUtils): Unit + protected def validateOutputPath(pathConfig: PathConfig)(implicit fileSystems: FileSystems): Unit - protected def validateIfPathAlreadyExists(s3Config: S3Config, path: String)(implicit fsUtils: DistributedFsUtils): Unit = { + protected def validateIfPathAlreadyExists(path: String)(implicit fsUtils: DistributedFsUtils): Unit = { if (fsUtils.exists(path)) { throw new IllegalStateException( s"Path $path already exists. Increment the run version, or delete $path" @@ -116,7 +116,7 @@ trait CommonJobExecution { } protected def runPostProcessing[T](sourcePhase: SourcePhase, preparationResult: PreparationResult, jobCmdConfig: JobConfigParser[T]) - (implicit spark: SparkSession, fileSystemVersionUtils: DistributedFsUtils): Unit = { + (implicit spark: SparkSession): Unit = { val outputPath = sourcePhase match { case Standardization => preparationResult.pathCfg.standardizationPath case _ => preparationResult.pathCfg.publishPath @@ -169,18 +169,6 @@ trait CommonJobExecution { ) } - protected def getS3Config: S3Config = { - val keyId = conf.getString("s3.kmsKeyId") - val region = Region.of(conf.getString("s3.region")) - - S3Config(region, keyId) - } - - protected def getS3FsUtil(implicit credentialsProvider: AwsCredentialsProvider): S3FsUtils = { - val s3Config = getS3Config - S3FsUtils(s3Config.region, S3KmsSettings(s3Config.kmsKeyId)) - } - private def buildPublishPath[T](cmd: JobConfigParser[T], ds: Dataset, reportVersion: Int): String = { val infoDateCol: String = InfoDateColumn val infoVersionCol: String = InfoVersionColumn @@ -251,11 +239,15 @@ trait CommonJobExecution { } } - private def getReportVersion[T](jobConfig: JobConfigParser[T], dataset: Dataset)(implicit fsUtils: DistributedFsUtils): Int = { + private def getReportVersion[T](jobConfig: JobConfigParser[T], dataset: Dataset)(implicit hadoopConf: Configuration): Int = { jobConfig.reportVersion match { case Some(version) => version case None => - val newVersion = fsUtils.getLatestVersion(dataset.hdfsPublishPath, jobConfig.reportDate) + 1 + import FileSystemUtils.FileSystemExt + + // publishFs for this specific feature (needed for missing reportVersion until reusable common FileSystems object is established) + implicit val tempPublishFs: FileSystem = FileSystemUtils.getFileSystemFromPath(dataset.hdfsPublishPath) + val newVersion = tempPublishFs.toFsUtils.getLatestVersion(dataset.hdfsPublishPath, jobConfig.reportDate) + 1 log.warn(s"Report version not provided, inferred report version: $newVersion") log.warn("This is an EXPERIMENTAL feature.") log.warn(" -> It can lead to issues when running multiple jobs on a dataset concurrently.") diff --git a/spark-jobs/src/main/scala/za/co/absa/enceladus/common/config/FileSystems.scala b/spark-jobs/src/main/scala/za/co/absa/enceladus/common/config/FileSystems.scala new file mode 100644 index 000000000..ce12c7256 --- /dev/null +++ b/spark-jobs/src/main/scala/za/co/absa/enceladus/common/config/FileSystems.scala @@ -0,0 +1,57 @@ +/* + * 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.common.config + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.FileSystem +import org.slf4j.{Logger, LoggerFactory} +import za.co.absa.enceladus.utils.fs.{FileSystemUtils, HadoopFsUtils} + +import scala.collection.concurrent.TrieMap + + +case class FileSystems(rawFs: FileSystem, publishFs: FileSystem, standardizationFs: FileSystem) + +object FileSystems { + val log: Logger = LoggerFactory.getLogger(this.getClass) + + def fromPathConfig(pathConfig: PathConfig)(implicit hadoopConf: Configuration): FileSystems = FileSystems( + rawFs = FileSystemUtils.getFileSystemFromPath(pathConfig.rawPath), + publishFs = FileSystemUtils.getFileSystemFromPath(pathConfig.publishPath), + standardizationFs = FileSystemUtils.getFileSystemFromPath(pathConfig.standardizationPath) + ) + +// private[config] val fsUtilsCache = TrieMap[FileSystem, HadoopFsUtils]() +// +// implicit class FileSystemExt(fs: FileSystem) { +// /** +// * Given the FileSystem object `fs` that this method is called on, +// * the appropriate HadoopFsUtils is either newly created or returned form cache. +// * @return +// */ +// def toFsUtils: HadoopFsUtils = { +// fsUtilsCache.getOrElseUpdate(fs, { +// log.info(s"reusing cached fsUtils for FS ${fs.getUri} / ${fs.toString}") +// new HadoopFsUtils()(fs)} +// +// ) +// } +// } + + +} + + 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 5325eccae..f46fae9ba 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 @@ -20,14 +20,11 @@ import java.io.{PrintWriter, StringWriter} import org.apache.spark.sql.functions.{lit, to_date} import org.apache.spark.sql.{DataFrame, SparkSession} import za.co.absa.atum.AtumImplicits -import za.co.absa.atum.AtumImplicits.{DataSetWrapper, SparkSessionWrapper} +import za.co.absa.atum.AtumImplicits.DataSetWrapper import za.co.absa.atum.core.Atum -import za.co.absa.atum.persistence.S3KmsSettings -import za.co.absa.atum.utils.S3Utils.StringS3LocationExt -import za.co.absa.enceladus.S3DefaultCredentialsProvider import za.co.absa.enceladus.common.Constants.{InfoDateColumn, InfoDateColumnString, InfoVersionColumn, ReportDateFormat} import za.co.absa.enceladus.common.RecordIdGeneration._ -import za.co.absa.enceladus.common.config.{JobConfigParser, PathConfig, S3Config} +import za.co.absa.enceladus.common.config.{FileSystems, JobConfigParser, PathConfig} import za.co.absa.enceladus.common.plugin.menas.MenasPlugin import za.co.absa.enceladus.common.{CommonJobExecution, Constants, RecordIdGeneration} import za.co.absa.enceladus.conformance.config.{ConformanceConfig, ConformanceConfigParser} @@ -37,7 +34,7 @@ import za.co.absa.enceladus.dao.MenasDAO import za.co.absa.enceladus.dao.auth.MenasCredentials import za.co.absa.enceladus.model.Dataset import za.co.absa.enceladus.standardization_conformance.config.StandardizationConformanceConfig -import za.co.absa.enceladus.utils.fs.DistributedFsUtils +import za.co.absa.enceladus.utils.fs.FileSystemUtils.FileSystemExt import za.co.absa.enceladus.utils.implicits.DataFrameImplicits.DataFrameEnhancements import za.co.absa.enceladus.utils.modules.SourcePhase import za.co.absa.enceladus.utils.performance.PerformanceMetricTools @@ -46,32 +43,33 @@ import za.co.absa.enceladus.utils.schema.SchemaUtils import scala.util.control.NonFatal import scala.util.{Failure, Success, Try} -trait ConformanceExecution extends CommonJobExecution with S3DefaultCredentialsProvider { +trait ConformanceExecution extends CommonJobExecution{ private val conformanceReader = new ConformancePropertiesProvider private val sourceId = SourcePhase.Conformance protected def prepareConformance[T](preparationResult: PreparationResult) (implicit dao: MenasDAO, cmd: ConformanceConfigParser[T], - fsUtils: DistributedFsUtils, spark: SparkSession): Unit = { - val stdDirSize = fsUtils.getDirectorySize(preparationResult.pathCfg.standardizationPath) + val stdFsUtils = preparationResult.fileSystems.standardizationFs.toFsUtils + + val stdDirSize = stdFsUtils.getDirectorySize(preparationResult.pathCfg.standardizationPath) preparationResult.performance.startMeasurement(stdDirSize) log.info(s"standardization path: ${preparationResult.pathCfg.standardizationPath}") log.info(s"publish path: ${preparationResult.pathCfg.publishPath}") + // Enable Control Framework + import za.co.absa.atum.AtumImplicits.SparkSessionWrapper + // reinitialize Control Framework in case of combined job - if(cmd.isInstanceOf[StandardizationConformanceConfig]) { + if (cmd.isInstanceOf[StandardizationConformanceConfig]) { spark.disableControlMeasuresTracking() } - val dataS3Location = preparationResult.pathCfg.standardizationPath.toS3Location(preparationResult.s3Config.region) - val infoS3Location = dataS3Location.copy(path = s"${dataS3Location.path}/_INFO") - - // Enable Control Framework - spark.enableControlMeasuresTrackingForS3(sourceS3Location = Some(infoS3Location), destinationS3Config = None) + // InputPath is standardizationPath in the combined job + spark.enableControlMeasuresTracking(s"${preparationResult.pathCfg.standardizationPath}/_INFO")(preparationResult.fileSystems.publishFs) .setControlMeasuresWorkflow(sourceId.toString) // Enable control framework performance optimization for pipeline-like jobs @@ -94,8 +92,9 @@ trait ConformanceExecution extends CommonJobExecution with S3DefaultCredentialsP } } - override def validateOutputPath(s3Config: S3Config, pathConfig: PathConfig)(implicit fsUtils: DistributedFsUtils): Unit = { - validateIfPathAlreadyExists(s3Config, pathConfig.publishPath) + override def validateOutputPath(pathConfig: PathConfig)(implicit fileSystems: FileSystems): Unit = { + // Conformance output is validated in the publish FS + validateIfPathAlreadyExists(pathConfig.publishPath)(fileSystems.publishFs.toFsUtils) } protected def readConformanceInputData(pathCfg: PathConfig)(implicit spark: SparkSession): DataFrame = { @@ -103,11 +102,11 @@ trait ConformanceExecution extends CommonJobExecution with S3DefaultCredentialsP } protected def conform[T](inputData: DataFrame, preparationResult: PreparationResult) - (implicit spark: SparkSession, cmd: ConformanceConfigParser[T], dao: MenasDAO, - fsUtils: DistributedFsUtils): DataFrame = { + (implicit spark: SparkSession, cmd: ConformanceConfigParser[T], dao: MenasDAO): DataFrame = { val recordIdGenerationStrategy = getRecordIdGenerationStrategyFromConfig(conf) implicit val featureSwitcher: FeatureSwitches = conformanceReader.readFeatureSwitches() + implicit val stdFs = preparationResult.fileSystems.standardizationFs Try { handleControlInfoValidation() @@ -135,16 +134,17 @@ trait ConformanceExecution extends CommonJobExecution with S3DefaultCredentialsP preparationResult: PreparationResult, menasCredentials: MenasCredentials) (implicit spark: SparkSession, - cmd: ConformanceConfigParser[T], - fsUtils: DistributedFsUtils): Unit = { + cmd: ConformanceConfigParser[T]): Unit = { val cmdLineArgs: String = args.mkString(" ") + val stdFs = preparationResult.fileSystems.standardizationFs + val publishFs = preparationResult.fileSystems.publishFs PerformanceMetricTools.addJobInfoToAtumMetadata( "conform", preparationResult.pathCfg.standardizationPath, preparationResult.pathCfg.publishPath, menasCredentials.username, cmdLineArgs - ) + )(spark, stdFs.toFsUtils) val withPartCols = result .withColumnIfDoesNotExist(InfoDateColumn, to_date(lit(cmd.reportDate), ReportDateFormat)) @@ -161,7 +161,7 @@ trait ConformanceExecution extends CommonJobExecution with S3DefaultCredentialsP withPartCols.write.parquet(preparationResult.pathCfg.publishPath) - val publishDirSize = fsUtils.getDirectorySize(preparationResult.pathCfg.publishPath) + val publishDirSize = publishFs.toFsUtils.getDirectorySize(preparationResult.pathCfg.publishPath) preparationResult.performance.finishMeasurement(publishDirSize, recordCount) PerformanceMetricTools.addPerformanceMetricsToAtumMetadata( spark, @@ -169,17 +169,13 @@ trait ConformanceExecution extends CommonJobExecution with S3DefaultCredentialsP preparationResult.pathCfg.standardizationPath, preparationResult.pathCfg.publishPath, menasCredentials.username, cmdLineArgs - ) - - val infoFilePath = s"${preparationResult.pathCfg.publishPath}/_INFO" - val infoFileLocation = infoFilePath.toS3Location(preparationResult.s3Config.region) - log.info(s"infoFilePath = $infoFilePath, infoFileLocation = $infoFileLocation") + )(stdFs.toFsUtils, publishFs.toFsUtils) - withPartCols.writeInfoFileOnS3(infoFileLocation, S3KmsSettings(preparationResult.s3Config.kmsKeyId)) + withPartCols.writeInfoFile(preparationResult.pathCfg.publishPath)(publishFs) writePerformanceMetrics(preparationResult.performance, cmd) if (conformanceReader.isAutocleanStdFolderEnabled()) { - fsUtils.deleteDirectoryRecursively(preparationResult.pathCfg.standardizationPath) + stdFs.toFsUtils.deleteDirectoryRecursively(preparationResult.pathCfg.standardizationPath) } log.info(s"$sourceId finished successfully") } diff --git a/spark-jobs/src/main/scala/za/co/absa/enceladus/conformance/DynamicConformanceJob.scala b/spark-jobs/src/main/scala/za/co/absa/enceladus/conformance/DynamicConformanceJob.scala index b48092382..fb254892f 100644 --- a/spark-jobs/src/main/scala/za/co/absa/enceladus/conformance/DynamicConformanceJob.scala +++ b/spark-jobs/src/main/scala/za/co/absa/enceladus/conformance/DynamicConformanceJob.scala @@ -32,7 +32,6 @@ object DynamicConformanceJob extends ConformanceExecution { implicit val cmd: ConformanceConfig = ConformanceConfig.getFromArguments(args) implicit val spark: SparkSession = obtainSparkSession(jobName) // initialize spark - implicit val fsUtils: DistributedFsUtils = getS3FsUtil val menasCredentials = cmd.menasCredentialsFactory.getInstance() implicit val dao: MenasDAO = RestDaoFactory.getInstance(menasCredentials, menasBaseUrls) 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 3ae28f8f7..cd5336c31 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 @@ -19,6 +19,7 @@ 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 import org.apache.spark.sql.functions._ import org.apache.spark.sql.{DataFrame, SparkSession} @@ -32,8 +33,9 @@ import za.co.absa.enceladus.dao.MenasDAO import za.co.absa.enceladus.dao.auth.{MenasCredentialsFactory, MenasKerberosCredentialsFactory, MenasPlainCredentialsFactory} import za.co.absa.enceladus.dao.rest.{MenasConnectionStringParser, RestDaoFactory} import za.co.absa.enceladus.model.Dataset -import za.co.absa.enceladus.utils.fs.HdfsUtils +import za.co.absa.enceladus.utils.fs.{FileSystemUtils, HadoopFsUtils} import za.co.absa.hyperdrive.ingestor.api.transformer.{StreamTransformer, StreamTransformerFactory} +import FileSystemUtils.FileSystemExt class HyperConformance (implicit cmd: ConformanceConfig, featureSwitches: FeatureSwitches, @@ -66,7 +68,8 @@ class HyperConformance (implicit cmd: ConformanceConfig, val infoDateColumn = infoDateFactory.getInfoDateColumn(rawDf) // using HDFS implementation until HyperConformance is S3-ready - implicit val fsUtils: HdfsUtils = new HdfsUtils(sparkSession.sparkContext.hadoopConfiguration) + implicit val hdfs = FileSystem.get(sparkSession.sparkContext.hadoopConfiguration) + implicit val hdfsUtils = hdfs.toFsUtils val conformedDf = DynamicInterpreter().interpret(conformance, rawDf) .withColumnIfDoesNotExist(InfoDateColumn, coalesce(infoDateColumn, current_date())) 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 00d9baae2..6897fd6b0 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 @@ -15,6 +15,7 @@ package za.co.absa.enceladus.conformance.interpreter +import org.apache.hadoop.fs.FileSystem import org.apache.spark.sql.execution.command.ExplainCommand import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.StructType @@ -31,12 +32,12 @@ import za.co.absa.enceladus.model.conformanceRule.{ConformanceRule, _} import za.co.absa.enceladus.model.{Dataset => ConfDataset} import za.co.absa.enceladus.utils.error.ErrorMessage import za.co.absa.enceladus.utils.explode.ExplosionContext -import za.co.absa.enceladus.utils.fs.DistributedFsUtils 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.enceladus.utils.fs.FileSystemUtils.FileSystemExt -case class DynamicInterpreter(implicit fsUtils: DistributedFsUtils) { +case class DynamicInterpreter(implicit inputFs: FileSystem) { private val log = LoggerFactory.getLogger(this.getClass) /** @@ -49,8 +50,10 @@ case class DynamicInterpreter(implicit fsUtils: DistributedFsUtils) { * */ def interpret[T](conformance: ConfDataset, inputDf: Dataset[Row], jobShortName: String = "Conformance") - (implicit spark: SparkSession, dao: MenasDAO, - progArgs: ConformanceConfigParser[T], featureSwitches: FeatureSwitches): DataFrame = { + (implicit spark: SparkSession, + dao: MenasDAO, + progArgs: ConformanceConfigParser[T], + featureSwitches: FeatureSwitches): DataFrame = { implicit val interpreterContext: InterpreterContext = InterpreterContext(inputDf.schema, conformance, featureSwitches, jobShortName, spark, dao, InterpreterContextArgs.fromConformanceConfig(progArgs)) @@ -268,7 +271,7 @@ case class DynamicInterpreter(implicit fsUtils: DistributedFsUtils) { val mappingTableDef = ictx.dao.getMappingTable(rule.mappingTable, rule.mappingTableVersion) val mappingTablePath = PartitioningUtils.getPartitionedPathName(mappingTableDef.hdfsPath, ictx.progArgs.reportDate) - val mappingTableSize = fsUtils.getDirectorySizeNoHidden(mappingTablePath) + val mappingTableSize = inputFs.toFsUtils.getDirectorySizeNoHidden(mappingTablePath) (mappingTableSize / (1024 * 1024)).toInt } 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 0d0a57d57..82944c83a 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 @@ -22,11 +22,9 @@ 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.atum.persistence.S3KmsSettings import za.co.absa.enceladus.S3DefaultCredentialsProvider import za.co.absa.enceladus.common.RecordIdGeneration.getRecordIdGenerationStrategyFromConfig -import za.co.absa.atum.utils.S3Utils.StringS3LocationExt -import za.co.absa.enceladus.common.config.{JobConfigParser, PathConfig, S3Config} +import za.co.absa.enceladus.common.config.{FileSystems, JobConfigParser, PathConfig} import za.co.absa.enceladus.common.plugin.menas.MenasPlugin import za.co.absa.enceladus.common.{CommonJobExecution, Constants} import za.co.absa.enceladus.dao.MenasDAO @@ -35,12 +33,14 @@ import za.co.absa.enceladus.model.Dataset import za.co.absa.enceladus.standardization.config.{StandardizationConfig, StandardizationConfigParser} import za.co.absa.enceladus.standardization.interpreter.StandardizationInterpreter import za.co.absa.enceladus.standardization.interpreter.stages.PlainSchemaGenerator -import za.co.absa.enceladus.utils.fs.{DistributedFsUtils, HdfsUtils} +import za.co.absa.enceladus.utils.fs.{DistributedFsUtils, HadoopFsUtils} import za.co.absa.enceladus.utils.modules.SourcePhase import za.co.absa.enceladus.utils.performance.PerformanceMetricTools import za.co.absa.enceladus.utils.schema.{MetadataKeys, SchemaUtils, SparkUtils} import za.co.absa.enceladus.utils.udf.UDFLibrary import za.co.absa.enceladus.utils.validation.ValidationException +import za.co.absa.enceladus.utils.fs.FileSystemUtils.FileSystemExt +import org.apache.hadoop.fs.FileSystem import scala.util.control.NonFatal @@ -52,23 +52,15 @@ trait StandardizationExecution extends CommonJobExecution with S3DefaultCredenti preparationResult: PreparationResult) (implicit dao: MenasDAO, cmd: StandardizationConfigParser[T], - fsUtils: DistributedFsUtils, spark: SparkSession): StructType = { + val rawFs = preparationResult.fileSystems.rawFs + + val stdDirSize = rawFs.toFsUtils.getDirectorySize(preparationResult.pathCfg.rawPath) + preparationResult.performance.startMeasurement(stdDirSize) - val stdDirSize = fsUtils.getDirectorySize(preparationResult.pathCfg.rawPath) - preparationResult.performance.startMeasurement(stdDirSize) // Enable Control Framework import za.co.absa.atum.AtumImplicits.SparkSessionWrapper - - val inputDataS3Location = preparationResult.pathCfg.rawPath.toS3Location(preparationResult.s3Config.region) - val inputInfoS3Location = inputDataS3Location.copy(path = s"${inputDataS3Location.path}/_INFO") - - val outputDataS3Location = preparationResult.pathCfg.standardizationPath.toS3Location(preparationResult.s3Config.region) - val outputInfoS3Location = outputDataS3Location.copy(path = s"${outputDataS3Location.path}/_INFO") - val kmsSettings = S3KmsSettings(preparationResult.s3Config.kmsKeyId) - - spark.enableControlMeasuresTrackingForS3(sourceS3Location = Some(inputInfoS3Location), - destinationS3Config = Some(outputInfoS3Location, kmsSettings)) + spark.enableControlMeasuresTracking(sourceInfoFile = s"${preparationResult.pathCfg.rawPath}/_INFO")(rawFs) .setControlMeasuresWorkflow(sourceId.toString) log.info(s"raw path: ${preparationResult.pathCfg.rawPath}") @@ -95,7 +87,7 @@ trait StandardizationExecution extends CommonJobExecution with S3DefaultCredenti PerformanceMetricTools.addJobInfoToAtumMetadata("std", preparationResult.pathCfg.rawPath, preparationResult.pathCfg.standardizationPath, - menasCredentials.username, args.mkString(" ")) + menasCredentials.username, args.mkString(" "))(spark, rawFs.toFsUtils) dao.getSchema(preparationResult.dataset.schemaName, preparationResult.dataset.schemaVersion) } @@ -108,8 +100,9 @@ trait StandardizationExecution extends CommonJobExecution with S3DefaultCredenti } } - override def validateOutputPath(s3Config: S3Config, pathConfig: PathConfig)(implicit fsUtils: DistributedFsUtils): Unit = { - validateIfPathAlreadyExists(s3Config, pathConfig.standardizationPath) + override def validateOutputPath(pathConfig: PathConfig)(implicit fileSystems: FileSystems): Unit = { + // Std output is validated in the std FS + validateIfPathAlreadyExists(pathConfig.standardizationPath)(fileSystems.standardizationFs.toFsUtils) } protected def readStandardizationInputData[T](schema: StructType, @@ -117,7 +110,7 @@ trait StandardizationExecution extends CommonJobExecution with S3DefaultCredenti path: String, dataset: Dataset) (implicit spark: SparkSession, - fsUtils: DistributedFsUtils, + rawFs: FileSystem, dao: MenasDAO): DataFrame = { val numberOfColumns = schema.fields.length val standardizationReader = new StandardizationPropertiesProvider() @@ -131,7 +124,7 @@ trait StandardizationExecution extends CommonJobExecution with S3DefaultCredenti } val dfWithSchema = readerWithOptSchema.load(s"$path/*") - ensureSplittable(dfWithSchema, path, schema) + ensureSplittable(dfWithSchema, path, schema)(spark, rawFs.toFsUtils) } private def getColumnNameOfCorruptRecord[R](schema: StructType, cmd: StandardizationConfigParser[R]) @@ -175,15 +168,18 @@ trait StandardizationExecution extends CommonJobExecution with S3DefaultCredenti schema: StructType, cmd: StandardizationConfigParser[T], menasCredentials: MenasCredentials) - (implicit spark: SparkSession, - fsUtils: DistributedFsUtils): DataFrame = { + (implicit spark: SparkSession): DataFrame = { import za.co.absa.atum.AtumImplicits._ + + val rawFs = preparationResult.fileSystems.rawFs + val stdFs = preparationResult.fileSystems.standardizationFs + val fieldRenames = SchemaUtils.getRenamesInSchema(schema) fieldRenames.foreach { - case (destinationName, sourceName) => standardizedDF.registerColumnRename(sourceName, destinationName) + case (destinationName, sourceName) => standardizedDF.registerColumnRename(sourceName, destinationName)(rawFs) } - standardizedDF.setCheckpoint(s"$sourceId - End", persistInDatabase = false) + standardizedDF.setCheckpoint(s"$sourceId - End", persistInDatabase = false)(rawFs) val recordCount = standardizedDF.lastCheckpointRowCount match { case None => standardizedDF.count @@ -199,8 +195,7 @@ trait StandardizationExecution extends CommonJobExecution with S3DefaultCredenti // Store performance metrics // (record count, directory sizes, elapsed time, etc. to _INFO file metadata and performance file) - - val stdDirSize = fsUtils.getDirectorySize(preparationResult.pathCfg.standardizationPath) + val stdDirSize = stdFs.toFsUtils.getDirectorySize(preparationResult.pathCfg.standardizationPath) preparationResult.performance.finishMeasurement(stdDirSize, recordCount) PerformanceMetricTools.addPerformanceMetricsToAtumMetadata( @@ -210,16 +205,13 @@ trait StandardizationExecution extends CommonJobExecution with S3DefaultCredenti preparationResult.pathCfg.standardizationPath, menasCredentials.username, args.mkString(" ") - ) + )(rawFs.toFsUtils, stdFs.toFsUtils) cmd.rowTag.foreach(rowTag => Atum.setAdditionalInfo("xml_row_tag" -> rowTag)) cmd.csvDelimiter.foreach(delimiter => Atum.setAdditionalInfo("csv_delimiter" -> delimiter)) - val infoFilePath = s"${preparationResult.pathCfg.standardizationPath}/_INFO" - val infoFileLocation = infoFilePath.toS3Location(preparationResult.s3Config.region) - log.info(s"infoFilePath = $infoFilePath, infoFileLocation = $infoFileLocation") - - standardizedDF.writeInfoFileOnS3(infoFileLocation, S3KmsSettings(preparationResult.s3Config.kmsKeyId)) + log.info(s"infoFilePath = ${preparationResult.pathCfg.standardizationPath}/_INFO") + standardizedDF.writeInfoFile(preparationResult.pathCfg.standardizationPath)(stdFs) writePerformanceMetrics(preparationResult.performance, cmd) log.info(s"$sourceId finished successfully") standardizedDF @@ -241,7 +233,7 @@ trait StandardizationExecution extends CommonJobExecution with S3DefaultCredenti log.warn("Dataset is stored in a non-splittable format. This can have a severe performance impact.") fsUtils match { - case utils: HdfsUtils => + case utils: HadoopFsUtils => val tempParquetDir = s"/tmp/nonsplittable-to-parquet-${UUID.randomUUID()}" log.warn(s"Converting to Parquet in temporary dir: $tempParquetDir") diff --git a/spark-jobs/src/main/scala/za/co/absa/enceladus/standardization/StandardizationJob.scala b/spark-jobs/src/main/scala/za/co/absa/enceladus/standardization/StandardizationJob.scala index c123a5a1b..d2c9fa769 100644 --- a/spark-jobs/src/main/scala/za/co/absa/enceladus/standardization/StandardizationJob.scala +++ b/spark-jobs/src/main/scala/za/co/absa/enceladus/standardization/StandardizationJob.scala @@ -15,6 +15,7 @@ package za.co.absa.enceladus.standardization +import org.apache.hadoop.fs.FileSystem import org.apache.spark.sql.SparkSession import za.co.absa.enceladus.dao.MenasDAO import za.co.absa.enceladus.dao.rest.RestDaoFactory @@ -31,7 +32,6 @@ object StandardizationJob extends StandardizationExecution { implicit val cmd: StandardizationConfig = StandardizationConfig.getFromArguments(args) implicit val spark: SparkSession = obtainSparkSession(jobName) - implicit val fsUtils: DistributedFsUtils = getS3FsUtil implicit val udfLib: UDFLibrary = new UDFLibrary val menasCredentials = cmd.menasCredentialsFactory.getInstance() @@ -39,6 +39,7 @@ object StandardizationJob extends StandardizationExecution { val preparationResult = prepareJob() val schema = prepareStandardization(args, menasCredentials, preparationResult) + implicit val rawFs: FileSystem = preparationResult.fileSystems.rawFs val inputData = readStandardizationInputData(schema, cmd, preparationResult.pathCfg.rawPath, preparationResult.dataset) try { diff --git a/spark-jobs/src/main/scala/za/co/absa/enceladus/standardization_conformance/StandardizationAndConformanceExecution.scala b/spark-jobs/src/main/scala/za/co/absa/enceladus/standardization_conformance/StandardizationAndConformanceExecution.scala index 1347fc6c2..036df5d85 100644 --- a/spark-jobs/src/main/scala/za/co/absa/enceladus/standardization_conformance/StandardizationAndConformanceExecution.scala +++ b/spark-jobs/src/main/scala/za/co/absa/enceladus/standardization_conformance/StandardizationAndConformanceExecution.scala @@ -16,12 +16,12 @@ package za.co.absa.enceladus.standardization_conformance import za.co.absa.enceladus.common.CommonJobExecution -import za.co.absa.enceladus.common.config.{JobConfigParser, PathConfig, S3Config} +import za.co.absa.enceladus.common.config.{FileSystems, JobConfigParser, PathConfig} import za.co.absa.enceladus.conformance.ConformanceExecution import za.co.absa.enceladus.model.Dataset import za.co.absa.enceladus.standardization.StandardizationExecution import za.co.absa.enceladus.standardization_conformance.config.StandardizationConformanceConfig -import za.co.absa.enceladus.utils.fs.DistributedFsUtils +import za.co.absa.enceladus.utils.fs.FileSystemUtils.FileSystemExt trait StandardizationAndConformanceExecution extends StandardizationExecution with ConformanceExecution @@ -36,8 +36,11 @@ trait StandardizationAndConformanceExecution extends StandardizationExecution publishPath = publishPathOverride.getOrElse(defaultConfig.publishPath)) } - override def validateOutputPath(s3Config: S3Config, pathConfig: PathConfig)(implicit fsUtils: DistributedFsUtils): Unit = { - validateIfPathAlreadyExists(s3Config, pathConfig.standardizationPath) - validateIfPathAlreadyExists(s3Config, pathConfig.publishPath) + override def validateOutputPath(pathConfig: PathConfig)(implicit fileSystems: FileSystems): Unit = { + // Std output is validated in the std FS + validateIfPathAlreadyExists(pathConfig.standardizationPath)(fileSystems.standardizationFs.toFsUtils) + + // publish output is validated in the publish FS + validateIfPathAlreadyExists(pathConfig.publishPath)(fileSystems.publishFs.toFsUtils) } } diff --git a/spark-jobs/src/main/scala/za/co/absa/enceladus/standardization_conformance/StandardizationAndConformanceJob.scala b/spark-jobs/src/main/scala/za/co/absa/enceladus/standardization_conformance/StandardizationAndConformanceJob.scala index 6bb023d1c..12a4614e9 100644 --- a/spark-jobs/src/main/scala/za/co/absa/enceladus/standardization_conformance/StandardizationAndConformanceJob.scala +++ b/spark-jobs/src/main/scala/za/co/absa/enceladus/standardization_conformance/StandardizationAndConformanceJob.scala @@ -19,7 +19,6 @@ import org.apache.spark.sql.SparkSession import za.co.absa.enceladus.dao.MenasDAO import za.co.absa.enceladus.dao.rest.RestDaoFactory import za.co.absa.enceladus.standardization_conformance.config.StandardizationConformanceConfig -import za.co.absa.enceladus.utils.fs.DistributedFsUtils import za.co.absa.enceladus.utils.modules.SourcePhase import za.co.absa.enceladus.utils.udf.UDFLibrary @@ -31,13 +30,13 @@ object StandardizationAndConformanceJob extends StandardizationAndConformanceExe implicit val cmd: StandardizationConformanceConfig = StandardizationConformanceConfig.getFromArguments(args) implicit val spark: SparkSession = obtainSparkSession(jobName) - implicit val fsUtils: DistributedFsUtils = getS3FsUtil implicit val udfLib: UDFLibrary = new UDFLibrary val menasCredentials = cmd.menasCredentialsFactory.getInstance() implicit val dao: MenasDAO = RestDaoFactory.getInstance(menasCredentials, menasBaseUrls) val preparationResult = prepareJob() val schema = prepareStandardization(args, menasCredentials, preparationResult) + implicit val rawFs = preparationResult.fileSystems.rawFs val inputData = readStandardizationInputData(schema, cmd, preparationResult.pathCfg.rawPath, preparationResult.dataset) try { 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 f93d56426..c14e5344f 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 @@ -17,20 +17,18 @@ package za.co.absa.enceladus.conformance.interpreter import org.apache.spark.sql.functions._ import org.mockito.Mockito.{mock, when => mockWhen} +import org.scalatest.BeforeAndAfterAll import org.scalatest.funsuite.AnyFunSuite -import org.scalatest.{BeforeAndAfterAll} import za.co.absa.enceladus.conformance.config.ConformanceConfig import za.co.absa.enceladus.conformance.datasource.DataSource -import za.co.absa.enceladus.dao.MenasDAO import za.co.absa.enceladus.conformance.samples._ -import za.co.absa.enceladus.utils.fs.HdfsUtils -import za.co.absa.enceladus.utils.testUtils.SparkTestBase +import za.co.absa.enceladus.dao.MenasDAO +import za.co.absa.enceladus.utils.testUtils.{HadoopFsTestBase, SparkTestBase} -class ArrayConformanceSuite extends AnyFunSuite with SparkTestBase with BeforeAndAfterAll { +class ArrayConformanceSuite extends AnyFunSuite with SparkTestBase with BeforeAndAfterAll with HadoopFsTestBase { import spark.implicits._ // spark.enableControlFrameworkTracking() - implicit val fsUtils: HdfsUtils = new HdfsUtils(spark.sparkContext.hadoopConfiguration) implicit var dao: MenasDAO = _ implicit var progArgs: ConformanceConfig = _ 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 9d5c94eca..5450d5ee7 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,15 +22,14 @@ 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.fs.HdfsUtils -import za.co.absa.enceladus.utils.testUtils.{LoggerTestBase, SparkTestBase} +import za.co.absa.enceladus.utils.testUtils.{HadoopFsTestBase, LoggerTestBase, SparkTestBase} case class MyMappingTable(id: Int, mappedAttr: MyMappingTableInner) case class MyMappingTableInner(description: String, name: String) case class MyData(id: Int, toJoin: Int) case class MyDataConfd(id: Int, toJoin: Int, confMapping: MyMappingTableInner) -class ChorusMockSuite extends AnyFunSuite with SparkTestBase with LoggerTestBase { +class ChorusMockSuite extends AnyFunSuite with SparkTestBase with LoggerTestBase with HadoopFsTestBase { def testChorusMockData(useExperimentalMappingRule: Boolean): Unit = { val d = Seq( @@ -72,8 +71,6 @@ class ChorusMockSuite extends AnyFunSuite with SparkTestBase with LoggerTestBase .setControlFrameworkEnabled(enableCF) .setBroadcastStrategyMode(Never) - implicit val fsUtils: HdfsUtils = new HdfsUtils(spark.sparkContext.hadoopConfiguration) - val confd = DynamicInterpreter().interpret(conformanceDef, inputDf).repartition(2) logDataFrameContent(confd) 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 fdc49e9b4..69160f4c6 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 @@ -15,20 +15,20 @@ package za.co.absa.enceladus.conformance.interpreter +import org.json4s._ +import org.json4s.jackson._ import org.mockito.Mockito.{mock, when => mockWhen} -import org.scalatest.{BeforeAndAfterAll} +import org.scalatest.BeforeAndAfterAll +import org.scalatest.funsuite.AnyFunSuite import za.co.absa.atum.model.ControlMeasure import za.co.absa.enceladus.conformance.config.ConformanceConfig import za.co.absa.enceladus.conformance.datasource.DataSource -import za.co.absa.enceladus.dao.MenasDAO import za.co.absa.enceladus.conformance.samples._ -import za.co.absa.enceladus.utils.testUtils.{LoggerTestBase, SparkTestBase} -import org.json4s._ -import org.json4s.jackson._ -import org.scalatest.funsuite.AnyFunSuite -import za.co.absa.enceladus.utils.fs.{FileReader, HdfsUtils} +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} -class InterpreterSuite extends AnyFunSuite with SparkTestBase with BeforeAndAfterAll with LoggerTestBase { +class InterpreterSuite extends AnyFunSuite with SparkTestBase with BeforeAndAfterAll with LoggerTestBase with HadoopFsTestBase { override def beforeAll(): Unit = { super.beforeAll @@ -56,7 +56,6 @@ class InterpreterSuite extends AnyFunSuite with SparkTestBase with BeforeAndAfte val isCatalystWorkaroundEnabled = true import spark.implicits._ - implicit val fsUtils: HdfsUtils = new HdfsUtils(spark.sparkContext.hadoopConfiguration) val mappingTablePattern = "{0}/{1}/{2}" val dfs = DataSource.getDataFrame(EmployeeConformance.employeeDS.hdfsPath, "2017-11-01", mappingTablePattern) @@ -114,7 +113,6 @@ class InterpreterSuite extends AnyFunSuite with SparkTestBase with BeforeAndAfte val isCatalystWorkaroundEnabled = true import spark.implicits._ - implicit val fsUtils: HdfsUtils = new HdfsUtils(spark.sparkContext.hadoopConfiguration) val mappingTablePattern = "{0}/{1}/{2}" val dfs = DataSource.getDataFrame(TradeConformance.tradeDS.hdfsPath, "2017-11-01", mappingTablePattern) 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 5853d33e9..9aa466482 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,10 +22,9 @@ 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.fs.HdfsUtils -import za.co.absa.enceladus.utils.testUtils.{LoggerTestBase, SparkTestBase} +import za.co.absa.enceladus.utils.testUtils.{HadoopFsTestBase, LoggerTestBase, SparkTestBase} -class LiteralJoinMappingRuleTest extends AnyFunSuite with SparkTestBase with LoggerTestBase { +class LiteralJoinMappingRuleTest extends AnyFunSuite with SparkTestBase with LoggerTestBase with HadoopFsTestBase { def testMappingRuleWithLiteral(useExperimentalMappingRule: Boolean): Unit = { @@ -68,8 +67,6 @@ class LiteralJoinMappingRuleTest extends AnyFunSuite with SparkTestBase with Log .setControlFrameworkEnabled(enableCF) .setBroadcastStrategyMode(Never) - implicit val fsUtils: HdfsUtils = new HdfsUtils(spark.sparkContext.hadoopConfiguration) - val confd = DynamicInterpreter().interpret(conformanceDef, inputDf).repartition(2) confd.write.mode("overwrite").parquet("_testOutput") 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 b044de35c..8d2940894 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,17 +17,14 @@ 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.fs.HdfsUtils -import za.co.absa.enceladus.utils.testUtils.SparkTestBase +import za.co.absa.enceladus.utils.testUtils.{HadoopFsTestBase, SparkTestBase} /** * The purpose of these tests is to ensure Catalyst optimizer issue is handled. * * Without applying a workaround any test in this suite makes Spark freeze. */ -class NestedStructSuite extends AnyFunSuite with SparkTestBase with NestedStructsFixture { - - implicit val fsUtils: HdfsUtils = new HdfsUtils(spark.sparkContext.hadoopConfiguration) +class NestedStructSuite extends AnyFunSuite with SparkTestBase with NestedStructsFixture with HadoopFsTestBase { test("Test Dynamic Conformance does not hang on many mixed conformance rules") { implicit val featureSwitches: FeatureSwitches = FeatureSwitches() 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 085fd7dbf..3cab124b6 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,11 +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.fs.HdfsUtils import za.co.absa.enceladus.utils.general.JsonUtils -import za.co.absa.enceladus.utils.testUtils.{LoggerTestBase, SparkTestBase} +import za.co.absa.enceladus.utils.testUtils.{HadoopFsTestBase, LoggerTestBase, SparkTestBase} -class CastingRuleSuite extends AnyFunSuite with SparkTestBase with LoggerTestBase { +class CastingRuleSuite extends AnyFunSuite with SparkTestBase with LoggerTestBase with HadoopFsTestBase { private val ruleName = "Casting rule" private val columnName = "dummy" @@ -52,8 +51,6 @@ class CastingRuleSuite extends AnyFunSuite with SparkTestBase with LoggerTestBas .setCatalystWorkaroundEnabled(isCatalystWorkaroundEnabled) .setControlFrameworkEnabled(enableCF) - implicit val fsUtils: HdfsUtils = new HdfsUtils(spark.sparkContext.hadoopConfiguration) - val conformed = DynamicInterpreter().interpret(CastingRuleSamples.ordersDS, inputDf).cache val conformedJSON = JsonUtils.prettySparkJSON(conformed.orderBy($"id").toJSON.collect) diff --git a/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/rules/MappingRuleBroadcastSuite.scala b/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/rules/MappingRuleBroadcastSuite.scala index 479cb7f32..7b9309586 100644 --- a/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/rules/MappingRuleBroadcastSuite.scala +++ b/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/rules/MappingRuleBroadcastSuite.scala @@ -24,18 +24,15 @@ import za.co.absa.enceladus.conformance.interpreter.rules.testcasefactories.Nest import za.co.absa.enceladus.conformance.interpreter.rules.testcasefactories.SimpleTestCaseFactory._ import za.co.absa.enceladus.conformance.interpreter.rules.testcasefactories.{NestedTestCaseFactory, SimpleTestCaseFactory} import za.co.absa.enceladus.utils.error.ErrorMessage -import za.co.absa.enceladus.utils.fs.HdfsUtils import za.co.absa.enceladus.utils.general.JsonUtils -import za.co.absa.enceladus.utils.testUtils.{LoggerTestBase, SparkTestBase} +import za.co.absa.enceladus.utils.testUtils.{HadoopFsTestBase, LoggerTestBase, SparkTestBase} -class MappingRuleBroadcastSuite extends AnyFunSuite with SparkTestBase with LoggerTestBase with BeforeAndAfterAll { +class MappingRuleBroadcastSuite extends AnyFunSuite with SparkTestBase with LoggerTestBase with BeforeAndAfterAll with HadoopFsTestBase { import spark.implicits._ private val simpleTestCaseFactory = new SimpleTestCaseFactory() private val nestedTestCaseFactory = new NestedTestCaseFactory() - implicit val fsUtils: HdfsUtils = new HdfsUtils(spark.sparkContext.hadoopConfiguration) - override def beforeAll(): Unit = { super.beforeAll() simpleTestCaseFactory.createMappingTables() diff --git a/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/rules/MappingRuleSuite.scala b/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/rules/MappingRuleSuite.scala index 1b92ebbe0..8fd156bec 100644 --- a/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/rules/MappingRuleSuite.scala +++ b/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/rules/MappingRuleSuite.scala @@ -21,14 +21,11 @@ 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.fs.HdfsUtils -import za.co.absa.enceladus.utils.testUtils.{LoggerTestBase, SparkTestBase} +import za.co.absa.enceladus.utils.testUtils.{HadoopFsTestBase, LoggerTestBase, SparkTestBase} -class MappingRuleSuite extends AnyFunSuite with SparkTestBase with LoggerTestBase with BeforeAndAfterAll { +class MappingRuleSuite extends AnyFunSuite with SparkTestBase with LoggerTestBase with BeforeAndAfterAll with HadoopFsTestBase { private val testCaseFactory = new SimpleTestCaseFactory() - implicit val fsUtils: HdfsUtils = new HdfsUtils(spark.sparkContext.hadoopConfiguration) - override def beforeAll(): Unit = { super.beforeAll() testCaseFactory.createMappingTables() 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 78ea290bb..fbbdd5fc6 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,10 +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.fs.HdfsUtils -import za.co.absa.enceladus.utils.testUtils.{LoggerTestBase, SparkTestBase} +import za.co.absa.enceladus.utils.testUtils.{HadoopFsTestBase, LoggerTestBase, SparkTestBase} -class NegationRuleSuite extends AnyFunSuite with SparkTestBase with LoggerTestBase{ +class NegationRuleSuite extends AnyFunSuite with SparkTestBase with LoggerTestBase with HadoopFsTestBase { import spark.implicits._ @@ -120,7 +119,7 @@ class NegationRuleSuite extends AnyFunSuite with SparkTestBase with LoggerTestBa .setExperimentalMappingRuleEnabled(experimentalMR) .setCatalystWorkaroundEnabled(isCatalystWorkaroundEnabled) .setControlFrameworkEnabled(enableCF) - implicit val fsUtils: HdfsUtils = new HdfsUtils(spark.sparkContext.hadoopConfiguration) + val conformed = DynamicInterpreter().interpret(enceladusDataset, inputDf).cache val conformedJSON = conformed.toJSON.collect().mkString("\n") if (conformedJSON != expectedJSON) { 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 2eb36d395..0628d925a 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 @@ -20,10 +20,10 @@ import org.scalatest.funsuite.AnyFunSuite 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.fs.HdfsUtils -import za.co.absa.enceladus.utils.testUtils.SparkTestBase +import za.co.absa.enceladus.utils.fs.HadoopFsUtils +import za.co.absa.enceladus.utils.testUtils.{HadoopFsTestBase, SparkTestBase} -class RuleOptimizationSuite extends AnyFunSuite with SparkTestBase { +class RuleOptimizationSuite extends AnyFunSuite with SparkTestBase with HadoopFsTestBase { private val schemaJson = """{ @@ -111,8 +111,6 @@ class RuleOptimizationSuite extends AnyFunSuite with SparkTestBase { null, null) - implicit val fsUtils: HdfsUtils = new HdfsUtils(spark.sparkContext.hadoopConfiguration) - test("Test non-mapping rules are not grouped") { val rules: List[ConformanceRule] = List(litRule, upperRule, lit2Rule) 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 0263f7842..d2686c826 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,11 +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.fs.HdfsUtils -import za.co.absa.enceladus.utils.testUtils.{LoggerTestBase, SparkTestBase} +import za.co.absa.enceladus.utils.testUtils.{HadoopFsTestBase, LoggerTestBase, SparkTestBase} - -trait TestRuleBehaviors extends AnyFunSuite with SparkTestBase with LoggerTestBase { +trait TestRuleBehaviors extends AnyFunSuite with SparkTestBase with LoggerTestBase with HadoopFsTestBase { def conformanceRuleShouldMatchExpected(inputDf: DataFrame, inputDataset: Dataset, expectedJSON: String) { implicit val dao: MenasDAO = mock(classOf[MenasDAO]) @@ -45,8 +43,6 @@ trait TestRuleBehaviors extends AnyFunSuite with SparkTestBase with LoggerTestB .setCatalystWorkaroundEnabled(isCatalystWorkaroundEnabled) .setControlFrameworkEnabled(enableCF) - implicit val fsUtils: HdfsUtils = new HdfsUtils(spark.sparkContext.hadoopConfiguration) - val conformed = DynamicInterpreter().interpret(inputDataset, inputDf) val conformedJSON = conformed.orderBy($"id").toJSON.collect().mkString("\n") 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 44067bd93..dd95ee693 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,8 +26,7 @@ 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.fs.HdfsUtils -import za.co.absa.enceladus.utils.testUtils.SparkTestBase +import za.co.absa.enceladus.utils.testUtils.{HadoopFsTestBase, SparkTestBase} case class MyCustomRule( order: Int, @@ -57,7 +56,7 @@ case class MyCustomRuleInterpreter(rule: MyCustomRule) extends RuleInterpreter { case class Mine(id: Int) case class MineConfd(id: Int, myOutputCol: Double, errCol: Seq[ErrorMessage]) -class CustomRuleSuite extends AnyFunSuite with SparkTestBase { +class CustomRuleSuite extends AnyFunSuite with SparkTestBase with HadoopFsTestBase { import spark.implicits._ // we may WANT to enable control framework & spline here @@ -90,8 +89,6 @@ class CustomRuleSuite extends AnyFunSuite with SparkTestBase { .setCatalystWorkaroundEnabled(isCatalystWorkaroundEnabled) .setControlFrameworkEnabled(enableCF) - implicit val fsUtils: HdfsUtils = new HdfsUtils(spark.sparkContext.hadoopConfiguration) - val actualDf: DataFrame = DynamicInterpreter().interpret(conformanceDef, inputData) val actual: Seq[MineConfd] = actualDf.as[MineConfd].collect().toSeq diff --git a/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/rules/testcasefactories/NestedTestCaseFactory.scala b/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/rules/testcasefactories/NestedTestCaseFactory.scala index 9fead4962..9720bf385 100644 --- a/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/rules/testcasefactories/NestedTestCaseFactory.scala +++ b/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/rules/testcasefactories/NestedTestCaseFactory.scala @@ -25,7 +25,8 @@ import za.co.absa.enceladus.dao.MenasDAO import za.co.absa.enceladus.model.conformanceRule.{ConformanceRule, MappingConformanceRule} import za.co.absa.enceladus.model.test.factories.{DatasetFactory, MappingTableFactory} import za.co.absa.enceladus.model.{Dataset, MappingTable} -import za.co.absa.enceladus.utils.fs.{HdfsUtils, LocalFsUtils} +import za.co.absa.enceladus.utils.fs.{HadoopFsUtils, LocalFsUtils} +import za.co.absa.enceladus.utils.testUtils.HadoopFsTestBase /** @@ -209,12 +210,10 @@ object NestedTestCaseFactory { )) } -class NestedTestCaseFactory(implicit spark: SparkSession) { +class NestedTestCaseFactory(implicit val spark: SparkSession) extends HadoopFsTestBase { import NestedTestCaseFactory._ - private val fs = FileSystem.get(spark.sparkContext.hadoopConfiguration) - private val fsUtils = new HdfsUtils(spark.sparkContext.hadoopConfiguration) private val tempDir = LocalFsUtils.getLocalTemporaryDirectory("test_case_factory") /** diff --git a/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/rules/testcasefactories/SimpleTestCaseFactory.scala b/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/rules/testcasefactories/SimpleTestCaseFactory.scala index a07cd9e71..447b8a9a3 100644 --- a/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/rules/testcasefactories/SimpleTestCaseFactory.scala +++ b/spark-jobs/src/test/scala/za/co/absa/enceladus/conformance/interpreter/rules/testcasefactories/SimpleTestCaseFactory.scala @@ -25,7 +25,8 @@ import za.co.absa.enceladus.dao.MenasDAO import za.co.absa.enceladus.model.conformanceRule.{ConformanceRule, MappingConformanceRule} import za.co.absa.enceladus.model.test.factories.{DatasetFactory, MappingTableFactory} import za.co.absa.enceladus.model.{Dataset, DefaultValue, MappingTable} -import za.co.absa.enceladus.utils.fs.{HdfsUtils, LocalFsUtils} +import za.co.absa.enceladus.utils.fs.{HadoopFsUtils, LocalFsUtils} +import za.co.absa.enceladus.utils.testUtils.HadoopFsTestBase object SimpleTestCaseFactory { @@ -118,13 +119,11 @@ object SimpleTestCaseFactory { * * Users of this factory can specify which conformance rules to include to the dataset definition to be created. */ -class SimpleTestCaseFactory(implicit spark: SparkSession) { +class SimpleTestCaseFactory(implicit val spark: SparkSession) extends HadoopFsTestBase { import SimpleTestCaseFactory._ import spark.implicits._ - private val fs = FileSystem.get(spark.sparkContext.hadoopConfiguration) - private val fsUtils = new HdfsUtils(spark.sparkContext.hadoopConfiguration) private val tempDir = LocalFsUtils.getLocalTemporaryDirectory("test_case_factory") /** diff --git a/utils/src/main/scala/za/co/absa/enceladus/utils/fs/DistributedFsUtils.scala b/utils/src/main/scala/za/co/absa/enceladus/utils/fs/DistributedFsUtils.scala index 862491650..42c80ece5 100644 --- a/utils/src/main/scala/za/co/absa/enceladus/utils/fs/DistributedFsUtils.scala +++ b/utils/src/main/scala/za/co/absa/enceladus/utils/fs/DistributedFsUtils.scala @@ -15,10 +15,6 @@ package za.co.absa.enceladus.utils.fs -/** - * A set of functions to help with the date partitioning and version control - */ - trait DistributedFsUtils { /** diff --git a/utils/src/main/scala/za/co/absa/enceladus/utils/fs/FileSystemUtils.scala b/utils/src/main/scala/za/co/absa/enceladus/utils/fs/FileSystemUtils.scala new file mode 100644 index 000000000..52ae347ce --- /dev/null +++ b/utils/src/main/scala/za/co/absa/enceladus/utils/fs/FileSystemUtils.scala @@ -0,0 +1,69 @@ +/* + * 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.fs + +import java.net.URI + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.FileSystem +import org.slf4j.{Logger, LoggerFactory} +import za.co.absa.atum.utils.S3Utils.StringS3LocationExt + +import scala.collection.concurrent.TrieMap + +object FileSystemUtils { + + val log: Logger = LoggerFactory.getLogger(this.getClass) + + /** + * Will yeild a [[FileSystem]] for path. If path prefix suggest S3, S3 FS is returned, HDFS otherwise. + * @param path + * @param hadoopConf + * @return + */ + def getFileSystemFromPath(path: String)(implicit hadoopConf: Configuration): FileSystem = { + path.toS3Location match { + + case Some(s3Location) => // s3 over hadoop fs api + val s3BucketUri: String = s"s3://${s3Location.bucketName}" // s3:// + val s3uri: URI = new URI(s3BucketUri) + FileSystem.get(s3uri, hadoopConf) + + case None => + FileSystem.get(hadoopConf) // HDFS + } + } + + private[fs] val fsUtilsCache = TrieMap[FileSystem, HadoopFsUtils]() + + implicit class FileSystemExt(fs: FileSystem) { + /** + * Given the FileSystem object `fs` that this method is called on, + * the appropriate HadoopFsUtils is either newly created or returned form cache. + * @return + */ + def toFsUtils: HadoopFsUtils = { + fsUtilsCache.getOrElseUpdate(fs, { + log.info(s"reusing cached fsUtils for FS ${fs.getUri} / ${fs.toString}") + new HadoopFsUtils()(fs)} + + ) + } + } + +} + diff --git a/utils/src/main/scala/za/co/absa/enceladus/utils/fs/HdfsUtils.scala b/utils/src/main/scala/za/co/absa/enceladus/utils/fs/HadoopFsUtils.scala similarity index 96% rename from utils/src/main/scala/za/co/absa/enceladus/utils/fs/HdfsUtils.scala rename to utils/src/main/scala/za/co/absa/enceladus/utils/fs/HadoopFsUtils.scala index dad506d11..8b6a55899 100644 --- a/utils/src/main/scala/za/co/absa/enceladus/utils/fs/HdfsUtils.scala +++ b/utils/src/main/scala/za/co/absa/enceladus/utils/fs/HadoopFsUtils.scala @@ -17,11 +17,8 @@ package za.co.absa.enceladus.utils.fs import java.io.{File, FileNotFoundException} import java.net.ConnectException -import java.nio.charset.StandardCharsets -import java.nio.file.{Files, Paths} import org.apache.commons.io.FileUtils -import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.log4j.LogManager @@ -31,10 +28,10 @@ import scala.util.Try * A set of functions to help with the date partitioning and version control */ -class HdfsUtils(conf: Configuration) extends DistributedFsUtils { +class HadoopFsUtils()(implicit fs: FileSystem) extends DistributedFsUtils { + + private val log = LogManager.getLogger("enceladus.utils.fs.HadoopFsUtils") - private val log = LogManager.getLogger("enceladus.utils.fs.HdfsUtils") - private val fs = FileSystem.get(conf) /** * Split HDFS path URI by separating scheme+server and path part * Example: diff --git a/utils/src/main/scala/za/co/absa/enceladus/utils/fs/S3FsUtils.scala b/utils/src/main/scala/za/co/absa/enceladus/utils/fs/S3SdkFsUtils.scala similarity index 95% rename from utils/src/main/scala/za/co/absa/enceladus/utils/fs/S3FsUtils.scala rename to utils/src/main/scala/za/co/absa/enceladus/utils/fs/S3SdkFsUtils.scala index f41c9e980..ab16a5bfd 100644 --- a/utils/src/main/scala/za/co/absa/enceladus/utils/fs/S3FsUtils.scala +++ b/utils/src/main/scala/za/co/absa/enceladus/utils/fs/S3SdkFsUtils.scala @@ -29,7 +29,7 @@ import scala.collection.JavaConverters._ import scala.util.{Failure, Success, Try} // kmsSettings: S3KmsSettings in not currently used, but would be necessary if any SDK calls needed to put data on S3 -case class S3FsUtils(region: Region, kmsSettings: S3KmsSettings)(implicit credentialsProvider: AwsCredentialsProvider) +case class S3SdkFsUtils(region: Region, kmsSettings: S3KmsSettings)(implicit credentialsProvider: AwsCredentialsProvider) extends DistributedFsUtils { protected val log: Logger = LoggerFactory.getLogger(this.getClass) @@ -41,7 +41,7 @@ case class S3FsUtils(region: Region, kmsSettings: S3KmsSettings)(implicit creden * Check if a given path exists on the distributed Fs */ override def exists(distPath: String): Boolean = { - val location = distPath.toS3Location(region) + val location = distPath.toS3LocationOrFail.withRegion(region) val headRequest = HeadObjectRequest .builder().bucket(location.bucketName).key(location.path) @@ -60,7 +60,7 @@ case class S3FsUtils(region: Region, kmsSettings: S3KmsSettings)(implicit creden } override def read(distPath: String): String = { - val location = distPath.toS3Location(region) + val location = distPath.toS3LocationOrFail.withRegion(region) val getRequest = GetObjectRequest .builder().bucket(location.bucketName).key(location.path) @@ -79,7 +79,7 @@ case class S3FsUtils(region: Region, kmsSettings: S3KmsSettings)(implicit creden private[fs] def getDirectorySize(distPath: String, keyNameFilter: String => Boolean): Long = { // setup accumulation - val location = distPath.toS3Location(region) + val location = distPath.toS3LocationOrFail.withRegion(region) val initSize = 0L def accumulateSizeOp(previousTotalSize: Long, response: ListObjectsV2Response): Long = { @@ -126,7 +126,7 @@ case class S3FsUtils(region: Region, kmsSettings: S3KmsSettings)(implicit creden */ override def isNonSplittable(distPath: String): Boolean = { // setup accumulation - val location = distPath.toS3Location(region) + val location = distPath.toS3LocationOrFail.withRegion(region) val initFoundValue = false // we want to break out of the recursion if a non-splittable is found, because it cannot ever be unfound. val breakOutCase = Some(true) @@ -147,7 +147,7 @@ case class S3FsUtils(region: Region, kmsSettings: S3KmsSettings)(implicit creden override def deleteDirectoryRecursively(distPath: String): Unit = { // setup accumulation - val location = distPath.toS3Location(region) + val location = distPath.toS3LocationOrFail.withRegion(region) def accumulateSizeOp(acc: Unit, response: ListObjectsV2Response): Unit = { // side-effect, "accumulates" to unit val objects = response.contents().asScala @@ -184,7 +184,7 @@ case class S3FsUtils(region: Region, kmsSettings: S3KmsSettings)(implicit creden override def getLatestVersion(publishPath: String, reportDate: String): Int = { // setup accumulation - val location = publishPath.toS3Location(region) + val location = publishPath.toS3LocationOrFail.withRegion(region) val initVersion = 0 // looking for $publishPath/enceladus_info_date=$reportDate\enceladus_info_version=$version diff --git a/utils/src/main/scala/za/co/absa/enceladus/utils/fs/example/S3FsUtilsTestJob.scala b/utils/src/main/scala/za/co/absa/enceladus/utils/fs/example/S3FsUtilsTestJob.scala index d5dfe8b90..bea6b4b17 100644 --- a/utils/src/main/scala/za/co/absa/enceladus/utils/fs/example/S3FsUtilsTestJob.scala +++ b/utils/src/main/scala/za/co/absa/enceladus/utils/fs/example/S3FsUtilsTestJob.scala @@ -19,7 +19,7 @@ import org.slf4j.LoggerFactory import software.amazon.awssdk.regions.Region import za.co.absa.atum.persistence.S3KmsSettings import za.co.absa.atum.utils.S3Utils -import za.co.absa.enceladus.utils.fs.S3FsUtils +import za.co.absa.enceladus.utils.fs.S3SdkFsUtils // open: remove or create a integtest like this instead. // implementation is directly suited to be runnable locally with a saml profile. @@ -36,7 +36,7 @@ object S3FsUtilsTestJob { val kmsKeyId = System.getenv("TOOLING_KMS_KEY_ID") // load from an environment property in order not to disclose it here log.info(s"kmsKeyId from env loaded = ${kmsKeyId.take(10)}...") - val s3utils = new S3FsUtils(Region.EU_WEST_1, S3KmsSettings(kmsKeyId)) { + val s3utils = new S3SdkFsUtils(Region.EU_WEST_1, S3KmsSettings(kmsKeyId)) { override val maxKeys = 5 // to test recursive listing/action } diff --git a/utils/src/main/scala/za/co/absa/enceladus/utils/performance/PerformanceMetricTools.scala b/utils/src/main/scala/za/co/absa/enceladus/utils/performance/PerformanceMetricTools.scala index 97aeed022..63e395950 100644 --- a/utils/src/main/scala/za/co/absa/enceladus/utils/performance/PerformanceMetricTools.scala +++ b/utils/src/main/scala/za/co/absa/enceladus/utils/performance/PerformanceMetricTools.scala @@ -20,7 +20,7 @@ import org.apache.spark.sql.functions.{col, size, sum} import org.slf4j.{Logger, LoggerFactory} import za.co.absa.atum.core.Atum import za.co.absa.enceladus.utils.error.ErrorMessage -import za.co.absa.enceladus.utils.fs.DistributedFsUtils +import za.co.absa.enceladus.utils.fs.{DistributedFsUtils, HadoopFsUtils} import za.co.absa.enceladus.utils.general.ProjectMetadataTools import za.co.absa.enceladus.utils.schema.SchemaUtils @@ -45,7 +45,7 @@ object PerformanceMetricTools { outputPath: String, loginUserName: String, cmdLineArgs: String) - (implicit spark: SparkSession, fsUtils: DistributedFsUtils): Unit = { + (implicit spark: SparkSession, inputFsUtils: HadoopFsUtils): Unit = { // Spark job configuration val sc = spark.sparkContext @@ -53,8 +53,8 @@ object PerformanceMetricTools { val numberOfExecutors = sc.getExecutorMemoryStatus.keys.size - 1 // Directory sizes and size ratio - val inputDirSize = fsUtils.getDirectorySize(inputPath) - val inputDataSize = fsUtils.getDirectorySizeNoHidden(inputPath) + val inputDirSize = inputFsUtils.getDirectorySize(inputPath) + val inputDataSize = inputFsUtils.getDirectorySizeNoHidden(inputPath) addSparkConfig(optionPrefix, "spark.driver.memory", "driver_memory") addSparkConfig(optionPrefix, "spark.driver.cores", "driver_cores") @@ -95,12 +95,12 @@ object PerformanceMetricTools { outputPath: String, loginUserName: String, cmdLineArgs: String - )(implicit fsUtils: DistributedFsUtils): Unit = { + )(inputFsUtils: HadoopFsUtils, outputFsUtils: HadoopFsUtils): Unit = { // Directory sizes and size ratio - val inputDirSize = fsUtils.getDirectorySize(inputPath) - val outputDirSize = fsUtils.getDirectorySize(outputPath) - val outputDataSize = fsUtils.getDirectorySizeNoHidden(outputPath) + val inputDirSize = inputFsUtils.getDirectorySize(inputPath) + val outputDirSize = outputFsUtils.getDirectorySize(outputPath) + val outputDataSize = outputFsUtils.getDirectorySizeNoHidden(outputPath) val (numRecordsFailed, numRecordsSuccessful, numOfErrors) = getNumberOfErrors(spark, outputPath) diff --git a/spark-jobs/src/main/scala/za/co/absa/enceladus/common/config/S3Config.scala b/utils/src/main/scala/za/co/absa/enceladus/utils/testUtils/HadoopFsTestBase.scala similarity index 59% rename from spark-jobs/src/main/scala/za/co/absa/enceladus/common/config/S3Config.scala rename to utils/src/main/scala/za/co/absa/enceladus/utils/testUtils/HadoopFsTestBase.scala index 9be6ea7f5..65d5c7c5b 100644 --- a/spark-jobs/src/main/scala/za/co/absa/enceladus/common/config/S3Config.scala +++ b/utils/src/main/scala/za/co/absa/enceladus/utils/testUtils/HadoopFsTestBase.scala @@ -13,8 +13,22 @@ * limitations under the License. */ -package za.co.absa.enceladus.common.config -import software.amazon.awssdk.regions.Region +package za.co.absa.enceladus.utils.testUtils + +import org.apache.hadoop.fs.FileSystem +import org.apache.spark.sql.SparkSession +import za.co.absa.enceladus.utils.fs.FileSystemUtils.FileSystemExt + + +trait HadoopFsTestBase extends HasSparkSession { + + implicit val fs = FileSystem.get(spark.sparkContext.hadoopConfiguration) + implicit val fsUtils = fs.toFsUtils +} + +trait HasSparkSession { + def spark: SparkSession +} + -case class S3Config(region: Region, kmsKeyId: String) 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 index b55379274..aceebe64e 100644 --- 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 @@ -16,7 +16,7 @@ package za.co.absa.enceladus.utils.testUtils import org.apache.log4j.{Level, Logger} -import org.apache.spark.sql. SparkSession +import org.apache.spark.sql.SparkSession import org.apache.spark.SparkConf import com.typesafe.config.ConfigFactory import org.apache.hadoop.conf.Configuration @@ -26,7 +26,7 @@ import java.io.File import za.co.absa.enceladus.utils.time.TimeZoneNormalizer import com.typesafe.config.Config -trait SparkTestBase { self => +trait SparkTestBase extends HasSparkSession { self => TimeZoneNormalizer.normalizeJVMTimeZone() val config: Config = ConfigFactory.load() 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 5f7b39e8c..87edb8e3e 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,13 +20,13 @@ 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.SparkTestBase +import za.co.absa.enceladus.utils.testUtils.{HadoopFsTestBase, SparkTestBase} /** * Unit tests for File system utils */ -class HdfsUtilsSpec extends AnyWordSpec with Matchers with SparkTestBase { - val hdfsUtils = new HdfsUtils(spark.sparkContext.hadoopConfiguration) +class HdfsUtilsSpec extends AnyWordSpec with Matchers with SparkTestBase with HadoopFsTestBase { + val hdfsUtils = fsUtils "splitUriPath" should { "split URI and path" in { diff --git a/utils/src/test/scala/za/co/absa/enceladus/utils/fs/S3FsUtilsSuite.scala b/utils/src/test/scala/za/co/absa/enceladus/utils/fs/S3SdkFsUtilsSuite.scala similarity index 99% rename from utils/src/test/scala/za/co/absa/enceladus/utils/fs/S3FsUtilsSuite.scala rename to utils/src/test/scala/za/co/absa/enceladus/utils/fs/S3SdkFsUtilsSuite.scala index 6d558c79d..b519cce93 100644 --- a/utils/src/test/scala/za/co/absa/enceladus/utils/fs/S3FsUtilsSuite.scala +++ b/utils/src/test/scala/za/co/absa/enceladus/utils/fs/S3SdkFsUtilsSuite.scala @@ -29,7 +29,7 @@ import za.co.absa.atum.persistence.S3KmsSettings import scala.collection.JavaConverters._ -class S3FsUtilsSuite extends AnyFlatSpec with IdiomaticMockito with Matchers { +class S3SdkFsUtilsSuite extends AnyFlatSpec with IdiomaticMockito with Matchers { val kmsSettigns = S3KmsSettings("testingKeyId123") val region = Region.EU_WEST_2 @@ -39,7 +39,7 @@ class S3FsUtilsSuite extends AnyFlatSpec with IdiomaticMockito with Matchers { // common fixture for all tests def fixture = new { val mockedS3Client = mock[S3Client] - val mockedS3FsUtils = new S3FsUtils(region, kmsSettigns) { + val mockedS3FsUtils = new S3SdkFsUtils(region, kmsSettigns) { override def getS3Client: S3Client = mockedS3Client override val maxKeys = 3 // to test recursion for listing From 5e7a503f7729cd0a531a7f8281cc92741820d219 Mon Sep 17 00:00:00 2001 From: Daniel Kavan Date: Mon, 26 Oct 2020 15:43:24 +0100 Subject: [PATCH 02/12] s3 sdk usage removed (pom, classes, tests) --- pom.xml | 18 +- .../S3DefaultCredentialsProvider.scala | 24 -- .../enceladus/common/CommonJobExecution.scala | 2 +- .../StandardizationExecution.scala | 3 +- .../enceladus/utils/fs/S3SdkFsUtils.scala | 271 ------------ .../utils/fs/example/S3FsUtilsTestJob.scala | 65 --- .../utils/fs/S3SdkFsUtilsSuite.scala | 394 ------------------ 7 files changed, 3 insertions(+), 774 deletions(-) delete mode 100644 spark-jobs/src/main/scala/za/co/absa/enceladus/S3DefaultCredentialsProvider.scala delete mode 100644 utils/src/main/scala/za/co/absa/enceladus/utils/fs/S3SdkFsUtils.scala delete mode 100644 utils/src/main/scala/za/co/absa/enceladus/utils/fs/example/S3FsUtilsTestJob.scala delete mode 100644 utils/src/test/scala/za/co/absa/enceladus/utils/fs/S3SdkFsUtilsSuite.scala diff --git a/pom.xml b/pom.xml index 3d1788a9a..7da051630 100644 --- a/pom.xml +++ b/pom.xml @@ -204,18 +204,6 @@ false - - - - software.amazon.awssdk - bom - ${aws.java.sdk.version} - pom - import - - - - org.scala-lang @@ -239,17 +227,13 @@ - - software.amazon.awssdk - s3 - org.apache.hadoop hadoop-aws ${hadoop.version} provided - + com.amazonaws aws-java-sdk-s3 diff --git a/spark-jobs/src/main/scala/za/co/absa/enceladus/S3DefaultCredentialsProvider.scala b/spark-jobs/src/main/scala/za/co/absa/enceladus/S3DefaultCredentialsProvider.scala deleted file mode 100644 index 3b209d32e..000000000 --- a/spark-jobs/src/main/scala/za/co/absa/enceladus/S3DefaultCredentialsProvider.scala +++ /dev/null @@ -1,24 +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 - -import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider - -trait S3DefaultCredentialsProvider { - - implicit val defaultCredentialsProvider = DefaultCredentialsProvider.create() - -} 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 d7096ed8e..582ba3c23 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 @@ -36,7 +36,7 @@ import za.co.absa.enceladus.dao.rest.MenasConnectionStringParser import za.co.absa.enceladus.model.Dataset import za.co.absa.enceladus.plugins.builtin.errorsender.params.ErrorSenderPluginParams import za.co.absa.enceladus.utils.config.{ConfigReader, SecureConfig} -import za.co.absa.enceladus.utils.fs.{DistributedFsUtils, FileSystemUtils, HadoopFsUtils, S3SdkFsUtils} +import za.co.absa.enceladus.utils.fs.{DistributedFsUtils, FileSystemUtils} import za.co.absa.enceladus.utils.general.ProjectMetadataTools import za.co.absa.enceladus.utils.modules.SourcePhase import za.co.absa.enceladus.utils.modules.SourcePhase.Standardization 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 82944c83a..5895de679 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 @@ -22,7 +22,6 @@ 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.S3DefaultCredentialsProvider import za.co.absa.enceladus.common.RecordIdGeneration.getRecordIdGenerationStrategyFromConfig import za.co.absa.enceladus.common.config.{FileSystems, JobConfigParser, PathConfig} import za.co.absa.enceladus.common.plugin.menas.MenasPlugin @@ -44,7 +43,7 @@ import org.apache.hadoop.fs.FileSystem import scala.util.control.NonFatal -trait StandardizationExecution extends CommonJobExecution with S3DefaultCredentialsProvider { +trait StandardizationExecution extends CommonJobExecution { private val sourceId = SourcePhase.Standardization protected def prepareStandardization[T](args: Array[String], diff --git a/utils/src/main/scala/za/co/absa/enceladus/utils/fs/S3SdkFsUtils.scala b/utils/src/main/scala/za/co/absa/enceladus/utils/fs/S3SdkFsUtils.scala deleted file mode 100644 index ab16a5bfd..000000000 --- a/utils/src/main/scala/za/co/absa/enceladus/utils/fs/S3SdkFsUtils.scala +++ /dev/null @@ -1,271 +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.fs - -import org.slf4j.{Logger, LoggerFactory} -import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider -import software.amazon.awssdk.regions.Region -import software.amazon.awssdk.services.s3.S3Client -import software.amazon.awssdk.services.s3.model.{S3Location => _, _} -import za.co.absa.atum.persistence.{S3KmsSettings, S3Location} -import za.co.absa.atum.utils.S3Utils -import za.co.absa.atum.utils.S3Utils.StringS3LocationExt - -import scala.annotation.tailrec -import scala.collection.JavaConverters._ -import scala.util.{Failure, Success, Try} - -// kmsSettings: S3KmsSettings in not currently used, but would be necessary if any SDK calls needed to put data on S3 -case class S3SdkFsUtils(region: Region, kmsSettings: S3KmsSettings)(implicit credentialsProvider: AwsCredentialsProvider) - extends DistributedFsUtils { - - protected val log: Logger = LoggerFactory.getLogger(this.getClass) - private[fs] val maxKeys = 1000 // overridable default - - val s3Client: S3Client = getS3Client - - /** - * Check if a given path exists on the distributed Fs - */ - override def exists(distPath: String): Boolean = { - val location = distPath.toS3LocationOrFail.withRegion(region) - - val headRequest = HeadObjectRequest - .builder().bucket(location.bucketName).key(location.path) - .build() - - // there seems to be no doesObjectExist method as of current version https://github.com/aws/aws-sdk-java-v2/issues/392 - Try { - s3Client.headObject(headRequest) - } match { - case Success(_) => - true - case Failure(_: NoSuchKeyException) => - false - case Failure(e) => throw e - } - } - - override def read(distPath: String): String = { - val location = distPath.toS3LocationOrFail.withRegion(region) - - val getRequest = GetObjectRequest - .builder().bucket(location.bucketName).key(location.path) - .build() - - val content = s3Client.getObjectAsBytes(getRequest).asUtf8String() - - content - } - - override def getDirectorySize(distPath: String): Long = getDirectorySize(distPath, _ => true) - - /** - * Returns distributed directory size in bytes - */ - private[fs] def getDirectorySize(distPath: String, keyNameFilter: String => Boolean): Long = { - - // setup accumulation - val location = distPath.toS3LocationOrFail.withRegion(region) - val initSize = 0L - - def accumulateSizeOp(previousTotalSize: Long, response: ListObjectsV2Response): Long = { - val objects = response.contents().asScala - val totalSize = objects - .filter(obj => keyNameFilter(obj.key)) - .foldLeft(0L) { (currentSize: Long, nextObject: S3Object) => currentSize + nextObject.size } - - previousTotalSize + totalSize - } - - listAndAccumulateRecursively(location, accumulateSizeOp, initSize) - } - - /** - * Hidden files = starting with `_` or `.` This method will return true for hidden keys. - * - * @param key path on s3 - * @return e.g. `/path/to/.hidden` => true, `/path/to/non-hidden` => false - */ - private[fs] def isKeyHidden(key: String): Boolean = { - val fn = key.split('/').last - - (fn.startsWith("_")) || (fn.startsWith(".")) - } - - /** - * Returns distributed directory size in bytes, skipping hidden files and directories (starting from '_' or '.'). - * - * @param distPath A path to a directory or a file. - * @return Directory size in bytes - */ - override def getDirectorySizeNoHidden(distPath: String): Long = getDirectorySize(distPath, key => !isKeyHidden(key)) - - - private[fs] def isKeyNonSplittable(key: String): Boolean = { - val fn = key.split('/').last - - DistributedFsUtils.nonSplittableExtensions.exists(fn.endsWith) - } - - /** - * Checks if the distributed-FS path contains non-splittable files - */ - override def isNonSplittable(distPath: String): Boolean = { - // setup accumulation - val location = distPath.toS3LocationOrFail.withRegion(region) - val initFoundValue = false - // we want to break out of the recursion if a non-splittable is found, because it cannot ever be unfound. - val breakOutCase = Some(true) - - def accumulateFoundOp(previouslyFound: Boolean, response: ListObjectsV2Response): Boolean = { - val objects = response.contents().asScala - val nonSplittableFound = objects.exists(obj => isKeyNonSplittable(obj.key)) - - previouslyFound || nonSplittableFound // true if ever found - } - - listAndAccumulateRecursively(location, accumulateFoundOp, initFoundValue, breakOutCase) - } - - /** - * Deletes a distributed-FS directory and all its contents recursively - */ - override def deleteDirectoryRecursively(distPath: String): Unit = { - - // setup accumulation - val location = distPath.toS3LocationOrFail.withRegion(region) - - def accumulateSizeOp(acc: Unit, response: ListObjectsV2Response): Unit = { // side-effect, "accumulates" to unit - val objects = response.contents().asScala - if (objects.nonEmpty) { - deleteKeys(location.bucketName, objects.map(_.key)) - } - } - - listAndAccumulateRecursively(location, accumulateSizeOp, ()) - } - - private[fs] def deleteKeys(bucketName: String, keys: Seq[String]): Unit = { - require(keys.nonEmpty) - - val objIds = keys.map(k => ObjectIdentifier.builder().key(k).build()) - val request: DeleteObjectsRequest = DeleteObjectsRequest.builder().bucket(bucketName) - .delete(Delete.builder().objects(objIds.asJava).build()) - .build() - - val delResp: DeleteObjectsResponse = s3Client.deleteObjects(request) - - if (delResp.errors().size() > 0) { - log.warn(s"Errors while deleting (${delResp.errors.size}):\n ${delResp.errors.asScala.map(_.message()).mkString("\n")}") - } - } - - /** - * Finds the latest version given a publish folder on distributed-FS - * - * @param publishPath The distributed-FS path to the publish folder containing versions - * @param reportDate The string representation of the report date used to infer the latest version - * @return the latest version or 0 in case no versions exist - */ - override def getLatestVersion(publishPath: String, reportDate: String): Int = { - - // setup accumulation - val location = publishPath.toS3LocationOrFail.withRegion(region) - val initVersion = 0 - - // looking for $publishPath/enceladus_info_date=$reportDate\enceladus_info_version=$version - val prefix = s"${location.path}/enceladus_info_date=$reportDate/enceladus_info_version=" - val prefixedLocation = location.copy(path = prefix) - - def accumulateSizeOp(previousMaxVersion: Int, response: ListObjectsV2Response): Int = { - val objects = response.contents().asScala - - val existingVersions = objects - .map(_.key) - .flatMap { key => - assert(key.startsWith(prefix), s"Retrieved keys should start with $prefix, but precondition fails for $key") - val noPrefix = key.stripPrefix(prefix) - Try { - noPrefix.takeWhile(_.isDigit).toInt // may not hold valid int >= 1 - } match { - case Success(version) if version >= 1 => Some(version) - case _ => None - } - } - .toSet - - if (existingVersions.isEmpty) { - previousMaxVersion - } else { - Math.max(previousMaxVersion, existingVersions.max) - } - } - - listAndAccumulateRecursively(prefixedLocation, accumulateSizeOp, initVersion) - } - - private[fs] def getS3Client: S3Client = S3Utils.getS3Client(region, credentialsProvider) - - /** - * General method to list and accumulate the objects info. Note, that the method strives to be memory-efficient - - * i.e. accumulate the current batch first and then load the next batch (instead of the naive "load all first, process later" - * - * @param location s3location - bucket & path are used - * @param accumulateOp operation to accumulate - * @param initialAccValue (initial/carry-over) accumulator value - * @param breakOut allows to break the recursion prematurely when the defined value equals the currently accumulated value. - * Default: None = no break out - * @tparam T accumulator value type - * @return accumulated value - */ - private def listAndAccumulateRecursively[T](location: S3Location, - accumulateOp: (T, ListObjectsV2Response) => T, - initialAccValue: T, - breakOut: Option[T] = None): T = { - - log.debug(s"listAndAccumulateRecursively($location, $accumulateOp, $initialAccValue, $breakOut)") - - @tailrec - def listAndAccumulateRecursivelyAcc(contToken: Option[String], acc: T): T = { - log.debug(s"listAndAccumulateRecursivelyAcc($contToken, $acc)") - - val listObjectsBuilder = ListObjectsV2Request.builder - .bucket(location.bucketName) - .prefix(location.path) - .maxKeys(maxKeys) - val listObjectsRequest = contToken.fold(listObjectsBuilder.build)(listObjectsBuilder.continuationToken(_).build) - - val response: ListObjectsV2Response = s3Client.listObjectsV2(listObjectsRequest) - val totalAccumulated: T = accumulateOp(acc, response) // result of previous with the currently accumulated together - - // the caller is able define a short-circuiting condition - at which no more processing is needed, hence we "break out" here - if (breakOut.contains(totalAccumulated)) { - log.debug(s"Breakout at totalAccumulated value $totalAccumulated") - totalAccumulated - } else if (!response.isTruncated) { - log.debug(s"Final recursion level reached.") - totalAccumulated - } else { - // need to recurse & accumulate deeper - listAndAccumulateRecursivelyAcc(Some(response.nextContinuationToken), totalAccumulated) - } - } - - // run the recursive call - listAndAccumulateRecursivelyAcc(contToken = None, acc = initialAccValue) - } -} diff --git a/utils/src/main/scala/za/co/absa/enceladus/utils/fs/example/S3FsUtilsTestJob.scala b/utils/src/main/scala/za/co/absa/enceladus/utils/fs/example/S3FsUtilsTestJob.scala deleted file mode 100644 index bea6b4b17..000000000 --- a/utils/src/main/scala/za/co/absa/enceladus/utils/fs/example/S3FsUtilsTestJob.scala +++ /dev/null @@ -1,65 +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.fs.example - -import org.slf4j.LoggerFactory -import software.amazon.awssdk.regions.Region -import za.co.absa.atum.persistence.S3KmsSettings -import za.co.absa.atum.utils.S3Utils -import za.co.absa.enceladus.utils.fs.S3SdkFsUtils - -// open: remove or create a integtest like this instead. -// implementation is directly suited to be runnable locally with a saml profile. -object S3FsUtilsTestJob { - - private val log = LoggerFactory.getLogger(this.getClass) - private val bucketName = "putYourBucketBucketNameHere" - - def main(args: Array[String]): Unit = { - val basePath = s"s3://$bucketName/exampleS3Path" - - // This sample example relies on local credentials profile named "saml" with access to the s3 location defined below - implicit val samlCredentialsProvider = S3Utils.getLocalProfileCredentialsProvider("saml") - val kmsKeyId = System.getenv("TOOLING_KMS_KEY_ID") // load from an environment property in order not to disclose it here - log.info(s"kmsKeyId from env loaded = ${kmsKeyId.take(10)}...") - - val s3utils = new S3SdkFsUtils(Region.EU_WEST_1, S3KmsSettings(kmsKeyId)) { - override val maxKeys = 5 // to test recursive listing/action - } - - log.info(s"dir size of $basePath is:" + s3utils.getDirectorySize(basePath)) - log.info(s"dir size (no hidden) of $basePath is:" + s3utils.getDirectorySizeNoHidden(basePath)) - - log.info(s"should exist:" + s3utils.exists(s"$basePath/1/2019/11/27/1/_INFO")) - log.info(s"should not exist:" + s3utils.exists(s"$basePath/1/2019/11/27/1/_INFObogus")) - - log.info("found version (1): " - + s3utils.getLatestVersion(s"s3://$bucketName/superhero/publish", "2020-08-06")) - - log.info("found no version (0): " - + s3utils.getLatestVersion(s"s3://$bucketName/aaa", "2020-08-06")) - - log.info(s"reading file content:" + s3utils.read(s"$basePath/1/2019/11/27/1/_INFO").take(50)) - - log.info(s"should find no gz-s:" + s3utils.isNonSplittable(s"s3://$bucketName/gz-list/nogz")) - log.info(s"should find some gz-s (and breakOut):" + - s3utils.isNonSplittable(s"s3://$bucketName/gz-list/somegz")) - - val deletePath = s"s3://$bucketName/delete" - log.info(s"deleting $deletePath: " + s3utils.deleteDirectoryRecursively(deletePath)) - } - -} diff --git a/utils/src/test/scala/za/co/absa/enceladus/utils/fs/S3SdkFsUtilsSuite.scala b/utils/src/test/scala/za/co/absa/enceladus/utils/fs/S3SdkFsUtilsSuite.scala deleted file mode 100644 index b519cce93..000000000 --- a/utils/src/test/scala/za/co/absa/enceladus/utils/fs/S3SdkFsUtilsSuite.scala +++ /dev/null @@ -1,394 +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.fs - -import org.mockito.captor.{ArgCaptor, Captor} -import org.mockito.scalatest.IdiomaticMockito -import org.mockito.{ArgumentMatchers, Mockito} -import org.scalatest.flatspec.AnyFlatSpec -import org.scalatest.matchers.should.Matchers -import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider -import software.amazon.awssdk.core.ResponseBytes -import software.amazon.awssdk.regions.Region -import software.amazon.awssdk.services.s3.S3Client -import software.amazon.awssdk.services.s3.model._ -import za.co.absa.atum.persistence.S3KmsSettings - -import scala.collection.JavaConverters._ - -class S3SdkFsUtilsSuite extends AnyFlatSpec with IdiomaticMockito with Matchers { - - val kmsSettigns = S3KmsSettings("testingKeyId123") - val region = Region.EU_WEST_2 - - implicit val credentialsProvider = DefaultCredentialsProvider.create() - - // common fixture for all tests - def fixture = new { - val mockedS3Client = mock[S3Client] - val mockedS3FsUtils = new S3SdkFsUtils(region, kmsSettigns) { - override def getS3Client: S3Client = mockedS3Client - - override val maxKeys = 3 // to test recursion for listing - } - } - - "S3FsUtilsTest" should "detect exiting file" in { - val f = fixture - val path = "s3://bucket1/path/to/existing.file" - - // mock S3 response for exist - val mockedResponse: HeadObjectResponse = mock[HeadObjectResponse] - Mockito.when(f.mockedS3Client.headObject(any[HeadObjectRequest])).thenReturn(mockedResponse) - - val existResult = f.mockedS3FsUtils.exists(path) - - // verify request content - val requestCaptor: Captor[HeadObjectRequest] = ArgCaptor[HeadObjectRequest] - Mockito.verify(f.mockedS3Client).headObject(requestCaptor.capture) - val capturedGetRequest = requestCaptor.value - - capturedGetRequest.bucket shouldBe "bucket1" - capturedGetRequest.key shouldBe "path/to/existing.file" - - // verify returned value - existResult shouldBe true - } - - it should "detect non-exiting file" in { - val f = fixture - val path = "s3://bucket1b/path/to/non-existing.file" - - // mock S3 response for exist - Mockito.when(f.mockedS3Client.headObject(any[HeadObjectRequest])) - .thenThrow(NoSuchKeyException.builder.message("the file does not exist!").build()) - - val existResult = f.mockedS3FsUtils.exists(path) - - // verify request content - val requestCaptor: Captor[HeadObjectRequest] = ArgCaptor[HeadObjectRequest] - Mockito.verify(f.mockedS3Client).headObject(requestCaptor.capture) - val capturedGetRequest = requestCaptor.value - - capturedGetRequest.bucket shouldBe "bucket1b" - capturedGetRequest.key shouldBe "path/to/non-existing.file" - - // verify returned value - existResult shouldBe false - } - - it should "read data from S3 path" in { - val f = fixture - val path = "s3://bucket2/path/to/read.file" - val mockedFileContent = "This is the file content on S3" - - val mockedResponseWithContent: ResponseBytes[GetObjectResponse] = mock[ResponseBytes[GetObjectResponse]] - - // mock S3 response - Mockito.when(f.mockedS3Client.getObjectAsBytes(ArgumentMatchers.any[GetObjectRequest])).thenReturn(mockedResponseWithContent) - Mockito.when(mockedResponseWithContent.asUtf8String()).thenReturn(mockedFileContent) - - val readingResult = f.mockedS3FsUtils.read(path) - - // verify request content - val requestCaptor: Captor[GetObjectRequest] = ArgCaptor[GetObjectRequest] - Mockito.verify(f.mockedS3Client).getObjectAsBytes(requestCaptor.capture) - val capturedGetRequest = requestCaptor.value - - capturedGetRequest.bucket shouldBe "bucket2" - capturedGetRequest.key shouldBe "path/to/read.file" - - // verify returned value - readingResult shouldBe mockedFileContent - } - - private case class MockedObjectDef(path: String, size: Long = 0L) { - def toObject: S3Object = S3Object.builder().key(path).size(size).build - } - - private val mockedObjects1 = Seq( - MockedObjectDef("/dir/to/size/.hidden_file1.abc", 1L), - MockedObjectDef("/dir/to/size/_hidden.file2.abc", 2L), - MockedObjectDef("/dir/to/size/regular-file3.abc", 4L) - ).map(_.toObject) - - private val mockedObjects2 = Seq( - MockedObjectDef("/dir/to/size/.hidden_file10.abc", 10L), - MockedObjectDef("/dir/to/size/_hidden.file20.abc", 20L), - MockedObjectDef("/dir/to/size/regular-file30.gz", 40L) - ).map(_.toObject) - - it should "get dir size - simple (no filtering, no pagination)" in { - val f = fixture - val path = "s3://bucket3/dir/to/size" - - val mockedListResponse: ListObjectsV2Response = ListObjectsV2Response.builder() - .isTruncated(false) - .contents(mockedObjects1.asJava) - .build - - // mock S3 response - Mockito.when(f.mockedS3Client.listObjectsV2(ArgumentMatchers.any[ListObjectsV2Request])).thenReturn(mockedListResponse) - val dirSizeResult = f.mockedS3FsUtils.getDirectorySize(path) - - // verify request content - val requestCaptor: Captor[ListObjectsV2Request] = ArgCaptor[ListObjectsV2Request] - Mockito.verify(f.mockedS3Client).listObjectsV2(requestCaptor.capture) - val capturedListRequest = requestCaptor.value - - capturedListRequest.bucket shouldBe "bucket3" - capturedListRequest.prefix shouldBe "dir/to/size" - capturedListRequest.continuationToken shouldBe null - - // verify returned value - dirSizeResult shouldBe 7L - } - - { - val (f1, f2) = (fixture, fixture) - Seq( - (f1, "all files", (f1.mockedS3FsUtils.getDirectorySize(_)): String => Long, 77L), - (f2, "only non-hidden", (f2.mockedS3FsUtils.getDirectorySizeNoHidden(_)): String => Long, 44L) - ) - }.foreach { case (f, testCaseName, getSizeOp, expectedSize) => - - it should s"get dir size for $testCaseName - with pagination listing" in { - val path = "s3://bucket3b/dir/to/size" - - val mockedListResponses: Seq[ListObjectsV2Response] = Seq( - ListObjectsV2Response.builder().isTruncated(true).nextContinuationToken("token1") - .contents(mockedObjects1.asJava).build, - ListObjectsV2Response.builder().isTruncated(false) - .contents(mockedObjects2.asJava).build - ) - - // mock S3 responses - Mockito.when(f.mockedS3Client.listObjectsV2(ArgumentMatchers.any[ListObjectsV2Request])) - .thenReturn(mockedListResponses(0)) - .thenReturn(mockedListResponses(1)) - val dirSizeResult = getSizeOp(path) - - // verify request content - val requestCaptor: Captor[ListObjectsV2Request] = ArgCaptor[ListObjectsV2Request] - Mockito.verify(f.mockedS3Client, Mockito.times(2)).listObjectsV2(requestCaptor.capture) - val capturedListRequests = requestCaptor.values - - // bucket & path should always be the same - capturedListRequests.foreach(_.bucket shouldBe "bucket3b") - capturedListRequests.foreach(_.prefix shouldBe "dir/to/size") - - // when truncated, the continuationToken was passed along to the next request to resume correctly - capturedListRequests.map(_.continuationToken) shouldBe List(null, "token1") - - // verify returned value - dirSizeResult shouldBe expectedSize - } - } - - Seq( - ("non-splittable", mockedObjects2, true), - ("splittable", mockedObjects1, false) - ).foreach { case (testCaseName, mockedObjects, expectedNonSplitability) => - it should s"find the file list be $testCaseName (simple case, no pagination)" in { - val f = fixture - val path = "s3://bucket4/dir/to/split" - - val mockedListResponse: ListObjectsV2Response = ListObjectsV2Response.builder() - .isTruncated(false) - .contents(mockedObjects.asJava) - .build - - // mock S3 response - Mockito.when(f.mockedS3Client.listObjectsV2(ArgumentMatchers.any[ListObjectsV2Request])).thenReturn(mockedListResponse) - val isNonSplittableResult = f.mockedS3FsUtils.isNonSplittable(path) - - // verify request content - val requestCaptor: Captor[ListObjectsV2Request] = ArgCaptor[ListObjectsV2Request] - Mockito.verify(f.mockedS3Client).listObjectsV2(requestCaptor.capture) - val capturedListRequest = requestCaptor.value - - capturedListRequest.bucket shouldBe "bucket4" - capturedListRequest.prefix shouldBe "dir/to/split" - capturedListRequest.continuationToken shouldBe null - - // verify returned value - isNonSplittableResult shouldBe expectedNonSplitability - } - } - - it should s"find the file list be non-splittable with breakOut" in { - val f = fixture - val path = "s3://bucket4b/dir/to/split" - - val mockedListResponses: Seq[ListObjectsV2Response] = Seq( - ListObjectsV2Response.builder().isTruncated(true).nextContinuationToken("token1") - .contents(mockedObjects1.asJava).build, - ListObjectsV2Response.builder().isTruncated(true).nextContinuationToken("token2") - .contents(mockedObjects2.asJava).build - ) - - // mock S3 responses: pretend that there could be a third response with objects, but it should not be reached - // because non-splittable file was already found and the breakOut should prevent from further processing - Mockito.when(f.mockedS3Client.listObjectsV2(ArgumentMatchers.any[ListObjectsV2Request])) - .thenReturn(mockedListResponses(0)) - .thenReturn(mockedListResponses(1)) - .thenThrow(new IllegalStateException("Unwanted state - breakOut for non-splitability does not work")) - val isNonSplittableResult = f.mockedS3FsUtils.isNonSplittable(path) - - // verify request content - val requestCaptor: Captor[ListObjectsV2Request] = ArgCaptor[ListObjectsV2Request] - Mockito.verify(f.mockedS3Client, Mockito.times(2)).listObjectsV2(requestCaptor.capture) - val capturedListRequests = requestCaptor.values - - // bucket & path should always be the same - capturedListRequests.foreach(_.bucket shouldBe "bucket4b") - capturedListRequests.foreach(_.prefix shouldBe "dir/to/split") - - // when truncated, the continuationToken was passed along to the next request to resume correctly - capturedListRequests.map(_.continuationToken) shouldBe List(null, "token1") - - // verify returned value - isNonSplittableResult shouldBe true - } - - it should s"delete files - with pagination listing" in { - val f = fixture - val path = "s3://bucket5/dir/to/delete" - - // mock S3 list responses - val mockedListResponses: Seq[ListObjectsV2Response] = Seq( - ListObjectsV2Response.builder().isTruncated(true).nextContinuationToken("token1") - .contents(mockedObjects1.asJava).build, - ListObjectsV2Response.builder().isTruncated(false) - .contents(mockedObjects2.asJava).build - ) - Mockito.when(f.mockedS3Client.listObjectsV2(ArgumentMatchers.any[ListObjectsV2Request])) - .thenReturn(mockedListResponses(0)) - .thenReturn(mockedListResponses(1)) - - // mock delete responses - val mockedDeleteReponse = mock[DeleteObjectsResponse] - Mockito.when(f.mockedS3Client.deleteObjects(ArgumentMatchers.any[DeleteObjectsRequest])) - .thenReturn(mockedDeleteReponse) - Mockito.when(mockedDeleteReponse.errors).thenReturn(List.empty[S3Error].asJava) - - f.mockedS3FsUtils.deleteDirectoryRecursively(path) - - // verify list request contents - val listRequestCaptor: Captor[ListObjectsV2Request] = ArgCaptor[ListObjectsV2Request] - Mockito.verify(f.mockedS3Client, Mockito.times(2)).listObjectsV2(listRequestCaptor.capture) - val capturedListRequests = listRequestCaptor.values - - // bucket & path should always be the same - capturedListRequests.foreach(_.bucket shouldBe "bucket5") - capturedListRequests.foreach(_.prefix shouldBe "dir/to/delete") - - // when truncated, the continuationToken was passed along to the next request to resume correctly - capturedListRequests.map(_.continuationToken) shouldBe List(null, "token1") - - // verify delete requests made - val deleteRequestCaptor: Captor[DeleteObjectsRequest] = ArgCaptor[DeleteObjectsRequest] - Mockito.verify(f.mockedS3Client, Mockito.times(2)).deleteObjects(deleteRequestCaptor.capture) - val capturedDeleteRequests = deleteRequestCaptor.values - - capturedDeleteRequests.foreach(_.bucket shouldBe "bucket5") - // the requests should hold the paths listed - val deletedKeysRequested = capturedDeleteRequests.flatMap(_.delete.objects.asScala.map(_.key)) - deletedKeysRequested should contain theSameElementsInOrderAs (mockedObjects1 ++ mockedObjects2).map(_.key) - } - - private val unrelatedVersionObjects = Seq( - MockedObjectDef("publish/path/enceladus_info_date=2020-02-22/enceladus_info_version=aaaa/unrelated.file"), - MockedObjectDef("publish/path/enceladus_info_date=2020-02-22/enceladus_info_version=-6/unrelated.file") - ).map(_.toObject) - - Seq( - ("unrelated objects", unrelatedVersionObjects), - ("no objecdts", List.empty[S3Object]) - ).foreach { case (testCaseName, mockedObjects) => - it should s"find the latest version (simple case of $testCaseName - no recursion) to be 0" in { - val f = fixture - val path = "s3://bucket6/publish/path" - val reportDate = "2020-02-22" - - // mock S3 list response - val mockedListResponse = ListObjectsV2Response.builder().isTruncated(false) - .contents(mockedObjects.asJava).build - Mockito.when(f.mockedS3Client.listObjectsV2(ArgumentMatchers.any[ListObjectsV2Request])) - .thenReturn(mockedListResponse) - - val lastestVersion = f.mockedS3FsUtils.getLatestVersion(path, reportDate) - - // verify request content - val requestCaptor: Captor[ListObjectsV2Request] = ArgCaptor[ListObjectsV2Request] - Mockito.verify(f.mockedS3Client).listObjectsV2(requestCaptor.capture) - val capturedListRequests = requestCaptor.value - - // bucket & path should always be the same - capturedListRequests.bucket shouldBe "bucket6" - capturedListRequests.prefix shouldBe "publish/path/enceladus_info_date=2020-02-22/enceladus_info_version=" - - // verify returned value - lastestVersion shouldBe 0 - } - } - - it should s"find the latest version (with recursion)" in { - val f = fixture - val path = "s3://bucket6b/publish/path" - val reportDate = "2020-02-22" - - // mock S3 list responses - val mockedObjectForVersionLookoup1 = Seq( - MockedObjectDef("publish/path/enceladus_info_date=2020-02-22/enceladus_info_version=1/file.abc"), - MockedObjectDef("publish/path/enceladus_info_date=2020-02-22/enceladus_info_version=2/file2.abc"), - MockedObjectDef("publish/path/enceladus_info_date=2020-02-22/enceladus_info_version=BOGUS/bogus.file") - ).map(_.toObject) - - val mockedObjectForVersionLookoup2 = Seq( - MockedObjectDef("publish/path/enceladus_info_date=2020-02-22/enceladus_info_version=4/file.abc"), - MockedObjectDef("publish/path/enceladus_info_date=2020-02-22/enceladus_info_version=6/.hidden.abc") // hidden = no problem - ).map(_.toObject) - - val mockedListResponses: Seq[ListObjectsV2Response] = Seq( - ListObjectsV2Response.builder().isTruncated(true).nextContinuationToken("token1") - .contents(mockedObjectForVersionLookoup1.asJava).build, - ListObjectsV2Response.builder().isTruncated(false) - .contents(mockedObjectForVersionLookoup2.asJava).build - ) - - Mockito.when(f.mockedS3Client.listObjectsV2(ArgumentMatchers.any[ListObjectsV2Request])) - .thenReturn(mockedListResponses(0)) - .thenReturn(mockedListResponses(1)) - val latestVersion = f.mockedS3FsUtils.getLatestVersion(path, reportDate) - - // verify request content - val requestCaptor: Captor[ListObjectsV2Request] = ArgCaptor[ListObjectsV2Request] - Mockito.verify(f.mockedS3Client, Mockito.times(2)).listObjectsV2(requestCaptor.capture) - val capturedListRequests = requestCaptor.values - - // bucket & path should always be the same - capturedListRequests.foreach(_.bucket shouldBe "bucket6b") - capturedListRequests.foreach(_.prefix shouldBe "publish/path/enceladus_info_date=2020-02-22/enceladus_info_version=") - - // when truncated, the continuationToken was passed along to the next request to resume correctly - capturedListRequests.map(_.continuationToken) shouldBe List(null, "token1") - - // verify returned value - latestVersion shouldBe 6 - } - -} From 17dcfbc5a8a6fc4bc5945b72421404b7cc4fe7dc Mon Sep 17 00:00:00 2001 From: Daniel Kavan Date: Mon, 26 Oct 2020 15:55:01 +0100 Subject: [PATCH 03/12] import cleanup, tiny updates --- .../za/co/absa/enceladus/dao/auth/MenasCredentials.scala | 4 ++-- .../co/absa/enceladus/examples/CustomRuleSample1.scala | 1 - .../enceladus/conformance/DynamicConformanceJob.scala | 1 - .../co/absa/enceladus/conformance/HyperConformance.scala | 3 +-- .../standardization/StandardizationExecution.scala | 9 ++++++--- 5 files changed, 9 insertions(+), 9 deletions(-) diff --git a/dao/src/main/scala/za/co/absa/enceladus/dao/auth/MenasCredentials.scala b/dao/src/main/scala/za/co/absa/enceladus/dao/auth/MenasCredentials.scala index 63a430899..21afec390 100644 --- a/dao/src/main/scala/za/co/absa/enceladus/dao/auth/MenasCredentials.scala +++ b/dao/src/main/scala/za/co/absa/enceladus/dao/auth/MenasCredentials.scala @@ -18,8 +18,8 @@ package za.co.absa.enceladus.dao.auth import com.typesafe.config.ConfigFactory import org.apache.spark.sql.SparkSession import sun.security.krb5.internal.ktab.KeyTab -import za.co.absa.enceladus.utils.fs.{DistributedFsUtils, FileSystemUtils} -import FileSystemUtils.FileSystemExt +import za.co.absa.enceladus.utils.fs.FileSystemUtils +import za.co.absa.enceladus.utils.fs.FileSystemUtils.FileSystemExt sealed abstract class MenasCredentials { val username: String diff --git a/examples/src/main/scala/za/co/absa/enceladus/examples/CustomRuleSample1.scala b/examples/src/main/scala/za/co/absa/enceladus/examples/CustomRuleSample1.scala index df13f7625..b1c34560c 100644 --- a/examples/src/main/scala/za/co/absa/enceladus/examples/CustomRuleSample1.scala +++ b/examples/src/main/scala/za/co/absa/enceladus/examples/CustomRuleSample1.scala @@ -23,7 +23,6 @@ import za.co.absa.enceladus.dao.auth.MenasKerberosCredentials import za.co.absa.enceladus.dao.rest.RestDaoFactory import za.co.absa.enceladus.examples.interpreter.rules.custom.UppercaseCustomConformanceRule import za.co.absa.enceladus.model.Dataset -import za.co.absa.enceladus.utils.fs.HadoopFsUtils import za.co.absa.enceladus.utils.testUtils.HadoopFsTestBase import za.co.absa.enceladus.utils.time.TimeZoneNormalizer diff --git a/spark-jobs/src/main/scala/za/co/absa/enceladus/conformance/DynamicConformanceJob.scala b/spark-jobs/src/main/scala/za/co/absa/enceladus/conformance/DynamicConformanceJob.scala index fb254892f..2ec412df9 100644 --- a/spark-jobs/src/main/scala/za/co/absa/enceladus/conformance/DynamicConformanceJob.scala +++ b/spark-jobs/src/main/scala/za/co/absa/enceladus/conformance/DynamicConformanceJob.scala @@ -19,7 +19,6 @@ import org.apache.spark.sql.SparkSession import za.co.absa.enceladus.conformance.config.ConformanceConfig import za.co.absa.enceladus.dao.MenasDAO import za.co.absa.enceladus.dao.rest.RestDaoFactory -import za.co.absa.enceladus.utils.fs.DistributedFsUtils import za.co.absa.enceladus.utils.modules.SourcePhase object DynamicConformanceJob extends ConformanceExecution { 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 cd5336c31..41ca2aeb5 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 @@ -33,9 +33,8 @@ import za.co.absa.enceladus.dao.MenasDAO import za.co.absa.enceladus.dao.auth.{MenasCredentialsFactory, MenasKerberosCredentialsFactory, MenasPlainCredentialsFactory} import za.co.absa.enceladus.dao.rest.{MenasConnectionStringParser, RestDaoFactory} import za.co.absa.enceladus.model.Dataset -import za.co.absa.enceladus.utils.fs.{FileSystemUtils, HadoopFsUtils} +import za.co.absa.enceladus.utils.fs.FileSystemUtils.FileSystemExt import za.co.absa.hyperdrive.ingestor.api.transformer.{StreamTransformer, StreamTransformerFactory} -import FileSystemUtils.FileSystemExt class HyperConformance (implicit cmd: ConformanceConfig, featureSwitches: FeatureSwitches, 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 5895de679..00a95d2f7 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 @@ -53,13 +53,16 @@ trait StandardizationExecution extends CommonJobExecution { cmd: StandardizationConfigParser[T], spark: SparkSession): StructType = { val rawFs = preparationResult.fileSystems.rawFs + val rawFsUtils = rawFs.toFsUtils - val stdDirSize = rawFs.toFsUtils.getDirectorySize(preparationResult.pathCfg.rawPath) + val stdFs = preparationResult.fileSystems.standardizationFs + + val stdDirSize = rawFsUtils.getDirectorySize(preparationResult.pathCfg.rawPath) preparationResult.performance.startMeasurement(stdDirSize) // Enable Control Framework import za.co.absa.atum.AtumImplicits.SparkSessionWrapper - spark.enableControlMeasuresTracking(sourceInfoFile = s"${preparationResult.pathCfg.rawPath}/_INFO")(rawFs) + spark.enableControlMeasuresTracking(sourceInfoFile = s"${preparationResult.pathCfg.rawPath}/_INFO")(stdFs) .setControlMeasuresWorkflow(sourceId.toString) log.info(s"raw path: ${preparationResult.pathCfg.rawPath}") @@ -86,7 +89,7 @@ trait StandardizationExecution extends CommonJobExecution { PerformanceMetricTools.addJobInfoToAtumMetadata("std", preparationResult.pathCfg.rawPath, preparationResult.pathCfg.standardizationPath, - menasCredentials.username, args.mkString(" "))(spark, rawFs.toFsUtils) + menasCredentials.username, args.mkString(" "))(spark, rawFsUtils) dao.getSchema(preparationResult.dataset.schemaName, preparationResult.dataset.schemaVersion) } From 37f440d40a17abf99416a187ac32c44f19c7c11d Mon Sep 17 00:00:00 2001 From: Daniel Kavan Date: Mon, 2 Nov 2020 15:02:28 +0100 Subject: [PATCH 04/12] atum snapshot update (fs not needed for control measurement tracking, is derived form the path internally) --- .../za/co/absa/enceladus/conformance/ConformanceExecution.scala | 2 +- .../enceladus/standardization/StandardizationExecution.scala | 2 +- .../enceladus/utils/performance/PerformanceMetricTools.scala | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) 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 f46fae9ba..450e75ed7 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 @@ -69,7 +69,7 @@ trait ConformanceExecution extends CommonJobExecution{ } // InputPath is standardizationPath in the combined job - spark.enableControlMeasuresTracking(s"${preparationResult.pathCfg.standardizationPath}/_INFO")(preparationResult.fileSystems.publishFs) + spark.enableControlMeasuresTracking(s"${preparationResult.pathCfg.standardizationPath}/_INFO") .setControlMeasuresWorkflow(sourceId.toString) // Enable control framework performance optimization for pipeline-like jobs 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 00a95d2f7..748852c71 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 @@ -62,7 +62,7 @@ trait StandardizationExecution extends CommonJobExecution { // Enable Control Framework import za.co.absa.atum.AtumImplicits.SparkSessionWrapper - spark.enableControlMeasuresTracking(sourceInfoFile = s"${preparationResult.pathCfg.rawPath}/_INFO")(stdFs) + spark.enableControlMeasuresTracking(sourceInfoFile = s"${preparationResult.pathCfg.rawPath}/_INFO") .setControlMeasuresWorkflow(sourceId.toString) log.info(s"raw path: ${preparationResult.pathCfg.rawPath}") diff --git a/utils/src/main/scala/za/co/absa/enceladus/utils/performance/PerformanceMetricTools.scala b/utils/src/main/scala/za/co/absa/enceladus/utils/performance/PerformanceMetricTools.scala index 63e395950..3e8a18f78 100644 --- a/utils/src/main/scala/za/co/absa/enceladus/utils/performance/PerformanceMetricTools.scala +++ b/utils/src/main/scala/za/co/absa/enceladus/utils/performance/PerformanceMetricTools.scala @@ -20,7 +20,7 @@ import org.apache.spark.sql.functions.{col, size, sum} import org.slf4j.{Logger, LoggerFactory} import za.co.absa.atum.core.Atum import za.co.absa.enceladus.utils.error.ErrorMessage -import za.co.absa.enceladus.utils.fs.{DistributedFsUtils, HadoopFsUtils} +import za.co.absa.enceladus.utils.fs.HadoopFsUtils import za.co.absa.enceladus.utils.general.ProjectMetadataTools import za.co.absa.enceladus.utils.schema.SchemaUtils From 95f4179886f351d4fc59daf41966b42b1cfcf039 Mon Sep 17 00:00:00 2001 From: Daniel Kavan Date: Mon, 2 Nov 2020 16:05:44 +0100 Subject: [PATCH 05/12] cleanup --- .../enceladus/common/config/FileSystems.scala | 23 +------------------ .../enceladus/utils/fs/FileSystemUtils.scala | 2 +- 2 files changed, 2 insertions(+), 23 deletions(-) diff --git a/spark-jobs/src/main/scala/za/co/absa/enceladus/common/config/FileSystems.scala b/spark-jobs/src/main/scala/za/co/absa/enceladus/common/config/FileSystems.scala index ce12c7256..ea38373d5 100644 --- a/spark-jobs/src/main/scala/za/co/absa/enceladus/common/config/FileSystems.scala +++ b/spark-jobs/src/main/scala/za/co/absa/enceladus/common/config/FileSystems.scala @@ -18,10 +18,7 @@ package za.co.absa.enceladus.common.config import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.FileSystem import org.slf4j.{Logger, LoggerFactory} -import za.co.absa.enceladus.utils.fs.{FileSystemUtils, HadoopFsUtils} - -import scala.collection.concurrent.TrieMap - +import za.co.absa.enceladus.utils.fs.FileSystemUtils case class FileSystems(rawFs: FileSystem, publishFs: FileSystem, standardizationFs: FileSystem) @@ -34,24 +31,6 @@ object FileSystems { standardizationFs = FileSystemUtils.getFileSystemFromPath(pathConfig.standardizationPath) ) -// private[config] val fsUtilsCache = TrieMap[FileSystem, HadoopFsUtils]() -// -// implicit class FileSystemExt(fs: FileSystem) { -// /** -// * Given the FileSystem object `fs` that this method is called on, -// * the appropriate HadoopFsUtils is either newly created or returned form cache. -// * @return -// */ -// def toFsUtils: HadoopFsUtils = { -// fsUtilsCache.getOrElseUpdate(fs, { -// log.info(s"reusing cached fsUtils for FS ${fs.getUri} / ${fs.toString}") -// new HadoopFsUtils()(fs)} -// -// ) -// } -// } - - } diff --git a/utils/src/main/scala/za/co/absa/enceladus/utils/fs/FileSystemUtils.scala b/utils/src/main/scala/za/co/absa/enceladus/utils/fs/FileSystemUtils.scala index 52ae347ce..8f52ddb4f 100644 --- a/utils/src/main/scala/za/co/absa/enceladus/utils/fs/FileSystemUtils.scala +++ b/utils/src/main/scala/za/co/absa/enceladus/utils/fs/FileSystemUtils.scala @@ -58,7 +58,7 @@ object FileSystemUtils { */ def toFsUtils: HadoopFsUtils = { fsUtilsCache.getOrElseUpdate(fs, { - log.info(s"reusing cached fsUtils for FS ${fs.getUri} / ${fs.toString}") + log.debug(s"reusing cached fsUtils for FS ${fs.getUri} / ${fs.toString}") new HadoopFsUtils()(fs)} ) From 33c7ee22f64f84c07b2af7f89424b68298f323c7 Mon Sep 17 00:00:00 2001 From: Daniel Kavan Date: Thu, 5 Nov 2020 13:53:43 +0100 Subject: [PATCH 06/12] atum final version usage PR touch ups: --- pom.xml | 2 +- .../za/co/absa/enceladus/common/config/FileSystems.scala | 2 -- .../absa/enceladus/conformance/ConformanceExecution.scala | 2 +- .../StandardizationAndConformanceExecution.scala | 2 +- .../za/co/absa/enceladus/utils/fs/FileSystemUtils.scala | 8 ++++---- 5 files changed, 7 insertions(+), 9 deletions(-) diff --git a/pom.xml b/pom.xml index 7da051630..ef05789c6 100644 --- a/pom.xml +++ b/pom.xml @@ -144,7 +144,7 @@ 1.6 3.1.1 - 3.0.1-SNAPSHOT + 3.1.0 2.13.65 2.7.3 3.5.4 diff --git a/spark-jobs/src/main/scala/za/co/absa/enceladus/common/config/FileSystems.scala b/spark-jobs/src/main/scala/za/co/absa/enceladus/common/config/FileSystems.scala index ea38373d5..34ddf1ae0 100644 --- a/spark-jobs/src/main/scala/za/co/absa/enceladus/common/config/FileSystems.scala +++ b/spark-jobs/src/main/scala/za/co/absa/enceladus/common/config/FileSystems.scala @@ -17,13 +17,11 @@ package za.co.absa.enceladus.common.config import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.FileSystem -import org.slf4j.{Logger, LoggerFactory} import za.co.absa.enceladus.utils.fs.FileSystemUtils case class FileSystems(rawFs: FileSystem, publishFs: FileSystem, standardizationFs: FileSystem) object FileSystems { - val log: Logger = LoggerFactory.getLogger(this.getClass) def fromPathConfig(pathConfig: PathConfig)(implicit hadoopConf: Configuration): FileSystems = FileSystems( rawFs = FileSystemUtils.getFileSystemFromPath(pathConfig.rawPath), 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 450e75ed7..3fcc91a58 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 @@ -43,7 +43,7 @@ import za.co.absa.enceladus.utils.schema.SchemaUtils import scala.util.control.NonFatal import scala.util.{Failure, Success, Try} -trait ConformanceExecution extends CommonJobExecution{ +trait ConformanceExecution extends CommonJobExecution { private val conformanceReader = new ConformancePropertiesProvider private val sourceId = SourcePhase.Conformance diff --git a/spark-jobs/src/main/scala/za/co/absa/enceladus/standardization_conformance/StandardizationAndConformanceExecution.scala b/spark-jobs/src/main/scala/za/co/absa/enceladus/standardization_conformance/StandardizationAndConformanceExecution.scala index 036df5d85..25c9e6210 100644 --- a/spark-jobs/src/main/scala/za/co/absa/enceladus/standardization_conformance/StandardizationAndConformanceExecution.scala +++ b/spark-jobs/src/main/scala/za/co/absa/enceladus/standardization_conformance/StandardizationAndConformanceExecution.scala @@ -25,7 +25,7 @@ import za.co.absa.enceladus.utils.fs.FileSystemUtils.FileSystemExt trait StandardizationAndConformanceExecution extends StandardizationExecution with ConformanceExecution - with CommonJobExecution{ + with CommonJobExecution { override def getPathConfig[T](cmd: JobConfigParser[T], dataset: Dataset, reportVersion: Int): PathConfig = { val defaultConfig = super[CommonJobExecution].getPathConfig(cmd, dataset, reportVersion) diff --git a/utils/src/main/scala/za/co/absa/enceladus/utils/fs/FileSystemUtils.scala b/utils/src/main/scala/za/co/absa/enceladus/utils/fs/FileSystemUtils.scala index 8f52ddb4f..d97671c53 100644 --- a/utils/src/main/scala/za/co/absa/enceladus/utils/fs/FileSystemUtils.scala +++ b/utils/src/main/scala/za/co/absa/enceladus/utils/fs/FileSystemUtils.scala @@ -31,9 +31,9 @@ object FileSystemUtils { /** * Will yeild a [[FileSystem]] for path. If path prefix suggest S3, S3 FS is returned, HDFS otherwise. - * @param path - * @param hadoopConf - * @return + * @param path full path - used to determinte the kind of FS used (e.g. "s3://bucket1/path/to/file" or "/on/hdfs") + * @param hadoopConf hadoop Configuration object + * @return FileSystem instance (backed by S3/HDFS) */ def getFileSystemFromPath(path: String)(implicit hadoopConf: Configuration): FileSystem = { path.toS3Location match { @@ -48,7 +48,7 @@ object FileSystemUtils { } } - private[fs] val fsUtilsCache = TrieMap[FileSystem, HadoopFsUtils]() + private val fsUtilsCache = TrieMap[FileSystem, HadoopFsUtils]() implicit class FileSystemExt(fs: FileSystem) { /** From cb48db74bdadcdd4e0877810b386e452bef013c9 Mon Sep 17 00:00:00 2001 From: Daniel Kavan Date: Thu, 5 Nov 2020 16:36:59 +0100 Subject: [PATCH 07/12] pr touch-ups --- .../utils/testUtils/HadoopFsTestBase.scala | 11 ++----- .../utils/testUtils/HasSparkSession.scala | 22 ++++++++++++++ .../enceladus/utils/fs/HdfsUtilsSpec.scala | 29 +++++++++---------- 3 files changed, 39 insertions(+), 23 deletions(-) create mode 100644 utils/src/main/scala/za/co/absa/enceladus/utils/testUtils/HasSparkSession.scala diff --git a/utils/src/main/scala/za/co/absa/enceladus/utils/testUtils/HadoopFsTestBase.scala b/utils/src/main/scala/za/co/absa/enceladus/utils/testUtils/HadoopFsTestBase.scala index 65d5c7c5b..041001d97 100644 --- a/utils/src/main/scala/za/co/absa/enceladus/utils/testUtils/HadoopFsTestBase.scala +++ b/utils/src/main/scala/za/co/absa/enceladus/utils/testUtils/HadoopFsTestBase.scala @@ -17,18 +17,13 @@ package za.co.absa.enceladus.utils.testUtils import org.apache.hadoop.fs.FileSystem -import org.apache.spark.sql.SparkSession import za.co.absa.enceladus.utils.fs.FileSystemUtils.FileSystemExt +import za.co.absa.enceladus.utils.fs.HadoopFsUtils trait HadoopFsTestBase extends HasSparkSession { - implicit val fs = FileSystem.get(spark.sparkContext.hadoopConfiguration) - implicit val fsUtils = fs.toFsUtils + implicit val fs: FileSystem = FileSystem.get(spark.sparkContext.hadoopConfiguration) + implicit val fsUtils: HadoopFsUtils = fs.toFsUtils } -trait HasSparkSession { - def spark: SparkSession -} - - diff --git a/utils/src/main/scala/za/co/absa/enceladus/utils/testUtils/HasSparkSession.scala b/utils/src/main/scala/za/co/absa/enceladus/utils/testUtils/HasSparkSession.scala new file mode 100644 index 000000000..4a3fdaaa4 --- /dev/null +++ b/utils/src/main/scala/za/co/absa/enceladus/utils/testUtils/HasSparkSession.scala @@ -0,0 +1,22 @@ +/* + * 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.spark.sql.SparkSession + +trait HasSparkSession { + def spark: SparkSession +} 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 87edb8e3e..5e32622a0 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 @@ -26,26 +26,25 @@ import za.co.absa.enceladus.utils.testUtils.{HadoopFsTestBase, SparkTestBase} * Unit tests for File system utils */ class HdfsUtilsSpec extends AnyWordSpec with Matchers with SparkTestBase with HadoopFsTestBase { - val hdfsUtils = fsUtils "splitUriPath" should { "split URI and path" in { val path = new Path("hdfs://some-host:8020/user/data/input") - val (prefix, rawPath) = hdfsUtils.splitUriPath(path) + val (prefix, rawPath) = fsUtils.splitUriPath(path) prefix shouldEqual "hdfs://some-host:8020" rawPath shouldEqual "/user/data/input" } "not split a path without URI prefix" in { val path = new Path("/projects/coreconformance/publish/dataset") - val (prefix, rawPath) = hdfsUtils.splitUriPath(path) + val (prefix, rawPath) = fsUtils.splitUriPath(path) prefix shouldEqual "" rawPath shouldEqual "/projects/coreconformance/publish/dataset" } "not split relative path" in { val path = new Path("data/input") - val (prefix, rawPath) = hdfsUtils.splitUriPath(path) + val (prefix, rawPath) = fsUtils.splitUriPath(path) prefix shouldEqual "" rawPath shouldEqual "data/input" } @@ -54,27 +53,27 @@ class HdfsUtilsSpec extends AnyWordSpec with Matchers with SparkTestBase with Ha "getDirectorySize" should { "throw an exception if the specified path does not exist" in { intercept[FileNotFoundException] { - hdfsUtils.getDirectorySize("src/test/resources/test_data/not_exist") + fsUtils.getDirectorySize("src/test/resources/test_data/not_exist") } } "return the file size if a single file is specified" in { - val dirSize = hdfsUtils.getDirectorySize("src/test/resources/test_data/test_dir/dummy.txt") + val dirSize = fsUtils.getDirectorySize("src/test/resources/test_data/test_dir/dummy.txt") assert(dirSize == 20L) } "return the file size if a single hidden file is specified" in { - val dirSize = hdfsUtils.getDirectorySize("src/test/resources/test_data/test_dir/_hidden_dummy.txt") + val dirSize = fsUtils.getDirectorySize("src/test/resources/test_data/test_dir/_hidden_dummy.txt") assert(dirSize == 27L) } "return the size of all files in a directory" in { - val dirSize = hdfsUtils.getDirectorySize("src/test/resources/test_data/test_dir") + val dirSize = fsUtils.getDirectorySize("src/test/resources/test_data/test_dir") assert(dirSize == 47L) } "return the size of all files recursively" in { - val dirSize = hdfsUtils.getDirectorySize("src/test/resources/test_data/test_dir2") + val dirSize = fsUtils.getDirectorySize("src/test/resources/test_data/test_dir2") assert(dirSize == 87L) } } @@ -82,32 +81,32 @@ class HdfsUtilsSpec extends AnyWordSpec with Matchers with SparkTestBase with Ha "getDirectorySizeNoHidden" should { "throw an exception if the specified path does not exist" in { intercept[FileNotFoundException] { - hdfsUtils.getDirectorySizeNoHidden("src/test/resources/test_data/not_exist") + fsUtils.getDirectorySizeNoHidden("src/test/resources/test_data/not_exist") } } "return the file size if a single file is specified" in { - val dirSize = hdfsUtils.getDirectorySizeNoHidden("src/test/resources/test_data/test_dir/dummy.txt") + val dirSize = fsUtils.getDirectorySizeNoHidden("src/test/resources/test_data/test_dir/dummy.txt") assert(dirSize == 20L) } "return the file size if a single hidden file is specified" in { - val dirSize = hdfsUtils.getDirectorySizeNoHidden("src/test/resources/test_data/test_dir/_hidden_dummy.txt") + val dirSize = fsUtils.getDirectorySizeNoHidden("src/test/resources/test_data/test_dir/_hidden_dummy.txt") assert(dirSize == 27L) } "return the size of all non-hidden files in a directory" in { - val dirSize = hdfsUtils.getDirectorySizeNoHidden("src/test/resources/test_data/test_dir") + val dirSize = fsUtils.getDirectorySizeNoHidden("src/test/resources/test_data/test_dir") assert(dirSize == 20L) } "return the size of all non-hidden files recursively along non-hidden paths" in { - val dirSize = hdfsUtils.getDirectorySizeNoHidden("src/test/resources/test_data/test_dir2") + val dirSize = fsUtils.getDirectorySizeNoHidden("src/test/resources/test_data/test_dir2") assert(dirSize == 40L) } "return the size of all non-hidden files if a hidden directory is specified explicitly" in { - val dirSize = hdfsUtils.getDirectorySizeNoHidden("src/test/resources/test_data/test_dir2/_inner_dir") + val dirSize = fsUtils.getDirectorySizeNoHidden("src/test/resources/test_data/test_dir2/_inner_dir") assert(dirSize == 20L) } } From 0b21ba5ae4a177de3da573b56f520156c05aef5a Mon Sep 17 00:00:00 2001 From: Daniel Kavan Date: Fri, 6 Nov 2020 09:48:44 +0100 Subject: [PATCH 08/12] FileSystems are now part of PathConfig --- .../enceladus/common/CommonJobExecution.scala | 40 +++++++------- .../enceladus/common/config/FileSystems.scala | 34 ------------ .../enceladus/common/config/PathConfig.scala | 31 +++++++++-- .../conformance/ConformanceExecution.scala | 46 ++++++++-------- .../StandardizationExecution.scala | 52 +++++++++---------- .../standardization/StandardizationJob.scala | 4 +- ...andardizationAndConformanceExecution.scala | 19 ++++--- .../StandardizationAndConformanceJob.scala | 4 +- .../config/ConformanceParserSuite.scala | 10 ++-- .../config/StandardizationParserSuite.scala | 10 ++-- 10 files changed, 125 insertions(+), 125 deletions(-) delete mode 100644 spark-jobs/src/main/scala/za/co/absa/enceladus/common/config/FileSystems.scala 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 582ba3c23..8fa7dfb6d 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 @@ -27,7 +27,7 @@ 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.{FileSystems, JobConfigParser, PathConfig} +import za.co.absa.enceladus.common.config.{JobConfigParser, PathConfig, PathConfigEntry} import za.co.absa.enceladus.common.plugin.PostProcessingService import za.co.absa.enceladus.common.plugin.menas.{MenasPlugin, MenasRunUrl} import za.co.absa.enceladus.common.version.SparkVersionGuard @@ -36,7 +36,8 @@ import za.co.absa.enceladus.dao.rest.MenasConnectionStringParser import za.co.absa.enceladus.model.Dataset import za.co.absa.enceladus.plugins.builtin.errorsender.params.ErrorSenderPluginParams import za.co.absa.enceladus.utils.config.{ConfigReader, SecureConfig} -import za.co.absa.enceladus.utils.fs.{DistributedFsUtils, FileSystemUtils} +import za.co.absa.enceladus.utils.fs.FileSystemUtils +import za.co.absa.enceladus.utils.fs.FileSystemUtils.FileSystemExt import za.co.absa.enceladus.utils.general.ProjectMetadataTools import za.co.absa.enceladus.utils.modules.SourcePhase import za.co.absa.enceladus.utils.modules.SourcePhase.Standardization @@ -51,7 +52,6 @@ trait CommonJobExecution { protected case class PreparationResult(dataset: Dataset, reportVersion: Int, pathCfg: PathConfig, - fileSystems: FileSystems, performance: PerformanceMeasurer) TimeZoneNormalizer.normalizeJVMTimeZone() @@ -91,9 +91,8 @@ trait CommonJobExecution { val dataset = dao.getDataset(cmd.datasetName, cmd.datasetVersion) val reportVersion = getReportVersion(cmd, dataset) val pathCfg: PathConfig = getPathConfig(cmd, dataset, reportVersion) - val fileSystems: FileSystems = FileSystems.fromPathConfig(pathCfg) - validateOutputPath(pathCfg)(fileSystems) + validateOutputPath(pathCfg) // Enable Spline import za.co.absa.spline.harvester.SparkLineageInitializer._ @@ -102,15 +101,16 @@ trait CommonJobExecution { // Enable non-default persistence storage level if provided in the command line cmd.persistStorageLevel.foreach(Atum.setCachingStorageLevel) - PreparationResult(dataset, reportVersion, pathCfg, fileSystems, new PerformanceMeasurer(spark.sparkContext.appName)) + PreparationResult(dataset, reportVersion, pathCfg, new PerformanceMeasurer(spark.sparkContext.appName)) } - protected def validateOutputPath(pathConfig: PathConfig)(implicit fileSystems: FileSystems): Unit + protected def validateOutputPath(pathConfig: PathConfig): Unit - protected def validateIfPathAlreadyExists(path: String)(implicit fsUtils: DistributedFsUtils): Unit = { - if (fsUtils.exists(path)) { + protected def validateIfPathAlreadyExists(entry: PathConfigEntry): Unit = { + val fsUtils = entry.fileSystem.toFsUtils + if (fsUtils.exists(entry.path)) { throw new IllegalStateException( - s"Path $path already exists. Increment the run version, or delete $path" + s"Path ${entry.path} already exists. Increment the run version, or delete ${entry.path}" ) } } @@ -118,8 +118,8 @@ trait CommonJobExecution { protected def runPostProcessing[T](sourcePhase: SourcePhase, preparationResult: PreparationResult, jobCmdConfig: JobConfigParser[T]) (implicit spark: SparkSession): Unit = { val outputPath = sourcePhase match { - case Standardization => preparationResult.pathCfg.standardizationPath - case _ => preparationResult.pathCfg.publishPath + case Standardization => preparationResult.pathCfg.standardization.path + case _ => preparationResult.pathCfg.publish.path } log.info(s"rereading outputPath $outputPath to run postProcessing") @@ -161,12 +161,14 @@ trait CommonJobExecution { } } - protected def getPathConfig[T](cmd: JobConfigParser[T], dataset: Dataset, reportVersion: Int): PathConfig = { - PathConfig( - rawPath = buildRawPath(cmd, dataset, reportVersion), - publishPath = buildPublishPath(cmd, dataset, reportVersion), - standardizationPath = getStandardizationPath(cmd, reportVersion) - ) + protected def getPathConfig[T](cmd: JobConfigParser[T], dataset: Dataset, reportVersion: Int) + (implicit hadoopConf: Configuration): PathConfig = { + + val rawPath = buildRawPath(cmd, dataset, reportVersion) + val publishPath = buildPublishPath(cmd, dataset, reportVersion) + val standardizationPath = getStandardizationPath(cmd, reportVersion) + + PathConfig.fromPaths(rawPath, publishPath, standardizationPath) } private def buildPublishPath[T](cmd: JobConfigParser[T], ds: Dataset, reportVersion: Int): String = { @@ -245,7 +247,7 @@ trait CommonJobExecution { case None => import FileSystemUtils.FileSystemExt - // publishFs for this specific feature (needed for missing reportVersion until reusable common FileSystems object is established) + // publishFs for this specific feature (needed for missing reportVersion until reusable common "PathConfig" with FS objects is established) implicit val tempPublishFs: FileSystem = FileSystemUtils.getFileSystemFromPath(dataset.hdfsPublishPath) val newVersion = tempPublishFs.toFsUtils.getLatestVersion(dataset.hdfsPublishPath, jobConfig.reportDate) + 1 log.warn(s"Report version not provided, inferred report version: $newVersion") diff --git a/spark-jobs/src/main/scala/za/co/absa/enceladus/common/config/FileSystems.scala b/spark-jobs/src/main/scala/za/co/absa/enceladus/common/config/FileSystems.scala deleted file mode 100644 index 34ddf1ae0..000000000 --- a/spark-jobs/src/main/scala/za/co/absa/enceladus/common/config/FileSystems.scala +++ /dev/null @@ -1,34 +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.common.config - -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.FileSystem -import za.co.absa.enceladus.utils.fs.FileSystemUtils - -case class FileSystems(rawFs: FileSystem, publishFs: FileSystem, standardizationFs: FileSystem) - -object FileSystems { - - def fromPathConfig(pathConfig: PathConfig)(implicit hadoopConf: Configuration): FileSystems = FileSystems( - rawFs = FileSystemUtils.getFileSystemFromPath(pathConfig.rawPath), - publishFs = FileSystemUtils.getFileSystemFromPath(pathConfig.publishPath), - standardizationFs = FileSystemUtils.getFileSystemFromPath(pathConfig.standardizationPath) - ) - -} - - diff --git a/spark-jobs/src/main/scala/za/co/absa/enceladus/common/config/PathConfig.scala b/spark-jobs/src/main/scala/za/co/absa/enceladus/common/config/PathConfig.scala index 3ba4b6a09..1a381df38 100644 --- a/spark-jobs/src/main/scala/za/co/absa/enceladus/common/config/PathConfig.scala +++ b/spark-jobs/src/main/scala/za/co/absa/enceladus/common/config/PathConfig.scala @@ -15,11 +15,34 @@ package za.co.absa.enceladus.common.config +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.FileSystem +import za.co.absa.enceladus.utils.fs.FileSystemUtils + /** * - * @param rawPath Input path of the job - * @param publishPath Output path of the job - * @param standardizationPath In case of StandardizationJob and ConformanceJob it should be None and for + * @param raw Input path+fs of the job + * @param publish Output path+fs of the job + * @param standardization In case of StandardizationJob and ConformanceJob it should be None and for * StandardizationConformanceJob it should represent the intermediate standardization path */ -case class PathConfig(rawPath: String, publishPath: String, standardizationPath: String) +case class PathConfig(raw: PathConfigEntry, + publish: PathConfigEntry, + standardization: PathConfigEntry) + +object PathConfig { + def fromPaths(rawPath: String, publishPath: String, standardizationPath: String) + (implicit hadoopConf: Configuration): PathConfig = + PathConfig( + PathConfigEntry.fromPath(rawPath), + PathConfigEntry.fromPath(publishPath), + PathConfigEntry.fromPath(standardizationPath) + ) +} + +case class PathConfigEntry(path: String, fileSystem: FileSystem) + +object PathConfigEntry { + def fromPath(path: String)(implicit hadoopConf: Configuration): PathConfigEntry = + PathConfigEntry(path, FileSystemUtils.getFileSystemFromPath(path)) +} 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 3fcc91a58..720d87f55 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 @@ -17,6 +17,7 @@ package za.co.absa.enceladus.conformance import java.io.{PrintWriter, StringWriter} +import org.apache.hadoop.conf.Configuration import org.apache.spark.sql.functions.{lit, to_date} import org.apache.spark.sql.{DataFrame, SparkSession} import za.co.absa.atum.AtumImplicits @@ -24,7 +25,7 @@ import za.co.absa.atum.AtumImplicits.DataSetWrapper import za.co.absa.atum.core.Atum import za.co.absa.enceladus.common.Constants.{InfoDateColumn, InfoDateColumnString, InfoVersionColumn, ReportDateFormat} import za.co.absa.enceladus.common.RecordIdGeneration._ -import za.co.absa.enceladus.common.config.{FileSystems, JobConfigParser, PathConfig} +import za.co.absa.enceladus.common.config.{JobConfigParser, PathConfig, PathConfigEntry} import za.co.absa.enceladus.common.plugin.menas.MenasPlugin import za.co.absa.enceladus.common.{CommonJobExecution, Constants, RecordIdGeneration} import za.co.absa.enceladus.conformance.config.{ConformanceConfig, ConformanceConfigParser} @@ -52,13 +53,13 @@ trait ConformanceExecution extends CommonJobExecution { cmd: ConformanceConfigParser[T], spark: SparkSession): Unit = { - val stdFsUtils = preparationResult.fileSystems.standardizationFs.toFsUtils + val stdFsUtils = preparationResult.pathCfg.standardization.fileSystem.toFsUtils - val stdDirSize = stdFsUtils.getDirectorySize(preparationResult.pathCfg.standardizationPath) + val stdDirSize = stdFsUtils.getDirectorySize(preparationResult.pathCfg.standardization.path) preparationResult.performance.startMeasurement(stdDirSize) - log.info(s"standardization path: ${preparationResult.pathCfg.standardizationPath}") - log.info(s"publish path: ${preparationResult.pathCfg.publishPath}") + log.info(s"standardization path: ${preparationResult.pathCfg.standardization.path}") + log.info(s"publish path: ${preparationResult.pathCfg.publish.path}") // Enable Control Framework import za.co.absa.atum.AtumImplicits.SparkSessionWrapper @@ -69,7 +70,7 @@ trait ConformanceExecution extends CommonJobExecution { } // InputPath is standardizationPath in the combined job - spark.enableControlMeasuresTracking(s"${preparationResult.pathCfg.standardizationPath}/_INFO") + spark.enableControlMeasuresTracking(s"${preparationResult.pathCfg.standardization.path}/_INFO") .setControlMeasuresWorkflow(sourceId.toString) // Enable control framework performance optimization for pipeline-like jobs @@ -84,21 +85,22 @@ trait ConformanceExecution extends CommonJobExecution { preparationResult.reportVersion) } - override def getPathConfig[T](cmd: JobConfigParser[T], dataset: Dataset, reportVersion: Int): PathConfig = { + override def getPathConfig[T](cmd: JobConfigParser[T], dataset: Dataset, reportVersion: Int) + (implicit hadoopConf: Configuration): PathConfig = { val initialConfig = super.getPathConfig(cmd, dataset, reportVersion) cmd.asInstanceOf[ConformanceConfig].publishPathOverride match { case None => initialConfig - case Some(providedRawPath) => initialConfig.copy(publishPath = providedRawPath) + case Some(providedPublishPath) => initialConfig.copy(publish = PathConfigEntry.fromPath(providedPublishPath)) } } - override def validateOutputPath(pathConfig: PathConfig)(implicit fileSystems: FileSystems): Unit = { + override def validateOutputPath(pathConfig: PathConfig): Unit = { // Conformance output is validated in the publish FS - validateIfPathAlreadyExists(pathConfig.publishPath)(fileSystems.publishFs.toFsUtils) + validateIfPathAlreadyExists(pathConfig.publish) } protected def readConformanceInputData(pathCfg: PathConfig)(implicit spark: SparkSession): DataFrame = { - spark.read.parquet(pathCfg.standardizationPath) + spark.read.parquet(pathCfg.standardization.path) } protected def conform[T](inputData: DataFrame, preparationResult: PreparationResult) @@ -106,7 +108,7 @@ trait ConformanceExecution extends CommonJobExecution { val recordIdGenerationStrategy = getRecordIdGenerationStrategyFromConfig(conf) implicit val featureSwitcher: FeatureSwitches = conformanceReader.readFeatureSwitches() - implicit val stdFs = preparationResult.fileSystems.standardizationFs + implicit val stdFs = preparationResult.pathCfg.standardization.fileSystem Try { handleControlInfoValidation() @@ -136,13 +138,13 @@ trait ConformanceExecution extends CommonJobExecution { (implicit spark: SparkSession, cmd: ConformanceConfigParser[T]): Unit = { val cmdLineArgs: String = args.mkString(" ") - val stdFs = preparationResult.fileSystems.standardizationFs - val publishFs = preparationResult.fileSystems.publishFs + val stdFs = preparationResult.pathCfg.standardization.fileSystem + val publishFs = preparationResult.pathCfg.publish.fileSystem PerformanceMetricTools.addJobInfoToAtumMetadata( "conform", - preparationResult.pathCfg.standardizationPath, - preparationResult.pathCfg.publishPath, + preparationResult.pathCfg.standardization.path, + preparationResult.pathCfg.publish.path, menasCredentials.username, cmdLineArgs )(spark, stdFs.toFsUtils) @@ -159,23 +161,23 @@ trait ConformanceExecution extends CommonJobExecution { handleEmptyOutput(SourcePhase.Conformance) } - withPartCols.write.parquet(preparationResult.pathCfg.publishPath) + withPartCols.write.parquet(preparationResult.pathCfg.publish.path) - val publishDirSize = publishFs.toFsUtils.getDirectorySize(preparationResult.pathCfg.publishPath) + val publishDirSize = publishFs.toFsUtils.getDirectorySize(preparationResult.pathCfg.publish.path) preparationResult.performance.finishMeasurement(publishDirSize, recordCount) PerformanceMetricTools.addPerformanceMetricsToAtumMetadata( spark, "conform", - preparationResult.pathCfg.standardizationPath, - preparationResult.pathCfg.publishPath, + preparationResult.pathCfg.standardization.path, + preparationResult.pathCfg.publish.path, menasCredentials.username, cmdLineArgs )(stdFs.toFsUtils, publishFs.toFsUtils) - withPartCols.writeInfoFile(preparationResult.pathCfg.publishPath)(publishFs) + withPartCols.writeInfoFile(preparationResult.pathCfg.publish.path)(publishFs) writePerformanceMetrics(preparationResult.performance, cmd) if (conformanceReader.isAutocleanStdFolderEnabled()) { - stdFs.toFsUtils.deleteDirectoryRecursively(preparationResult.pathCfg.standardizationPath) + stdFs.toFsUtils.deleteDirectoryRecursively(preparationResult.pathCfg.standardization.path) } log.info(s"$sourceId finished successfully") } 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 748852c71..43f6ac554 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 @@ -18,12 +18,14 @@ package za.co.absa.enceladus.standardization import java.io.{PrintWriter, StringWriter} import java.util.UUID +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.FileSystem 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.common.RecordIdGeneration.getRecordIdGenerationStrategyFromConfig -import za.co.absa.enceladus.common.config.{FileSystems, JobConfigParser, PathConfig} +import za.co.absa.enceladus.common.config.{JobConfigParser, PathConfig, PathConfigEntry} import za.co.absa.enceladus.common.plugin.menas.MenasPlugin import za.co.absa.enceladus.common.{CommonJobExecution, Constants} import za.co.absa.enceladus.dao.MenasDAO @@ -32,14 +34,13 @@ import za.co.absa.enceladus.model.Dataset import za.co.absa.enceladus.standardization.config.{StandardizationConfig, StandardizationConfigParser} import za.co.absa.enceladus.standardization.interpreter.StandardizationInterpreter import za.co.absa.enceladus.standardization.interpreter.stages.PlainSchemaGenerator +import za.co.absa.enceladus.utils.fs.FileSystemUtils.FileSystemExt import za.co.absa.enceladus.utils.fs.{DistributedFsUtils, HadoopFsUtils} import za.co.absa.enceladus.utils.modules.SourcePhase import za.co.absa.enceladus.utils.performance.PerformanceMetricTools import za.co.absa.enceladus.utils.schema.{MetadataKeys, SchemaUtils, SparkUtils} import za.co.absa.enceladus.utils.udf.UDFLibrary import za.co.absa.enceladus.utils.validation.ValidationException -import za.co.absa.enceladus.utils.fs.FileSystemUtils.FileSystemExt -import org.apache.hadoop.fs.FileSystem import scala.util.control.NonFatal @@ -52,21 +53,21 @@ trait StandardizationExecution extends CommonJobExecution { (implicit dao: MenasDAO, cmd: StandardizationConfigParser[T], spark: SparkSession): StructType = { - val rawFs = preparationResult.fileSystems.rawFs + val rawFs = preparationResult.pathCfg.raw.fileSystem val rawFsUtils = rawFs.toFsUtils - val stdFs = preparationResult.fileSystems.standardizationFs + val stdFs = preparationResult.pathCfg.standardization.fileSystem - val stdDirSize = rawFsUtils.getDirectorySize(preparationResult.pathCfg.rawPath) + val stdDirSize = rawFsUtils.getDirectorySize(preparationResult.pathCfg.raw.path) preparationResult.performance.startMeasurement(stdDirSize) // Enable Control Framework import za.co.absa.atum.AtumImplicits.SparkSessionWrapper - spark.enableControlMeasuresTracking(sourceInfoFile = s"${preparationResult.pathCfg.rawPath}/_INFO") + spark.enableControlMeasuresTracking(sourceInfoFile = s"${preparationResult.pathCfg.raw.path}/_INFO") .setControlMeasuresWorkflow(sourceId.toString) - log.info(s"raw path: ${preparationResult.pathCfg.rawPath}") - log.info(s"standardization path: ${preparationResult.pathCfg.standardizationPath}") + log.info(s"raw path: ${preparationResult.pathCfg.raw.path}") + log.info(s"standardization path: ${preparationResult.pathCfg.standardization.path}") // Enable control framework performance optimization for pipeline-like jobs Atum.setAllowUnpersistOldDatasets(true) @@ -87,24 +88,25 @@ trait StandardizationExecution extends CommonJobExecution { Atum.setAdditionalInfo("raw_format" -> cmd.rawFormat) PerformanceMetricTools.addJobInfoToAtumMetadata("std", - preparationResult.pathCfg.rawPath, - preparationResult.pathCfg.standardizationPath, + preparationResult.pathCfg.raw.path, + preparationResult.pathCfg.standardization.path, menasCredentials.username, args.mkString(" "))(spark, rawFsUtils) dao.getSchema(preparationResult.dataset.schemaName, preparationResult.dataset.schemaVersion) } - override def getPathConfig[T](cmd: JobConfigParser[T], dataset: Dataset, reportVersion: Int): PathConfig = { + override def getPathConfig[T](cmd: JobConfigParser[T], dataset: Dataset, reportVersion: Int) + (implicit hadoopConf: Configuration): PathConfig = { val initialConfig = super.getPathConfig(cmd, dataset, reportVersion) cmd.asInstanceOf[StandardizationConfig].rawPathOverride match { case None => initialConfig - case Some(providedRawPath) => initialConfig.copy(rawPath = providedRawPath) + case Some(providedRawPath) => initialConfig.copy(raw = PathConfigEntry.fromPath(providedRawPath)) } } - override def validateOutputPath(pathConfig: PathConfig)(implicit fileSystems: FileSystems): Unit = { + override def validateOutputPath(pathConfig: PathConfig): Unit = { // Std output is validated in the std FS - validateIfPathAlreadyExists(pathConfig.standardizationPath)(fileSystems.standardizationFs.toFsUtils) + validateIfPathAlreadyExists(pathConfig.standardization) } protected def readStandardizationInputData[T](schema: StructType, @@ -132,8 +134,6 @@ trait StandardizationExecution extends CommonJobExecution { private def getColumnNameOfCorruptRecord[R](schema: StructType, cmd: StandardizationConfigParser[R]) (implicit spark: SparkSession): Option[String] = { // SparkUtils.setUniqueColumnNameOfCorruptRecord is called even if result is not used to avoid conflict - - import AtumImplicits.DataSetWrapper val columnNameOfCorruptRecord = SparkUtils.setUniqueColumnNameOfCorruptRecord(spark, schema) if (cmd.rawFormat.equalsIgnoreCase("fixed-width") || cmd.failOnInputNotPerSchema) { None @@ -173,8 +173,8 @@ trait StandardizationExecution extends CommonJobExecution { (implicit spark: SparkSession): DataFrame = { import za.co.absa.atum.AtumImplicits._ - val rawFs = preparationResult.fileSystems.rawFs - val stdFs = preparationResult.fileSystems.standardizationFs + val rawFs = preparationResult.pathCfg.raw.fileSystem + val stdFs = preparationResult.pathCfg.standardization.fileSystem val fieldRenames = SchemaUtils.getRenamesInSchema(schema) fieldRenames.foreach { @@ -192,19 +192,19 @@ trait StandardizationExecution extends CommonJobExecution { handleEmptyOutput(sourceId) } - log.info(s"Writing into standardized path ${preparationResult.pathCfg.standardizationPath}") - standardizedDF.write.parquet(preparationResult.pathCfg.standardizationPath) + log.info(s"Writing into standardized path ${preparationResult.pathCfg.standardization.path}") + standardizedDF.write.parquet(preparationResult.pathCfg.standardization.path) // Store performance metrics // (record count, directory sizes, elapsed time, etc. to _INFO file metadata and performance file) - val stdDirSize = stdFs.toFsUtils.getDirectorySize(preparationResult.pathCfg.standardizationPath) + val stdDirSize = stdFs.toFsUtils.getDirectorySize(preparationResult.pathCfg.standardization.path) preparationResult.performance.finishMeasurement(stdDirSize, recordCount) PerformanceMetricTools.addPerformanceMetricsToAtumMetadata( spark, "std", - preparationResult.pathCfg.rawPath, - preparationResult.pathCfg.standardizationPath, + preparationResult.pathCfg.raw.path, + preparationResult.pathCfg.standardization.path, menasCredentials.username, args.mkString(" ") )(rawFs.toFsUtils, stdFs.toFsUtils) @@ -212,8 +212,8 @@ trait StandardizationExecution extends CommonJobExecution { cmd.rowTag.foreach(rowTag => Atum.setAdditionalInfo("xml_row_tag" -> rowTag)) cmd.csvDelimiter.foreach(delimiter => Atum.setAdditionalInfo("csv_delimiter" -> delimiter)) - log.info(s"infoFilePath = ${preparationResult.pathCfg.standardizationPath}/_INFO") - standardizedDF.writeInfoFile(preparationResult.pathCfg.standardizationPath)(stdFs) + log.info(s"infoFilePath = ${preparationResult.pathCfg.standardization.path}/_INFO") + standardizedDF.writeInfoFile(preparationResult.pathCfg.standardization.path)(stdFs) writePerformanceMetrics(preparationResult.performance, cmd) log.info(s"$sourceId finished successfully") standardizedDF diff --git a/spark-jobs/src/main/scala/za/co/absa/enceladus/standardization/StandardizationJob.scala b/spark-jobs/src/main/scala/za/co/absa/enceladus/standardization/StandardizationJob.scala index d2c9fa769..f091aadda 100644 --- a/spark-jobs/src/main/scala/za/co/absa/enceladus/standardization/StandardizationJob.scala +++ b/spark-jobs/src/main/scala/za/co/absa/enceladus/standardization/StandardizationJob.scala @@ -39,8 +39,8 @@ object StandardizationJob extends StandardizationExecution { val preparationResult = prepareJob() val schema = prepareStandardization(args, menasCredentials, preparationResult) - implicit val rawFs: FileSystem = preparationResult.fileSystems.rawFs - val inputData = readStandardizationInputData(schema, cmd, preparationResult.pathCfg.rawPath, preparationResult.dataset) + implicit val rawFs: FileSystem = preparationResult.pathCfg.raw.fileSystem + val inputData = readStandardizationInputData(schema, cmd, preparationResult.pathCfg.raw.path, preparationResult.dataset) try { val result = standardize(inputData, schema, cmd) diff --git a/spark-jobs/src/main/scala/za/co/absa/enceladus/standardization_conformance/StandardizationAndConformanceExecution.scala b/spark-jobs/src/main/scala/za/co/absa/enceladus/standardization_conformance/StandardizationAndConformanceExecution.scala index 25c9e6210..e5eb73073 100644 --- a/spark-jobs/src/main/scala/za/co/absa/enceladus/standardization_conformance/StandardizationAndConformanceExecution.scala +++ b/spark-jobs/src/main/scala/za/co/absa/enceladus/standardization_conformance/StandardizationAndConformanceExecution.scala @@ -15,32 +15,35 @@ package za.co.absa.enceladus.standardization_conformance +import org.apache.hadoop.conf.Configuration import za.co.absa.enceladus.common.CommonJobExecution -import za.co.absa.enceladus.common.config.{FileSystems, JobConfigParser, PathConfig} +import za.co.absa.enceladus.common.config.{JobConfigParser, PathConfig, PathConfigEntry} import za.co.absa.enceladus.conformance.ConformanceExecution import za.co.absa.enceladus.model.Dataset import za.co.absa.enceladus.standardization.StandardizationExecution import za.co.absa.enceladus.standardization_conformance.config.StandardizationConformanceConfig -import za.co.absa.enceladus.utils.fs.FileSystemUtils.FileSystemExt trait StandardizationAndConformanceExecution extends StandardizationExecution with ConformanceExecution with CommonJobExecution { - override def getPathConfig[T](cmd: JobConfigParser[T], dataset: Dataset, reportVersion: Int): PathConfig = { + override def getPathConfig[T](cmd: JobConfigParser[T], dataset: Dataset, reportVersion: Int) + (implicit hadoopConf: Configuration): PathConfig = { val defaultConfig = super[CommonJobExecution].getPathConfig(cmd, dataset, reportVersion) val jobCmd = cmd.asInstanceOf[StandardizationConformanceConfig] val rawPathOverride = jobCmd.rawPathOverride val publishPathOverride = jobCmd.publishPathOverride - defaultConfig.copy(rawPath = rawPathOverride.getOrElse(defaultConfig.rawPath), - publishPath = publishPathOverride.getOrElse(defaultConfig.publishPath)) + defaultConfig.copy( + raw = PathConfigEntry.fromPath(rawPathOverride.getOrElse(defaultConfig.raw.path)), + publish = PathConfigEntry.fromPath(publishPathOverride.getOrElse(defaultConfig.publish.path)) + ) } - override def validateOutputPath(pathConfig: PathConfig)(implicit fileSystems: FileSystems): Unit = { + override def validateOutputPath(pathConfig: PathConfig): Unit = { // Std output is validated in the std FS - validateIfPathAlreadyExists(pathConfig.standardizationPath)(fileSystems.standardizationFs.toFsUtils) + validateIfPathAlreadyExists(pathConfig.standardization) // publish output is validated in the publish FS - validateIfPathAlreadyExists(pathConfig.publishPath)(fileSystems.publishFs.toFsUtils) + validateIfPathAlreadyExists(pathConfig.publish) } } diff --git a/spark-jobs/src/main/scala/za/co/absa/enceladus/standardization_conformance/StandardizationAndConformanceJob.scala b/spark-jobs/src/main/scala/za/co/absa/enceladus/standardization_conformance/StandardizationAndConformanceJob.scala index 12a4614e9..06bd70449 100644 --- a/spark-jobs/src/main/scala/za/co/absa/enceladus/standardization_conformance/StandardizationAndConformanceJob.scala +++ b/spark-jobs/src/main/scala/za/co/absa/enceladus/standardization_conformance/StandardizationAndConformanceJob.scala @@ -36,8 +36,8 @@ object StandardizationAndConformanceJob extends StandardizationAndConformanceExe val preparationResult = prepareJob() val schema = prepareStandardization(args, menasCredentials, preparationResult) - implicit val rawFs = preparationResult.fileSystems.rawFs - val inputData = readStandardizationInputData(schema, cmd, preparationResult.pathCfg.rawPath, preparationResult.dataset) + implicit val rawFs = preparationResult.pathCfg.raw.fileSystem + val inputData = readStandardizationInputData(schema, cmd, preparationResult.pathCfg.raw.path, preparationResult.dataset) try { val standardized = standardize(inputData, schema, cmd) 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 04ef0ce3c..a3a925dba 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 @@ -56,6 +56,8 @@ class ConformanceParserSuite extends AnyFunSuite with SparkTestBase { private object TestDynamicConformance extends ConformanceExecution + implicit val hadoopConf = spark.sparkContext.hadoopConfiguration + test("Test credentials file parsing "){ val credentials = MenasPlainCredentials.fromFile(menasCredentialsFile) @@ -180,19 +182,19 @@ class ConformanceParserSuite extends AnyFunSuite with SparkTestBase { "--menas-credentials-file", menasCredentialsFile, "--debug-set-publish-path", hdfsPublishPathOverride)) val publishPathNoFolderPrefix = TestDynamicConformance - .getPathConfig(cmdConfigNoFolderPrefix, conformanceDataset, cmdConfigNoFolderPrefix.reportVersion.get).publishPath + .getPathConfig(cmdConfigNoFolderPrefix, conformanceDataset, cmdConfigNoFolderPrefix.reportVersion.get).publish.path assert(publishPathNoFolderPrefix === s"$hdfsPublishPath/$infoDateColumn=$reportDate/$infoVersionColumn=$reportVersion") val publishPathFolderPrefix = TestDynamicConformance - .getPathConfig(cmdConfigFolderPrefix, conformanceDataset, cmdConfigFolderPrefix.reportVersion.get).publishPath + .getPathConfig(cmdConfigFolderPrefix, conformanceDataset, cmdConfigFolderPrefix.reportVersion.get).publish.path assert(publishPathFolderPrefix === s"$hdfsPublishPath/$folderPrefix/$infoDateColumn=$reportDate/$infoVersionColumn=$reportVersion") val publishPathPublishPathOverride = TestDynamicConformance .getPathConfig(cmdConfigPublishPathOverride, conformanceDataset, cmdConfigPublishPathOverride.reportVersion.get) - .publishPath + .publish.path assert(publishPathPublishPathOverride === hdfsPublishPathOverride) val publishPathPublishPathOverrideAndFolderPrefix = TestDynamicConformance .getPathConfig(cmdConfigPublishPathOverrideAndFolderPrefix, - conformanceDataset, cmdConfigPublishPathOverrideAndFolderPrefix.reportVersion.get).publishPath + conformanceDataset, cmdConfigPublishPathOverrideAndFolderPrefix.reportVersion.get).publish.path assert(publishPathPublishPathOverrideAndFolderPrefix === hdfsPublishPathOverride) } 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 ed20ba4c1..91d518a24 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 @@ -53,6 +53,8 @@ class StandardizationParserSuite extends AnyFunSuite with SparkTestBase { private val rawFormat = "parquet" private val folderPrefix = s"year=$year/month=$month/day=$day" + implicit val hadoopConf = spark.sparkContext.hadoopConfiguration + private object TestStandardization extends StandardizationExecution test("Test credentials file parsing "){ @@ -168,17 +170,17 @@ class StandardizationParserSuite extends AnyFunSuite with SparkTestBase { "--raw-format", rawFormat)) val publishPathNoFolderPrefix = TestStandardization.getPathConfig(cmdConfigNoFolderPrefix, standardiseDataset, - cmdConfigRawPathOverride.reportVersion.get).rawPath + cmdConfigRawPathOverride.reportVersion.get).raw.path assert(publishPathNoFolderPrefix === s"${standardiseDataset.hdfsPath}/${dateTokens(0)}/${dateTokens(1)}/${dateTokens(2)}/v${cmdConfigNoFolderPrefix.reportVersion.get}") val publishPathFolderPrefix = TestStandardization - .getPathConfig(cmdConfigFolderPrefix, standardiseDataset, cmdConfigRawPathOverride.reportVersion.get).rawPath + .getPathConfig(cmdConfigFolderPrefix, standardiseDataset, cmdConfigRawPathOverride.reportVersion.get).raw.path assert(publishPathFolderPrefix === s"${standardiseDataset.hdfsPath}/$folderPrefix/${dateTokens(0)}/${dateTokens(1)}/${dateTokens(2)}/v${cmdConfigFolderPrefix.reportVersion.get}") val publishPathRawPathOverride = TestStandardization - .getPathConfig(cmdConfigRawPathOverride, standardiseDataset, cmdConfigRawPathOverride.reportVersion.get).rawPath + .getPathConfig(cmdConfigRawPathOverride, standardiseDataset, cmdConfigRawPathOverride.reportVersion.get).raw.path assert(publishPathRawPathOverride === hdfsRawPathOverride) val publishPathRawPathOverrideAndFolderPrefix = TestStandardization .getPathConfig(cmdConfigRawPathOverrideAndFolderPrefix, standardiseDataset, - cmdConfigRawPathOverride.reportVersion.get).rawPath + cmdConfigRawPathOverride.reportVersion.get).raw.path assert(publishPathRawPathOverrideAndFolderPrefix === hdfsRawPathOverride) } From c88bd35db83759899b2aea0d8cc24d143c17c4b3 Mon Sep 17 00:00:00 2001 From: Daniel Kavan Date: Fri, 6 Nov 2020 10:48:02 +0100 Subject: [PATCH 09/12] PathConfigEntry -> PathWithFs + moved to utils to stand be used for PerformanceMetricTools.addJobInfoToAtumMetadata etc. --- .../enceladus/common/CommonJobExecution.scala | 6 +-- .../enceladus/common/config/PathConfig.scala | 21 +++------ .../conformance/ConformanceExecution.scala | 15 ++++--- .../StandardizationExecution.scala | 15 ++++--- ...andardizationAndConformanceExecution.scala | 7 +-- .../enceladus/utils/config/PathWithFs.scala | 28 ++++++++++++ .../performance/PerformanceMetricTools.scala | 44 +++++++++++-------- 7 files changed, 83 insertions(+), 53 deletions(-) create mode 100644 utils/src/main/scala/za/co/absa/enceladus/utils/config/PathWithFs.scala 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 8fa7dfb6d..29a01596c 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 @@ -27,7 +27,7 @@ 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.{JobConfigParser, PathConfig, PathConfigEntry} +import za.co.absa.enceladus.common.config.{JobConfigParser, PathConfig} import za.co.absa.enceladus.common.plugin.PostProcessingService import za.co.absa.enceladus.common.plugin.menas.{MenasPlugin, MenasRunUrl} import za.co.absa.enceladus.common.version.SparkVersionGuard @@ -35,7 +35,7 @@ import za.co.absa.enceladus.dao.MenasDAO import za.co.absa.enceladus.dao.rest.MenasConnectionStringParser import za.co.absa.enceladus.model.Dataset import za.co.absa.enceladus.plugins.builtin.errorsender.params.ErrorSenderPluginParams -import za.co.absa.enceladus.utils.config.{ConfigReader, SecureConfig} +import za.co.absa.enceladus.utils.config.{ConfigReader, PathWithFs, SecureConfig} import za.co.absa.enceladus.utils.fs.FileSystemUtils import za.co.absa.enceladus.utils.fs.FileSystemUtils.FileSystemExt import za.co.absa.enceladus.utils.general.ProjectMetadataTools @@ -106,7 +106,7 @@ trait CommonJobExecution { protected def validateOutputPath(pathConfig: PathConfig): Unit - protected def validateIfPathAlreadyExists(entry: PathConfigEntry): Unit = { + protected def validateIfPathAlreadyExists(entry: PathWithFs): Unit = { val fsUtils = entry.fileSystem.toFsUtils if (fsUtils.exists(entry.path)) { throw new IllegalStateException( diff --git a/spark-jobs/src/main/scala/za/co/absa/enceladus/common/config/PathConfig.scala b/spark-jobs/src/main/scala/za/co/absa/enceladus/common/config/PathConfig.scala index 1a381df38..4abed2f8b 100644 --- a/spark-jobs/src/main/scala/za/co/absa/enceladus/common/config/PathConfig.scala +++ b/spark-jobs/src/main/scala/za/co/absa/enceladus/common/config/PathConfig.scala @@ -16,8 +16,7 @@ package za.co.absa.enceladus.common.config import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.FileSystem -import za.co.absa.enceladus.utils.fs.FileSystemUtils +import za.co.absa.enceladus.utils.config.PathWithFs /** * @@ -26,23 +25,17 @@ import za.co.absa.enceladus.utils.fs.FileSystemUtils * @param standardization In case of StandardizationJob and ConformanceJob it should be None and for * StandardizationConformanceJob it should represent the intermediate standardization path */ -case class PathConfig(raw: PathConfigEntry, - publish: PathConfigEntry, - standardization: PathConfigEntry) +case class PathConfig(raw: PathWithFs, + publish: PathWithFs, + standardization: PathWithFs) object PathConfig { def fromPaths(rawPath: String, publishPath: String, standardizationPath: String) (implicit hadoopConf: Configuration): PathConfig = PathConfig( - PathConfigEntry.fromPath(rawPath), - PathConfigEntry.fromPath(publishPath), - PathConfigEntry.fromPath(standardizationPath) + PathWithFs.fromPath(rawPath), + PathWithFs.fromPath(publishPath), + PathWithFs.fromPath(standardizationPath) ) } -case class PathConfigEntry(path: String, fileSystem: FileSystem) - -object PathConfigEntry { - def fromPath(path: String)(implicit hadoopConf: Configuration): PathConfigEntry = - PathConfigEntry(path, FileSystemUtils.getFileSystemFromPath(path)) -} 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 720d87f55..912f2d926 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 @@ -25,7 +25,7 @@ import za.co.absa.atum.AtumImplicits.DataSetWrapper import za.co.absa.atum.core.Atum import za.co.absa.enceladus.common.Constants.{InfoDateColumn, InfoDateColumnString, InfoVersionColumn, ReportDateFormat} import za.co.absa.enceladus.common.RecordIdGeneration._ -import za.co.absa.enceladus.common.config.{JobConfigParser, PathConfig, PathConfigEntry} +import za.co.absa.enceladus.common.config.{JobConfigParser, PathConfig} import za.co.absa.enceladus.common.plugin.menas.MenasPlugin import za.co.absa.enceladus.common.{CommonJobExecution, Constants, RecordIdGeneration} import za.co.absa.enceladus.conformance.config.{ConformanceConfig, ConformanceConfigParser} @@ -35,6 +35,7 @@ import za.co.absa.enceladus.dao.MenasDAO import za.co.absa.enceladus.dao.auth.MenasCredentials import za.co.absa.enceladus.model.Dataset import za.co.absa.enceladus.standardization_conformance.config.StandardizationConformanceConfig +import za.co.absa.enceladus.utils.config.PathWithFs import za.co.absa.enceladus.utils.fs.FileSystemUtils.FileSystemExt import za.co.absa.enceladus.utils.implicits.DataFrameImplicits.DataFrameEnhancements import za.co.absa.enceladus.utils.modules.SourcePhase @@ -90,7 +91,7 @@ trait ConformanceExecution extends CommonJobExecution { val initialConfig = super.getPathConfig(cmd, dataset, reportVersion) cmd.asInstanceOf[ConformanceConfig].publishPathOverride match { case None => initialConfig - case Some(providedPublishPath) => initialConfig.copy(publish = PathConfigEntry.fromPath(providedPublishPath)) + case Some(providedPublishPath) => initialConfig.copy(publish = PathWithFs.fromPath(providedPublishPath)) } } @@ -143,10 +144,10 @@ trait ConformanceExecution extends CommonJobExecution { PerformanceMetricTools.addJobInfoToAtumMetadata( "conform", - preparationResult.pathCfg.standardization.path, + preparationResult.pathCfg.standardization, preparationResult.pathCfg.publish.path, menasCredentials.username, cmdLineArgs - )(spark, stdFs.toFsUtils) + ) val withPartCols = result .withColumnIfDoesNotExist(InfoDateColumn, to_date(lit(cmd.reportDate), ReportDateFormat)) @@ -168,10 +169,10 @@ trait ConformanceExecution extends CommonJobExecution { PerformanceMetricTools.addPerformanceMetricsToAtumMetadata( spark, "conform", - preparationResult.pathCfg.standardization.path, - preparationResult.pathCfg.publish.path, + preparationResult.pathCfg.standardization, + preparationResult.pathCfg.publish, menasCredentials.username, cmdLineArgs - )(stdFs.toFsUtils, publishFs.toFsUtils) + ) withPartCols.writeInfoFile(preparationResult.pathCfg.publish.path)(publishFs) writePerformanceMetrics(preparationResult.performance, cmd) 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 43f6ac554..42e5b6577 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 @@ -25,7 +25,7 @@ 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.common.RecordIdGeneration.getRecordIdGenerationStrategyFromConfig -import za.co.absa.enceladus.common.config.{JobConfigParser, PathConfig, PathConfigEntry} +import za.co.absa.enceladus.common.config.{JobConfigParser, PathConfig} import za.co.absa.enceladus.common.plugin.menas.MenasPlugin import za.co.absa.enceladus.common.{CommonJobExecution, Constants} import za.co.absa.enceladus.dao.MenasDAO @@ -34,6 +34,7 @@ import za.co.absa.enceladus.model.Dataset import za.co.absa.enceladus.standardization.config.{StandardizationConfig, StandardizationConfigParser} import za.co.absa.enceladus.standardization.interpreter.StandardizationInterpreter import za.co.absa.enceladus.standardization.interpreter.stages.PlainSchemaGenerator +import za.co.absa.enceladus.utils.config.PathWithFs import za.co.absa.enceladus.utils.fs.FileSystemUtils.FileSystemExt import za.co.absa.enceladus.utils.fs.{DistributedFsUtils, HadoopFsUtils} import za.co.absa.enceladus.utils.modules.SourcePhase @@ -88,9 +89,9 @@ trait StandardizationExecution extends CommonJobExecution { Atum.setAdditionalInfo("raw_format" -> cmd.rawFormat) PerformanceMetricTools.addJobInfoToAtumMetadata("std", - preparationResult.pathCfg.raw.path, + preparationResult.pathCfg.raw, preparationResult.pathCfg.standardization.path, - menasCredentials.username, args.mkString(" "))(spark, rawFsUtils) + menasCredentials.username, args.mkString(" ")) dao.getSchema(preparationResult.dataset.schemaName, preparationResult.dataset.schemaVersion) } @@ -100,7 +101,7 @@ trait StandardizationExecution extends CommonJobExecution { val initialConfig = super.getPathConfig(cmd, dataset, reportVersion) cmd.asInstanceOf[StandardizationConfig].rawPathOverride match { case None => initialConfig - case Some(providedRawPath) => initialConfig.copy(raw = PathConfigEntry.fromPath(providedRawPath)) + case Some(providedRawPath) => initialConfig.copy(raw = PathWithFs.fromPath(providedRawPath)) } } @@ -203,11 +204,11 @@ trait StandardizationExecution extends CommonJobExecution { PerformanceMetricTools.addPerformanceMetricsToAtumMetadata( spark, "std", - preparationResult.pathCfg.raw.path, - preparationResult.pathCfg.standardization.path, + preparationResult.pathCfg.raw, + preparationResult.pathCfg.standardization, menasCredentials.username, args.mkString(" ") - )(rawFs.toFsUtils, stdFs.toFsUtils) + ) cmd.rowTag.foreach(rowTag => Atum.setAdditionalInfo("xml_row_tag" -> rowTag)) cmd.csvDelimiter.foreach(delimiter => Atum.setAdditionalInfo("csv_delimiter" -> delimiter)) diff --git a/spark-jobs/src/main/scala/za/co/absa/enceladus/standardization_conformance/StandardizationAndConformanceExecution.scala b/spark-jobs/src/main/scala/za/co/absa/enceladus/standardization_conformance/StandardizationAndConformanceExecution.scala index e5eb73073..47a9bc617 100644 --- a/spark-jobs/src/main/scala/za/co/absa/enceladus/standardization_conformance/StandardizationAndConformanceExecution.scala +++ b/spark-jobs/src/main/scala/za/co/absa/enceladus/standardization_conformance/StandardizationAndConformanceExecution.scala @@ -17,11 +17,12 @@ package za.co.absa.enceladus.standardization_conformance import org.apache.hadoop.conf.Configuration import za.co.absa.enceladus.common.CommonJobExecution -import za.co.absa.enceladus.common.config.{JobConfigParser, PathConfig, PathConfigEntry} +import za.co.absa.enceladus.common.config.{JobConfigParser, PathConfig} import za.co.absa.enceladus.conformance.ConformanceExecution import za.co.absa.enceladus.model.Dataset import za.co.absa.enceladus.standardization.StandardizationExecution import za.co.absa.enceladus.standardization_conformance.config.StandardizationConformanceConfig +import za.co.absa.enceladus.utils.config.PathWithFs trait StandardizationAndConformanceExecution extends StandardizationExecution with ConformanceExecution @@ -34,8 +35,8 @@ trait StandardizationAndConformanceExecution extends StandardizationExecution val rawPathOverride = jobCmd.rawPathOverride val publishPathOverride = jobCmd.publishPathOverride defaultConfig.copy( - raw = PathConfigEntry.fromPath(rawPathOverride.getOrElse(defaultConfig.raw.path)), - publish = PathConfigEntry.fromPath(publishPathOverride.getOrElse(defaultConfig.publish.path)) + raw = PathWithFs.fromPath(rawPathOverride.getOrElse(defaultConfig.raw.path)), + publish = PathWithFs.fromPath(publishPathOverride.getOrElse(defaultConfig.publish.path)) ) } diff --git a/utils/src/main/scala/za/co/absa/enceladus/utils/config/PathWithFs.scala b/utils/src/main/scala/za/co/absa/enceladus/utils/config/PathWithFs.scala new file mode 100644 index 000000000..0a1139c0a --- /dev/null +++ b/utils/src/main/scala/za/co/absa/enceladus/utils/config/PathWithFs.scala @@ -0,0 +1,28 @@ +/* + * 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.config + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.FileSystem +import za.co.absa.enceladus.utils.fs.FileSystemUtils + +case class PathWithFs(path: String, fileSystem: FileSystem) + +object PathWithFs { + def fromPath(path: String)(implicit hadoopConf: Configuration): PathWithFs = + PathWithFs(path, FileSystemUtils.getFileSystemFromPath(path)) +} diff --git a/utils/src/main/scala/za/co/absa/enceladus/utils/performance/PerformanceMetricTools.scala b/utils/src/main/scala/za/co/absa/enceladus/utils/performance/PerformanceMetricTools.scala index 3e8a18f78..bef903391 100644 --- a/utils/src/main/scala/za/co/absa/enceladus/utils/performance/PerformanceMetricTools.scala +++ b/utils/src/main/scala/za/co/absa/enceladus/utils/performance/PerformanceMetricTools.scala @@ -19,8 +19,9 @@ import org.apache.spark.sql.SparkSession import org.apache.spark.sql.functions.{col, size, sum} import org.slf4j.{Logger, LoggerFactory} import za.co.absa.atum.core.Atum +import za.co.absa.enceladus.utils.config.PathWithFs import za.co.absa.enceladus.utils.error.ErrorMessage -import za.co.absa.enceladus.utils.fs.HadoopFsUtils +import za.co.absa.enceladus.utils.fs.FileSystemUtils.FileSystemExt import za.co.absa.enceladus.utils.general.ProjectMetadataTools import za.co.absa.enceladus.utils.schema.SchemaUtils @@ -35,26 +36,28 @@ object PerformanceMetricTools { * * @param spark A Spark Session * @param optionPrefix A prefix for all performance metrics options, e.g. 'std' for Standardization and 'conform' for Conformance - * @param inputPath A path to an input directory of the job + * @param input A path+fs to an input directory of the job * @param outputPath A path to an output directory of the job * @param loginUserName A login user name who performed the job * **/ def addJobInfoToAtumMetadata(optionPrefix: String, - inputPath: String, - outputPath: String, - loginUserName: String, - cmdLineArgs: String) - (implicit spark: SparkSession, inputFsUtils: HadoopFsUtils): Unit = { + input: PathWithFs, + outputPath: String, + loginUserName: String, + cmdLineArgs: String) + (implicit spark: SparkSession): Unit = { // Spark job configuration val sc = spark.sparkContext // The number of executors minus the driver val numberOfExecutors = sc.getExecutorMemoryStatus.keys.size - 1 + val inputFsUtils = input.fileSystem.toFsUtils + // Directory sizes and size ratio - val inputDirSize = inputFsUtils.getDirectorySize(inputPath) - val inputDataSize = inputFsUtils.getDirectorySizeNoHidden(inputPath) + val inputDirSize = inputFsUtils.getDirectorySize(input.path) + val inputDataSize = inputFsUtils.getDirectorySizeNoHidden(input.path) addSparkConfig(optionPrefix, "spark.driver.memory", "driver_memory") addSparkConfig(optionPrefix, "spark.driver.cores", "driver_cores") @@ -68,7 +71,7 @@ object PerformanceMetricTools { sc.applicationAttemptId .foreach(attemptId => Atum.setAdditionalInfo(s"${optionPrefix}_spark_attempt_id" -> attemptId)) Atum.setAdditionalInfo(s"${optionPrefix}_cmd_line_args" -> cmdLineArgs) - Atum.setAdditionalInfo(s"${optionPrefix}_input_dir" -> inputPath) + Atum.setAdditionalInfo(s"${optionPrefix}_input_dir" -> input.path) Atum.setAdditionalInfo(s"${optionPrefix}_output_dir" -> outputPath) Atum.setAdditionalInfo(s"${optionPrefix}_input_dir_size" -> inputDirSize.toString) Atum.setAdditionalInfo(s"${optionPrefix}_input_data_size" -> inputDataSize.toString) @@ -84,25 +87,28 @@ object PerformanceMetricTools { * * @param spark A Spark Session * @param optionPrefix A prefix for all performance metrics options, e.g. 'std' for Standardization and 'conform' for Conformance - * @param inputPath A path to an input directory of the job - * @param outputPath A path to an output directory of the job + * @param input A path to an input directory of the job + * @param output A path to an output directory of the job * @param loginUserName A login user name who performed the job * **/ def addPerformanceMetricsToAtumMetadata(spark: SparkSession, optionPrefix: String, - inputPath: String, - outputPath: String, + input: PathWithFs, + output: PathWithFs, loginUserName: String, cmdLineArgs: String - )(inputFsUtils: HadoopFsUtils, outputFsUtils: HadoopFsUtils): Unit = { + ): Unit = { + + val inputFsUtils = input.fileSystem.toFsUtils + val outputFsUtils = output.fileSystem.toFsUtils // Directory sizes and size ratio - val inputDirSize = inputFsUtils.getDirectorySize(inputPath) - val outputDirSize = outputFsUtils.getDirectorySize(outputPath) - val outputDataSize = outputFsUtils.getDirectorySizeNoHidden(outputPath) + val inputDirSize = inputFsUtils.getDirectorySize(input.path) + val outputDirSize = outputFsUtils.getDirectorySize(output.path) + val outputDataSize = outputFsUtils.getDirectorySizeNoHidden(output.path) - val (numRecordsFailed, numRecordsSuccessful, numOfErrors) = getNumberOfErrors(spark, outputPath) + val (numRecordsFailed, numRecordsSuccessful, numOfErrors) = getNumberOfErrors(spark, output.path) calculateSizeRatio(inputDirSize, outputDataSize, numRecordsFailed + numRecordsSuccessful) .foreach(ratio => { From 768dac3ce061d76d9b8beda1d37b4f35703f0f75 Mon Sep 17 00:00:00 2001 From: Daniel Kavan Date: Fri, 6 Nov 2020 10:59:02 +0100 Subject: [PATCH 10/12] readStandardizationInputData(... path: String)(implicit ... fs: FileSystem) -> readStandardizationInputData(input: PathWithFs) --- .../StandardizationExecution.scala | 16 +++++++--------- .../standardization/StandardizationJob.scala | 3 +-- .../StandardizationAndConformanceJob.scala | 3 +-- 3 files changed, 9 insertions(+), 13 deletions(-) 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 42e5b6577..5515f8063 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 @@ -57,8 +57,6 @@ trait StandardizationExecution extends CommonJobExecution { val rawFs = preparationResult.pathCfg.raw.fileSystem val rawFsUtils = rawFs.toFsUtils - val stdFs = preparationResult.pathCfg.standardization.fileSystem - val stdDirSize = rawFsUtils.getDirectorySize(preparationResult.pathCfg.raw.path) preparationResult.performance.startMeasurement(stdDirSize) @@ -112,10 +110,9 @@ trait StandardizationExecution extends CommonJobExecution { protected def readStandardizationInputData[T](schema: StructType, cmd: StandardizationConfigParser[T], - path: String, + rawInput: PathWithFs, dataset: Dataset) (implicit spark: SparkSession, - rawFs: FileSystem, dao: MenasDAO): DataFrame = { val numberOfColumns = schema.fields.length val standardizationReader = new StandardizationPropertiesProvider() @@ -127,9 +124,9 @@ trait StandardizationExecution extends CommonJobExecution { val inputSchema = PlainSchemaGenerator.generateInputSchema(schema, optColumnNameOfCorruptRecord) dfReaderConfigured.schema(inputSchema) } - val dfWithSchema = readerWithOptSchema.load(s"$path/*") + val dfWithSchema = readerWithOptSchema.load(s"${rawInput.path}/*") - ensureSplittable(dfWithSchema, path, schema)(spark, rawFs.toFsUtils) + ensureSplittable(dfWithSchema, rawInput, schema) } private def getColumnNameOfCorruptRecord[R](schema: StructType, cmd: StandardizationConfigParser[R]) @@ -222,9 +219,10 @@ trait StandardizationExecution extends CommonJobExecution { //scalastyle:off parameter.number - private def ensureSplittable(df: DataFrame, path: String, schema: StructType) - (implicit spark: SparkSession, fsUtils: DistributedFsUtils): DataFrame = { - if (fsUtils.isNonSplittable(path)) { + private def ensureSplittable(df: DataFrame, input: PathWithFs, schema: StructType) + (implicit spark: SparkSession): DataFrame = { + implicit val fsUtils = input.fileSystem.toFsUtils + if (fsUtils.isNonSplittable(input.path)) { convertToSplittable(df, schema) } else { df diff --git a/spark-jobs/src/main/scala/za/co/absa/enceladus/standardization/StandardizationJob.scala b/spark-jobs/src/main/scala/za/co/absa/enceladus/standardization/StandardizationJob.scala index f091aadda..13af430d2 100644 --- a/spark-jobs/src/main/scala/za/co/absa/enceladus/standardization/StandardizationJob.scala +++ b/spark-jobs/src/main/scala/za/co/absa/enceladus/standardization/StandardizationJob.scala @@ -39,8 +39,7 @@ object StandardizationJob extends StandardizationExecution { val preparationResult = prepareJob() val schema = prepareStandardization(args, menasCredentials, preparationResult) - implicit val rawFs: FileSystem = preparationResult.pathCfg.raw.fileSystem - val inputData = readStandardizationInputData(schema, cmd, preparationResult.pathCfg.raw.path, preparationResult.dataset) + val inputData = readStandardizationInputData(schema, cmd, preparationResult.pathCfg.raw, preparationResult.dataset) try { val result = standardize(inputData, schema, cmd) diff --git a/spark-jobs/src/main/scala/za/co/absa/enceladus/standardization_conformance/StandardizationAndConformanceJob.scala b/spark-jobs/src/main/scala/za/co/absa/enceladus/standardization_conformance/StandardizationAndConformanceJob.scala index 06bd70449..a5689a36a 100644 --- a/spark-jobs/src/main/scala/za/co/absa/enceladus/standardization_conformance/StandardizationAndConformanceJob.scala +++ b/spark-jobs/src/main/scala/za/co/absa/enceladus/standardization_conformance/StandardizationAndConformanceJob.scala @@ -36,8 +36,7 @@ object StandardizationAndConformanceJob extends StandardizationAndConformanceExe val preparationResult = prepareJob() val schema = prepareStandardization(args, menasCredentials, preparationResult) - implicit val rawFs = preparationResult.pathCfg.raw.fileSystem - val inputData = readStandardizationInputData(schema, cmd, preparationResult.pathCfg.raw.path, preparationResult.dataset) + val inputData = readStandardizationInputData(schema, cmd, preparationResult.pathCfg.raw, preparationResult.dataset) try { val standardized = standardize(inputData, schema, cmd) From 912678ca82f83950b25f93da4ebe2cbae17e9daf Mon Sep 17 00:00:00 2001 From: Daniel Kavan Date: Fri, 6 Nov 2020 13:22:48 +0100 Subject: [PATCH 11/12] Infamous `FileSystem.toFsUtils` is now replaced by `HadoopFsUtils.getOrCreate(fs: FileSystem)` - hadoopFsUtils can only be created this way (from the outside PoV) which is to force cache control. --- .../enceladus/dao/auth/MenasCredentials.scala | 7 +- .../enceladus/common/CommonJobExecution.scala | 14 ++-- .../conformance/ConformanceExecution.scala | 8 +-- .../conformance/HyperConformance.scala | 4 +- .../interpreter/DynamicInterpreter.scala | 4 +- .../StandardizationExecution.scala | 8 +-- .../enceladus/utils/fs/FileSystemUtils.scala | 18 ----- .../enceladus/utils/fs/HadoopFsUtils.scala | 71 ++++++++++++------- .../performance/PerformanceMetricTools.scala | 8 +-- .../utils/testUtils/HadoopFsTestBase.scala | 3 +- 10 files changed, 72 insertions(+), 73 deletions(-) diff --git a/dao/src/main/scala/za/co/absa/enceladus/dao/auth/MenasCredentials.scala b/dao/src/main/scala/za/co/absa/enceladus/dao/auth/MenasCredentials.scala index 21afec390..5fa24a3a2 100644 --- a/dao/src/main/scala/za/co/absa/enceladus/dao/auth/MenasCredentials.scala +++ b/dao/src/main/scala/za/co/absa/enceladus/dao/auth/MenasCredentials.scala @@ -18,8 +18,7 @@ package za.co.absa.enceladus.dao.auth import com.typesafe.config.ConfigFactory import org.apache.spark.sql.SparkSession import sun.security.krb5.internal.ktab.KeyTab -import za.co.absa.enceladus.utils.fs.FileSystemUtils -import za.co.absa.enceladus.utils.fs.FileSystemUtils.FileSystemExt +import za.co.absa.enceladus.utils.fs.{FileSystemUtils, HadoopFsUtils} sealed abstract class MenasCredentials { val username: String @@ -43,7 +42,7 @@ object MenasPlainCredentials { def fromFile(path: String)(implicit spark: SparkSession): MenasPlainCredentials = { val fs = FileSystemUtils.getFileSystemFromPath(path)(spark.sparkContext.hadoopConfiguration) - val conf = ConfigFactory.parseString(fs.toFsUtils.getLocalOrDistributedFileContent(path)) + val conf = ConfigFactory.parseString(HadoopFsUtils.getOrCreate(fs).getLocalOrDistributedFileContent(path)) MenasPlainCredentials(conf.getString("username"), conf.getString("password")) } } @@ -58,7 +57,7 @@ object MenasKerberosCredentials { def fromFile(path: String)(implicit spark: SparkSession): MenasKerberosCredentials = { val fs = FileSystemUtils.getFileSystemFromPath(path)(spark.sparkContext.hadoopConfiguration) - val localKeyTabPath = fs.toFsUtils.getLocalPathToFileOrCopyToLocal(path) + val localKeyTabPath = HadoopFsUtils.getOrCreate(fs).getLocalPathToFileOrCopyToLocal(path) val keytab = KeyTab.getInstance(localKeyTabPath) val username = keytab.getOneName.getName 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 29a01596c..a434e423d 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 @@ -36,8 +36,7 @@ import za.co.absa.enceladus.dao.rest.MenasConnectionStringParser import za.co.absa.enceladus.model.Dataset import za.co.absa.enceladus.plugins.builtin.errorsender.params.ErrorSenderPluginParams import za.co.absa.enceladus.utils.config.{ConfigReader, PathWithFs, SecureConfig} -import za.co.absa.enceladus.utils.fs.FileSystemUtils -import za.co.absa.enceladus.utils.fs.FileSystemUtils.FileSystemExt +import za.co.absa.enceladus.utils.fs.{FileSystemUtils, HadoopFsUtils} import za.co.absa.enceladus.utils.general.ProjectMetadataTools import za.co.absa.enceladus.utils.modules.SourcePhase import za.co.absa.enceladus.utils.modules.SourcePhase.Standardization @@ -107,7 +106,7 @@ trait CommonJobExecution { protected def validateOutputPath(pathConfig: PathConfig): Unit protected def validateIfPathAlreadyExists(entry: PathWithFs): Unit = { - val fsUtils = entry.fileSystem.toFsUtils + val fsUtils = HadoopFsUtils.getOrCreate(entry.fileSystem) if (fsUtils.exists(entry.path)) { throw new IllegalStateException( s"Path ${entry.path} already exists. Increment the run version, or delete ${entry.path}" @@ -245,11 +244,12 @@ trait CommonJobExecution { jobConfig.reportVersion match { case Some(version) => version case None => - import FileSystemUtils.FileSystemExt - // publishFs for this specific feature (needed for missing reportVersion until reusable common "PathConfig" with FS objects is established) - implicit val tempPublishFs: FileSystem = FileSystemUtils.getFileSystemFromPath(dataset.hdfsPublishPath) - val newVersion = tempPublishFs.toFsUtils.getLatestVersion(dataset.hdfsPublishPath, jobConfig.reportDate) + 1 + // publishFs for this specific feature (needed for missing reportVersion until reusable + // common "PathConfig" with FS objects is established) + val tempPublishFs: FileSystem = FileSystemUtils.getFileSystemFromPath(dataset.hdfsPublishPath) + val fsUtils = HadoopFsUtils.getOrCreate(tempPublishFs) + val newVersion = fsUtils.getLatestVersion(dataset.hdfsPublishPath, jobConfig.reportDate) + 1 log.warn(s"Report version not provided, inferred report version: $newVersion") log.warn("This is an EXPERIMENTAL feature.") log.warn(" -> It can lead to issues when running multiple jobs on a dataset concurrently.") 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 912f2d926..86f83f260 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 @@ -36,7 +36,7 @@ import za.co.absa.enceladus.dao.auth.MenasCredentials import za.co.absa.enceladus.model.Dataset import za.co.absa.enceladus.standardization_conformance.config.StandardizationConformanceConfig import za.co.absa.enceladus.utils.config.PathWithFs -import za.co.absa.enceladus.utils.fs.FileSystemUtils.FileSystemExt +import za.co.absa.enceladus.utils.fs.HadoopFsUtils import za.co.absa.enceladus.utils.implicits.DataFrameImplicits.DataFrameEnhancements import za.co.absa.enceladus.utils.modules.SourcePhase import za.co.absa.enceladus.utils.performance.PerformanceMetricTools @@ -54,7 +54,7 @@ trait ConformanceExecution extends CommonJobExecution { cmd: ConformanceConfigParser[T], spark: SparkSession): Unit = { - val stdFsUtils = preparationResult.pathCfg.standardization.fileSystem.toFsUtils + val stdFsUtils = HadoopFsUtils.getOrCreate(preparationResult.pathCfg.standardization.fileSystem) val stdDirSize = stdFsUtils.getDirectorySize(preparationResult.pathCfg.standardization.path) preparationResult.performance.startMeasurement(stdDirSize) @@ -164,7 +164,7 @@ trait ConformanceExecution extends CommonJobExecution { withPartCols.write.parquet(preparationResult.pathCfg.publish.path) - val publishDirSize = publishFs.toFsUtils.getDirectorySize(preparationResult.pathCfg.publish.path) + val publishDirSize = HadoopFsUtils.getOrCreate(publishFs).getDirectorySize(preparationResult.pathCfg.publish.path) preparationResult.performance.finishMeasurement(publishDirSize, recordCount) PerformanceMetricTools.addPerformanceMetricsToAtumMetadata( spark, @@ -178,7 +178,7 @@ trait ConformanceExecution extends CommonJobExecution { writePerformanceMetrics(preparationResult.performance, cmd) if (conformanceReader.isAutocleanStdFolderEnabled()) { - stdFs.toFsUtils.deleteDirectoryRecursively(preparationResult.pathCfg.standardization.path) + HadoopFsUtils.getOrCreate(stdFs).deleteDirectoryRecursively(preparationResult.pathCfg.standardization.path) } log.info(s"$sourceId finished successfully") } 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 41ca2aeb5..68526d7e4 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 @@ -33,7 +33,7 @@ import za.co.absa.enceladus.dao.MenasDAO import za.co.absa.enceladus.dao.auth.{MenasCredentialsFactory, MenasKerberosCredentialsFactory, MenasPlainCredentialsFactory} import za.co.absa.enceladus.dao.rest.{MenasConnectionStringParser, RestDaoFactory} import za.co.absa.enceladus.model.Dataset -import za.co.absa.enceladus.utils.fs.FileSystemUtils.FileSystemExt +import za.co.absa.enceladus.utils.fs.HadoopFsUtils import za.co.absa.hyperdrive.ingestor.api.transformer.{StreamTransformer, StreamTransformerFactory} class HyperConformance (implicit cmd: ConformanceConfig, @@ -68,7 +68,7 @@ class HyperConformance (implicit cmd: ConformanceConfig, // using HDFS implementation until HyperConformance is S3-ready implicit val hdfs = FileSystem.get(sparkSession.sparkContext.hadoopConfiguration) - implicit val hdfsUtils = hdfs.toFsUtils + implicit val hdfsUtils = HadoopFsUtils.getOrCreate(hdfs) val conformedDf = DynamicInterpreter().interpret(conformance, rawDf) .withColumnIfDoesNotExist(InfoDateColumn, coalesce(infoDateColumn, current_date())) 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 6897fd6b0..2ef75fdbd 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 @@ -32,10 +32,10 @@ import za.co.absa.enceladus.model.conformanceRule.{ConformanceRule, _} import za.co.absa.enceladus.model.{Dataset => ConfDataset} 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.enceladus.utils.fs.FileSystemUtils.FileSystemExt case class DynamicInterpreter(implicit inputFs: FileSystem) { private val log = LoggerFactory.getLogger(this.getClass) @@ -271,7 +271,7 @@ case class DynamicInterpreter(implicit inputFs: FileSystem) { val mappingTableDef = ictx.dao.getMappingTable(rule.mappingTable, rule.mappingTableVersion) val mappingTablePath = PartitioningUtils.getPartitionedPathName(mappingTableDef.hdfsPath, ictx.progArgs.reportDate) - val mappingTableSize = inputFs.toFsUtils.getDirectorySizeNoHidden(mappingTablePath) + val mappingTableSize = HadoopFsUtils.getOrCreate(inputFs).getDirectorySizeNoHidden(mappingTablePath) (mappingTableSize / (1024 * 1024)).toInt } 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 5515f8063..8fe54196f 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 @@ -19,7 +19,6 @@ import java.io.{PrintWriter, StringWriter} import java.util.UUID import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.FileSystem import org.apache.spark.sql.types.{StructField, StructType} import org.apache.spark.sql.{Column, DataFrame, SparkSession} import za.co.absa.atum.AtumImplicits @@ -35,7 +34,6 @@ import za.co.absa.enceladus.standardization.config.{StandardizationConfig, Stand import za.co.absa.enceladus.standardization.interpreter.StandardizationInterpreter import za.co.absa.enceladus.standardization.interpreter.stages.PlainSchemaGenerator import za.co.absa.enceladus.utils.config.PathWithFs -import za.co.absa.enceladus.utils.fs.FileSystemUtils.FileSystemExt import za.co.absa.enceladus.utils.fs.{DistributedFsUtils, HadoopFsUtils} import za.co.absa.enceladus.utils.modules.SourcePhase import za.co.absa.enceladus.utils.performance.PerformanceMetricTools @@ -55,7 +53,7 @@ trait StandardizationExecution extends CommonJobExecution { cmd: StandardizationConfigParser[T], spark: SparkSession): StructType = { val rawFs = preparationResult.pathCfg.raw.fileSystem - val rawFsUtils = rawFs.toFsUtils + val rawFsUtils = HadoopFsUtils.getOrCreate(rawFs) val stdDirSize = rawFsUtils.getDirectorySize(preparationResult.pathCfg.raw.path) preparationResult.performance.startMeasurement(stdDirSize) @@ -195,7 +193,7 @@ trait StandardizationExecution extends CommonJobExecution { // Store performance metrics // (record count, directory sizes, elapsed time, etc. to _INFO file metadata and performance file) - val stdDirSize = stdFs.toFsUtils.getDirectorySize(preparationResult.pathCfg.standardization.path) + val stdDirSize = HadoopFsUtils.getOrCreate(stdFs).getDirectorySize(preparationResult.pathCfg.standardization.path) preparationResult.performance.finishMeasurement(stdDirSize, recordCount) PerformanceMetricTools.addPerformanceMetricsToAtumMetadata( @@ -221,7 +219,7 @@ trait StandardizationExecution extends CommonJobExecution { private def ensureSplittable(df: DataFrame, input: PathWithFs, schema: StructType) (implicit spark: SparkSession): DataFrame = { - implicit val fsUtils = input.fileSystem.toFsUtils + implicit val fsUtils = HadoopFsUtils.getOrCreate(input.fileSystem) if (fsUtils.isNonSplittable(input.path)) { convertToSplittable(df, schema) } else { diff --git a/utils/src/main/scala/za/co/absa/enceladus/utils/fs/FileSystemUtils.scala b/utils/src/main/scala/za/co/absa/enceladus/utils/fs/FileSystemUtils.scala index d97671c53..7cce26313 100644 --- a/utils/src/main/scala/za/co/absa/enceladus/utils/fs/FileSystemUtils.scala +++ b/utils/src/main/scala/za/co/absa/enceladus/utils/fs/FileSystemUtils.scala @@ -23,8 +23,6 @@ import org.apache.hadoop.fs.FileSystem import org.slf4j.{Logger, LoggerFactory} import za.co.absa.atum.utils.S3Utils.StringS3LocationExt -import scala.collection.concurrent.TrieMap - object FileSystemUtils { val log: Logger = LoggerFactory.getLogger(this.getClass) @@ -48,22 +46,6 @@ object FileSystemUtils { } } - private val fsUtilsCache = TrieMap[FileSystem, HadoopFsUtils]() - - implicit class FileSystemExt(fs: FileSystem) { - /** - * Given the FileSystem object `fs` that this method is called on, - * the appropriate HadoopFsUtils is either newly created or returned form cache. - * @return - */ - def toFsUtils: HadoopFsUtils = { - fsUtilsCache.getOrElseUpdate(fs, { - log.debug(s"reusing cached fsUtils for FS ${fs.getUri} / ${fs.toString}") - new HadoopFsUtils()(fs)} - - ) - } - } } diff --git a/utils/src/main/scala/za/co/absa/enceladus/utils/fs/HadoopFsUtils.scala b/utils/src/main/scala/za/co/absa/enceladus/utils/fs/HadoopFsUtils.scala index 8b6a55899..4dd7de05a 100644 --- a/utils/src/main/scala/za/co/absa/enceladus/utils/fs/HadoopFsUtils.scala +++ b/utils/src/main/scala/za/co/absa/enceladus/utils/fs/HadoopFsUtils.scala @@ -21,14 +21,36 @@ import java.net.ConnectException import org.apache.commons.io.FileUtils import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.log4j.LogManager +import za.co.absa.enceladus.utils.fs.FileSystemUtils.log +import scala.collection.concurrent.TrieMap import scala.util.Try +object HadoopFsUtils { + private val fsUtilsCache = TrieMap[FileSystem, HadoopFsUtils]() + + /** + * Given the FileSystem object `fs`, an appropriate HadoopFsUtils is either + * newly created or returned form cache. + * + * @return cached [[HadoopFsUtils]] instance + */ + def getOrCreate(fs: FileSystem): HadoopFsUtils = { + fsUtilsCache.getOrElseUpdate(fs, { + log.debug(s"reusing cached fsUtils for FS ${fs.getUri} / ${fs.toString}") + new HadoopFsUtils()(fs) + }) + + } +} + /** * A set of functions to help with the date partitioning and version control + * + * This class has a private constructor - to achieve instance cache control - use + * [[za.co.absa.enceladus.utils.fs.HadoopFsUtils#getOrCreate(org.apache.hadoop.fs.FileSystem)]] */ - -class HadoopFsUtils()(implicit fs: FileSystem) extends DistributedFsUtils { +class HadoopFsUtils private()(implicit fs: FileSystem) extends DistributedFsUtils { private val log = LogManager.getLogger("enceladus.utils.fs.HadoopFsUtils") @@ -71,7 +93,6 @@ class HadoopFsUtils()(implicit fs: FileSystem) extends DistributedFsUtils { } - /** * Check if a given path exists on HDFS */ @@ -99,7 +120,7 @@ class HadoopFsUtils()(implicit fs: FileSystem) extends DistributedFsUtils { exists(path) } catch { case e: IllegalArgumentException => false - case e: ConnectException => false + case e: ConnectException => false } if (hdfs) { log.debug(s"HDFS file $path exists") @@ -111,12 +132,12 @@ class HadoopFsUtils()(implicit fs: FileSystem) extends DistributedFsUtils { } /** - * Checks if a file is located on HDFS or the local file system. - * If the file is in HDFS, it is copied to a temporary location. - * - * @param path A path to a file. Can be either local or HDFS location. - * @return A path to a file in the local filesystem. - */ + * Checks if a file is located on HDFS or the local file system. + * If the file is in HDFS, it is copied to a temporary location. + * + * @param path A path to a file. Can be either local or HDFS location. + * @return A path to a file in the local filesystem. + */ @throws[FileNotFoundException] def getLocalPathToFileOrCopyToLocal(path: String): String = { val absolutePath = LocalFsUtils.replaceHome(path) @@ -130,12 +151,12 @@ class HadoopFsUtils()(implicit fs: FileSystem) extends DistributedFsUtils { } /** - * Reads a file fully and returns its content. - * The file can be either in a HDFS or in a local file system. - * - * @param path A path to a file. Can be either local or HDFS location. - * @return The file's content. - */ + * Reads a file fully and returns its content. + * The file can be either in a HDFS or in a local file system. + * + * @param path A path to a file. Can be either local or HDFS location. + * @return The file's content. + */ @throws[FileNotFoundException] def getLocalOrDistributedFileContent(path: String): String = { val absolutePath = LocalFsUtils.replaceHome(path) @@ -182,11 +203,11 @@ class HadoopFsUtils()(implicit fs: FileSystem) extends DistributedFsUtils { } /** - * Returns directory size in bytes, skipping hidden files and directories (starting from '_' or '.'). - * - * @param path A path to a directory or a file. - * @return Directory size in bytes - */ + * Returns directory size in bytes, skipping hidden files and directories (starting from '_' or '.'). + * + * @param path A path to a directory or a file. + * @return Directory size in bytes + */ def getDirectorySizeNoHidden(path: String): Long = { def getDirSizeHelper(f: Path): Long = { var totalLength = 0L @@ -235,8 +256,8 @@ class HadoopFsUtils()(implicit fs: FileSystem) extends DistributedFsUtils { } /** - * Deletes a HDFS directory and all its contents recursively - */ + * Deletes a HDFS directory and all its contents recursively + */ def deleteDirectoryRecursively(path: String): Unit = { log.info(s"Deleting '$path' recursively...") val hdfsPath = new Path(path) @@ -255,7 +276,7 @@ class HadoopFsUtils()(implicit fs: FileSystem) extends DistributedFsUtils { * Finds the latest version given a publish folder on HDFS * * @param publishPath The HDFS path to the publish folder containing versions - * @param reportDate The string representation of the report date used to infer the latest version + * @param reportDate The string representation of the report date used to infer the latest version * @return the latest version or 0 in case no versions exist */ def getLatestVersion(publishPath: String, reportDate: String): Int = { @@ -267,7 +288,7 @@ class HadoopFsUtils()(implicit fs: FileSystem) extends DistributedFsUtils { val versions = files.filter(_.isDirectory()).map({ file => file.getPath.getName.replace("enceladus_info_version=", "").toInt }) - if(versions.isEmpty) 0 else versions.max + if (versions.isEmpty) 0 else versions.max case None => 0 } } diff --git a/utils/src/main/scala/za/co/absa/enceladus/utils/performance/PerformanceMetricTools.scala b/utils/src/main/scala/za/co/absa/enceladus/utils/performance/PerformanceMetricTools.scala index bef903391..21eddcf50 100644 --- a/utils/src/main/scala/za/co/absa/enceladus/utils/performance/PerformanceMetricTools.scala +++ b/utils/src/main/scala/za/co/absa/enceladus/utils/performance/PerformanceMetricTools.scala @@ -21,7 +21,7 @@ import org.slf4j.{Logger, LoggerFactory} import za.co.absa.atum.core.Atum import za.co.absa.enceladus.utils.config.PathWithFs import za.co.absa.enceladus.utils.error.ErrorMessage -import za.co.absa.enceladus.utils.fs.FileSystemUtils.FileSystemExt +import za.co.absa.enceladus.utils.fs.HadoopFsUtils import za.co.absa.enceladus.utils.general.ProjectMetadataTools import za.co.absa.enceladus.utils.schema.SchemaUtils @@ -53,7 +53,7 @@ object PerformanceMetricTools { // The number of executors minus the driver val numberOfExecutors = sc.getExecutorMemoryStatus.keys.size - 1 - val inputFsUtils = input.fileSystem.toFsUtils + val inputFsUtils = HadoopFsUtils.getOrCreate(input.fileSystem) // Directory sizes and size ratio val inputDirSize = inputFsUtils.getDirectorySize(input.path) @@ -100,8 +100,8 @@ object PerformanceMetricTools { cmdLineArgs: String ): Unit = { - val inputFsUtils = input.fileSystem.toFsUtils - val outputFsUtils = output.fileSystem.toFsUtils + val inputFsUtils = HadoopFsUtils.getOrCreate(input.fileSystem) + val outputFsUtils = HadoopFsUtils.getOrCreate(output.fileSystem) // Directory sizes and size ratio val inputDirSize = inputFsUtils.getDirectorySize(input.path) diff --git a/utils/src/main/scala/za/co/absa/enceladus/utils/testUtils/HadoopFsTestBase.scala b/utils/src/main/scala/za/co/absa/enceladus/utils/testUtils/HadoopFsTestBase.scala index 041001d97..7805bb92e 100644 --- a/utils/src/main/scala/za/co/absa/enceladus/utils/testUtils/HadoopFsTestBase.scala +++ b/utils/src/main/scala/za/co/absa/enceladus/utils/testUtils/HadoopFsTestBase.scala @@ -17,13 +17,12 @@ package za.co.absa.enceladus.utils.testUtils import org.apache.hadoop.fs.FileSystem -import za.co.absa.enceladus.utils.fs.FileSystemUtils.FileSystemExt import za.co.absa.enceladus.utils.fs.HadoopFsUtils trait HadoopFsTestBase extends HasSparkSession { implicit val fs: FileSystem = FileSystem.get(spark.sparkContext.hadoopConfiguration) - implicit val fsUtils: HadoopFsUtils = fs.toFsUtils + implicit val fsUtils: HadoopFsUtils = HadoopFsUtils.getOrCreate(fs) } From 1e36265dca58fbffa108b4e486ba61bf0dca67e1 Mon Sep 17 00:00:00 2001 From: Daniel Kavan Date: Mon, 9 Nov 2020 09:59:48 +0100 Subject: [PATCH 12/12] PR touchup: convertToSplittable's fsUtils is now a regular paramter (non-implicit) --- .../standardization/StandardizationExecution.scala | 11 +++++------ .../standardization/StandardizationJob.scala | 2 -- 2 files changed, 5 insertions(+), 8 deletions(-) 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 8fe54196f..05ed2d478 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 @@ -219,18 +219,17 @@ trait StandardizationExecution extends CommonJobExecution { private def ensureSplittable(df: DataFrame, input: PathWithFs, schema: StructType) (implicit spark: SparkSession): DataFrame = { - implicit val fsUtils = HadoopFsUtils.getOrCreate(input.fileSystem) + val fsUtils = HadoopFsUtils.getOrCreate(input.fileSystem) if (fsUtils.isNonSplittable(input.path)) { - convertToSplittable(df, schema) + convertToSplittable(df, schema, fsUtils) } else { df } } - private def convertToSplittable(df: DataFrame, schema: StructType) - (implicit spark: SparkSession, fsUtils: DistributedFsUtils): DataFrame = { + private def convertToSplittable(df: DataFrame, schema: StructType, fsUtils: DistributedFsUtils) + (implicit spark: SparkSession): DataFrame = { log.warn("Dataset is stored in a non-splittable format. This can have a severe performance impact.") - fsUtils match { case utils: HadoopFsUtils => val tempParquetDir = s"/tmp/nonsplittable-to-parquet-${UUID.randomUUID()}" @@ -250,7 +249,7 @@ trait StandardizationExecution extends CommonJobExecution { }: _*) case utils => - log.warn(s"Splittability conversion only available for HDFS, leaving as is for ${utils.getClass.getName}") + log.warn(s"Splittability conversion only available for 'HadoopFsUtils', leaving as is for ${utils.getClass.getName}") df } diff --git a/spark-jobs/src/main/scala/za/co/absa/enceladus/standardization/StandardizationJob.scala b/spark-jobs/src/main/scala/za/co/absa/enceladus/standardization/StandardizationJob.scala index 13af430d2..de8a1b2e9 100644 --- a/spark-jobs/src/main/scala/za/co/absa/enceladus/standardization/StandardizationJob.scala +++ b/spark-jobs/src/main/scala/za/co/absa/enceladus/standardization/StandardizationJob.scala @@ -15,12 +15,10 @@ package za.co.absa.enceladus.standardization -import org.apache.hadoop.fs.FileSystem import org.apache.spark.sql.SparkSession import za.co.absa.enceladus.dao.MenasDAO import za.co.absa.enceladus.dao.rest.RestDaoFactory import za.co.absa.enceladus.standardization.config.StandardizationConfig -import za.co.absa.enceladus.utils.fs.DistributedFsUtils import za.co.absa.enceladus.utils.modules.SourcePhase import za.co.absa.enceladus.utils.udf.UDFLibrary