diff --git a/build.sbt b/build.sbt index 3085bf92e..f2c2e63cb 100644 --- a/build.sbt +++ b/build.sbt @@ -2,7 +2,7 @@ name := "overwatch" organization := "com.databricks.labs" -version := "0.8.1.2" +version := "0.8.2.0" scalaVersion := "2.12.12" scalacOptions ++= Seq("-Xmax-classfile-name", "78") @@ -10,6 +10,9 @@ scalacOptions ++= Seq("-Xmax-classfile-name", "78") Test / fork := true Test / envVars := Map("OVERWATCH_ENV" -> " ","OVERWATCH_TOKEN" -> " ","OVERWATCH" -> " ") +logBuffered in Test := false +// parallelExecution in Test := false + val sparkVersion = "3.1.2" libraryDependencies += "org.apache.spark" %% "spark-core" % sparkVersion % Provided libraryDependencies += "org.apache.spark" %% "spark-sql" % sparkVersion % Provided @@ -18,6 +21,7 @@ libraryDependencies += "com.databricks" % "dbutils-api_2.12" % "0.0.5" % Provide libraryDependencies += "com.amazonaws" % "aws-java-sdk-s3" % "1.11.595" % Provided libraryDependencies += "io.delta" % "delta-core_2.12" % "1.0.0" % Provided libraryDependencies += "org.scalaj" %% "scalaj-http" % "2.4.2" +libraryDependencies += "com.lihaoyi" %% "sourcecode" % "0.4.1" //libraryDependencies += "org.apache.hive" % "hive-metastore" % "2.3.9" @@ -51,4 +55,4 @@ assemblyMergeStrategy in assembly := { case PathList("META-INF", xs @ _*) => MergeStrategy.discard case x => MergeStrategy.first } - assembly / assemblyOption := (assembly / assemblyOption).value.copy(includeScala = false) \ No newline at end of file + assembly / assemblyOption := (assembly / assemblyOption).value.copy(includeScala = false) diff --git a/src/main/resources/Warehouse_DBU_Details.csv b/src/main/resources/Warehouse_DBU_Details.csv new file mode 100644 index 000000000..9939a9a7e --- /dev/null +++ b/src/main/resources/Warehouse_DBU_Details.csv @@ -0,0 +1,19 @@ +cloud,cluster_size,driver_size,worker_count,total_dbus +AWS,2X-Small,i3.2xlarge,1,4 +AWS,X-Small,i3.2xlarge,2,6 +AWS,Small,i3.4xlarge,4,12 +AWS,Medium,i3.8xlarge,8,24 +AWS,Large,i3.8xlarge,16,40 +AWS,X-Large,i3.16xlarge,32,80 +AWS,2X-Large,i3.16xlarge,64,144 +AWS,3X-Large,i3.16xlarge,128,272 +AWS,4X-Large,i3.16xlarge,256,528 +AZURE,2X-Small,E8ds v4,1,4 +AZURE,X-Small,E8ds v4,2,6 +AZURE,Small,E16ds v4,4,12 +AZURE,Medium,E32ds v4,8,24 +AZURE,Large,E32ds v4,16,40 +AZURE,X-Large,E64ds v4,32,80 +AZURE,2X-Large,E64ds v4,64,144 +AZURE,3X-Large,E64ds v4,128,272 +AZURE,4X-Large,E64ds v4,256,528 \ No newline at end of file diff --git a/src/main/scala/com/databricks/labs/overwatch/MultiWorkspaceDeployment.scala b/src/main/scala/com/databricks/labs/overwatch/MultiWorkspaceDeployment.scala index 24c5de6a1..9913d1b22 100644 --- a/src/main/scala/com/databricks/labs/overwatch/MultiWorkspaceDeployment.scala +++ b/src/main/scala/com/databricks/labs/overwatch/MultiWorkspaceDeployment.scala @@ -122,7 +122,7 @@ class MultiWorkspaceDeployment extends SparkSessionWrapper { val sqlComputerDBUPrice: Double = config.sql_compute_dbu_price val jobsLightDBUPrice: Double = config.jobs_light_dbu_price val customWorkspaceName: String = config.workspace_name - val standardScopes = "audit,sparkEvents,jobs,clusters,clusterEvents,notebooks,pools,accounts,dbsql,notebookCommands".split(",") + val standardScopes = OverwatchScope.toArray val scopesToExecute = (standardScopes.map(_.toLowerCase).toSet -- config.excluded_scopes.getOrElse("").split(":").map(_.toLowerCase).toSet).toArray diff --git a/src/main/scala/com/databricks/labs/overwatch/api/ApiMeta.scala b/src/main/scala/com/databricks/labs/overwatch/api/ApiMeta.scala index 1944144eb..827a5ee10 100644 --- a/src/main/scala/com/databricks/labs/overwatch/api/ApiMeta.scala +++ b/src/main/scala/com/databricks/labs/overwatch/api/ApiMeta.scala @@ -200,6 +200,32 @@ trait ApiMeta { jsonObject.toString } + /** + * Function will add the meta info to the api response. + * + * @param response + * @param jsonQuery + * @param queryMap + * @return a string containing the api response and the meta for the api call. + */ + private[overwatch] def enrichAPIResponse(response: HttpResponse[String], jsonQuery: String, queryMap: Map[String, String]): String = { + val filter: String = if (apiCallType.equals("POST")) jsonQuery else { + val mapper = new ObjectMapper() + mapper.registerModule(DefaultScalaModule) + mapper.writeValueAsString(queryMap) + } + val jsonObject = new JSONObject(); + val apiTraceabilityMeta = new JSONObject(); + apiTraceabilityMeta.put("endPoint", apiName) + apiTraceabilityMeta.put("type", apiCallType) + apiTraceabilityMeta.put("apiVersion", apiV) + apiTraceabilityMeta.put("responseCode", response.code) + apiTraceabilityMeta.put("batchKeyFilter", filter) + jsonObject.put("rawResponse", response.body.trim) + jsonObject.put("apiTraceabilityMeta", apiTraceabilityMeta) + jsonObject.toString + } + } /** diff --git a/src/main/scala/com/databricks/labs/overwatch/env/Workspace.scala b/src/main/scala/com/databricks/labs/overwatch/env/Workspace.scala index 5bf767505..5188ba0ef 100644 --- a/src/main/scala/com/databricks/labs/overwatch/env/Workspace.scala +++ b/src/main/scala/com/databricks/labs/overwatch/env/Workspace.scala @@ -17,6 +17,8 @@ import scala.concurrent.Future import scala.concurrent.forkjoin.ForkJoinPool import scala.util.{Failure, Success, Try} import scala.concurrent.ExecutionContext.Implicits.global +import java.time.LocalDateTime +import java.time.format.DateTimeFormatter /** @@ -422,6 +424,40 @@ class Workspace(config: Config) extends SparkSessionWrapper { addReport } + /** + * Fetch the warehouse event data from system.compute.warehouse_events + * @param fromTime : from time to fetch the data + * @param untilTime: until time to fetch the data + * @param maxHistoryDays: maximum history days to fetch the data + * @return + */ + def getWarehousesEventDF(fromTime: TimeTypes, + untilTime: TimeTypes, + config: Config, + maxHistoryDays: Int = 30 + ): DataFrame = { + val sysTableFormat = DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss.SSS") + val moduleFromTime = fromTime.asLocalDateTime.format(sysTableFormat) + val moduleUntilTime = untilTime.asLocalDateTime.format(sysTableFormat) + val useSystemTableMessage = "Use system tables as a source to audit logs" + val tableDoesNotExistsMessage = "Table system.compute.warehouse_events does not exists" + + if(config.auditLogConfig.systemTableName.isEmpty) + throw new NoNewDataException(useSystemTableMessage, Level.WARN, allowModuleProgression = false) + + if(!spark.catalog.tableExists("system.compute.warehouse_events")) + throw new NoNewDataException(tableDoesNotExistsMessage, Level.WARN, allowModuleProgression = false) + + spark.sql(s""" + select * from system.compute.warehouse_events + WHERE workspace_id = '${config.organizationId}' + and event_time >= DATE_SUB('${moduleFromTime}', ${maxHistoryDays}) + and event_time <= '${moduleUntilTime}' + """) + .withColumnRenamed("event_type","state") + .withColumnRenamed("workspace_id","organization_id") + .withColumnRenamed("event_time","timestamp") + } } diff --git a/src/main/scala/com/databricks/labs/overwatch/pipeline/Bronze.scala b/src/main/scala/com/databricks/labs/overwatch/pipeline/Bronze.scala index 4f7e6f704..446d55649 100644 --- a/src/main/scala/com/databricks/labs/overwatch/pipeline/Bronze.scala +++ b/src/main/scala/com/databricks/labs/overwatch/pipeline/Bronze.scala @@ -104,8 +104,6 @@ class Bronze(_workspace: Workspace, _database: Database, _config: Config) } - - lazy private[overwatch] val jobsSnapshotModule = Module(1001, "Bronze_Jobs_Snapshot", this) lazy private val appendJobsProcess: () => ETLDefinition = { () => @@ -171,6 +169,7 @@ class Bronze(_workspace: Workspace, _database: Database, _config: Config) BronzeTargets.clustersSnapshotTarget.asDF, Seq( prepClusterEventLogs( + clusterEventLogsModule.isFirstRun, BronzeTargets.auditLogsTarget.asIncrementalDF(clusterEventLogsModule, BronzeTargets.auditLogsTarget.incrementalColumns, additionalLagDays = 1), // 1 lag day to get laggard records clusterEventLogsModule.fromTime, clusterEventLogsModule.untilTime, diff --git a/src/main/scala/com/databricks/labs/overwatch/pipeline/BronzeTransforms.scala b/src/main/scala/com/databricks/labs/overwatch/pipeline/BronzeTransforms.scala index fca983e93..aa9c68481 100644 --- a/src/main/scala/com/databricks/labs/overwatch/pipeline/BronzeTransforms.scala +++ b/src/main/scala/com/databricks/labs/overwatch/pipeline/BronzeTransforms.scala @@ -410,12 +410,21 @@ trait BronzeTransforms extends SparkSessionWrapper { pipelineSnapTime: Long, tmpClusterEventsSuccessPath: String, tmpClusterEventsErrorPath: String, - config: Config) = { + config: Config, + isFirstRun: Boolean) = { val finalResponseCount = clusterIDs.length val clusterEventsEndpoint = "clusters/events" val lagTime = 86400000 //1 day - val lagStartTime = startTime.asUnixTimeMilli - lagTime + + val lagStartTime = if (isFirstRun) { + logger.log(Level.INFO, "First run, acquiring all cluster events") + 0.toLong + } else { + logger.log(Level.INFO, "Subsequent run, acquiring new cluster events") + startTime.asUnixTimeMilli - lagTime + } + // creating Json input for parallel API calls val jsonInput = Map( "start_value" -> "0", @@ -601,6 +610,7 @@ trait BronzeTransforms extends SparkSessionWrapper { } protected def prepClusterEventLogs( + isFirstRun : Boolean, filteredAuditLogDF: DataFrame, startTime: TimeTypes, endTime: TimeTypes, @@ -626,10 +636,15 @@ trait BronzeTransforms extends SparkSessionWrapper { val tmpClusterEventsSuccessPath = s"${config.tempWorkingDir}/${apiEndpointTempDir}/success_" + pipelineSnapTS.asUnixTimeMilli val tmpClusterEventsErrorPath = s"${config.tempWorkingDir}/${apiEndpointTempDir}/error_" + pipelineSnapTS.asUnixTimeMilli - try{ - landClusterEvents(clusterIDs, startTime, endTime, pipelineSnapTS.asUnixTimeMilli, tmpClusterEventsSuccessPath, - tmpClusterEventsErrorPath, config) - }catch { + try { + landClusterEvents( + clusterIDs, startTime, endTime, + pipelineSnapTS.asUnixTimeMilli, + tmpClusterEventsSuccessPath, + tmpClusterEventsErrorPath, + config, + isFirstRun) + } catch { case e: Throwable => val errMsg = s"Error in landing cluster events: ${e.getMessage}" logger.log(Level.ERROR, errMsg) @@ -637,7 +652,7 @@ trait BronzeTransforms extends SparkSessionWrapper { } if (Helpers.pathExists(tmpClusterEventsErrorPath)) { persistErrors( - deriveRawApiResponseDF(spark.read.json(tmpClusterEventsErrorPath)) + spark.read.json(tmpClusterEventsErrorPath) .withColumn("from_ts", toTS(col("from_epoch"))) .withColumn("until_ts", toTS(col("until_epoch"))), database, @@ -1375,4 +1390,4 @@ trait BronzeTransforms extends SparkSessionWrapper { } -} \ No newline at end of file +} diff --git a/src/main/scala/com/databricks/labs/overwatch/pipeline/ETLDefinition.scala b/src/main/scala/com/databricks/labs/overwatch/pipeline/ETLDefinition.scala index 403ea6bf7..abe7b9c8a 100644 --- a/src/main/scala/com/databricks/labs/overwatch/pipeline/ETLDefinition.scala +++ b/src/main/scala/com/databricks/labs/overwatch/pipeline/ETLDefinition.scala @@ -27,7 +27,20 @@ class ETLDefinition( val transformedDF = transforms.foldLeft(verifiedSourceDF) { case (df, transform) => - df.transform(transform) + + /* + * reverting Spark UI Job Group labels for now + * + * TODO: enumerate the regressions this would introduce + * when the labels set by then platform are replaced + * this way. + * df.sparkSession.sparkContext.setJobGroup( + * s"${module.pipeline.config.workspaceName}:${module.moduleName}", + * transform.toString) + */ + + df.transform( transform) + } write(transformedDF, module) } diff --git a/src/main/scala/com/databricks/labs/overwatch/pipeline/Gold.scala b/src/main/scala/com/databricks/labs/overwatch/pipeline/Gold.scala index 08d5c6aca..f094bbea5 100644 --- a/src/main/scala/com/databricks/labs/overwatch/pipeline/Gold.scala +++ b/src/main/scala/com/databricks/labs/overwatch/pipeline/Gold.scala @@ -320,6 +320,24 @@ class Gold(_workspace: Workspace, _database: Database, _config: Config) ) } + lazy private[overwatch] val warehouseStateFactModule = Module(3020, "Gold_WarehouseStateFact", this, Array(2022, 2021), 3.0) + lazy private val appendWarehouseStateFactProcess: () => ETLDefinition = { + () => + ETLDefinition( + SilverTargets.warehousesStateDetailTarget.asIncrementalDF( + warehouseStateFactModule, + SilverTargets.warehousesStateDetailTarget.incrementalColumns, + GoldTargets.warehouseStateFactTarget.maxMergeScanDates + ), + Seq(buildWarehouseStateFact( + BronzeTargets.cloudMachineDetail, + BronzeTargets.warehouseDbuDetail, + SilverTargets.warehousesSpecTarget + )), + append(GoldTargets.warehouseStateFactTarget) + ) + } + private def processSparkEvents(): Unit = { sparkExecutorModule.execute(appendSparkExecutorProcess) @@ -400,6 +418,10 @@ class Gold(_workspace: Workspace, _database: Database, _config: Config) notebookCommandsFactModule.execute(appendNotebookCommandsFactProcess) GoldTargets.notebookCommandsFactViewTarget.publish(notebookCommandsFactViewColumnMapping) } + case OverwatchScope.warehouseEvents => { + warehouseStateFactModule.execute(appendWarehouseStateFactProcess) + GoldTargets.warehouseStateFactViewTarget.publish(warehouseStateFactViewColumnMappings) + } case _ => } } diff --git a/src/main/scala/com/databricks/labs/overwatch/pipeline/GoldTransforms.scala b/src/main/scala/com/databricks/labs/overwatch/pipeline/GoldTransforms.scala index 3f41ab076..9605e2706 100644 --- a/src/main/scala/com/databricks/labs/overwatch/pipeline/GoldTransforms.scala +++ b/src/main/scala/com/databricks/labs/overwatch/pipeline/GoldTransforms.scala @@ -6,6 +6,7 @@ import com.databricks.labs.overwatch.utils.{ModuleDisabled, NoNewDataException, import org.apache.log4j.{Level, Logger} import org.apache.spark.sql.expressions.Window import org.apache.spark.sql.functions._ +import org.apache.spark.sql.types.DateType import org.apache.spark.sql.{Column, DataFrame} trait GoldTransforms extends SparkSessionWrapper { @@ -505,6 +506,7 @@ trait GoldTransforms extends SparkSessionWrapper { val auditDF_base = auditIncrementalDF .filter(col("serviceName") === "notebook" && col("actionName") === "runCommand") + .verifyMinimumSchema(Schema.auditMasterSchema) .selectExpr("*", "requestParams.*").drop("requestParams") if (auditDF_base.columns.contains("executionTime")){ @@ -950,6 +952,87 @@ trait GoldTransforms extends SparkSessionWrapper { df.select(warehouseCols: _*) } + protected def buildWarehouseStateFact( + instanceDetailsTarget: PipelineTable, + warehouseDbuDetailsTarget: PipelineTable, + warehouseSpec: PipelineTable + )(warehouseStateDetail: DataFrame): DataFrame = { + val warehouseDbuDetails = warehouseDbuDetailsTarget.asDF() + + val warehouseDbuDetailsColumns = warehouseDbuDetails.columns.map(colName => col(s"warehouseDbuDetails.$colName")) + + val warehouseClustersDetails = warehouseDbuDetails.alias("warehouseDbuDetails") + .join(instanceDetailsTarget.asDF.alias("instanceDetails"), + $"warehouseDbuDetails.organization_id" === $"instanceDetails.organization_id" && + $"warehouseDbuDetails.driver_size" === $"instanceDetails.API_Name") + .select(warehouseDbuDetailsColumns :+ col("instanceDetails.vCPUs"): _*) + + val warehousePotMetaToFill = Array( + "warehouse_name", "channel", "warehouse_type","cluster_size","driver_size" + ) + val warehousePotKeys = Seq("organization_id", "warehouse_id") + val warehousePotIncrementals = Seq("state_start_date", "unixTimeMS_state_start") + + val warehousePotential = warehouseStateDetail + .withColumn("timestamp", 'unixTimeMS_state_start) // time control temporary column + .toTSDF("timestamp", "organization_id", "warehouse_id") + .lookupWhen( + warehouseSpec.asDF.toTSDF("timestamp", "organization_id", "warehouse_id"), + maxLookAhead = Window.unboundedFollowing,maxLookback = 0L, + tsPartitionVal = 4 + ).df + .alias("warehousePotential") + .join( // estimated node type details at start time of run. If contract changes during state, not handled but very infrequent and negligible impact + warehouseClustersDetails + .withColumnRenamed("organization_id","warehouse_organization_id") + .withColumnRenamed("workspace_name","warehouse_workspace_name") + .withColumnRenamed("cluster_size","warehouse_cluster_size") + .withColumnRenamed("Pipeline_SnapTS","warehouse_Pipeline_SnapTS") + .withColumnRenamed("Overwatch_RunID","warehouse_Overwatch_RunID") + .alias("warehouseClustersDetails"), + $"warehousePotential.organization_id" === $"warehouseClustersDetails.warehouse_organization_id" && + trim(lower($"warehousePotential.cluster_size")) === trim(lower($"warehouseClustersDetails.warehouse_cluster_size")) + && + $"warehousePotential.unixTimeMS_state_start" + .between($"warehouseClustersDetails.activeFromEpochMillis", $"warehouseClustersDetails.activeUntilEpochMillis"), + "left" + ).drop("warehouse_organization_id","warehouse_cluster_size","warehouse_Pipeline_SnapTS") + .fillMeta(warehousePotMetaToFill, warehousePotKeys, warehousePotIncrementals, noiseBuckets = getTotalCores) + + val workerPotentialCoreS = when('databricks_billable, 'vCPUs * 'current_num_clusters * 'uptime_in_state_S) + .otherwise(lit(0)) + val warehouseDBUs = when('databricks_billable, 'total_dbus * 'uptime_in_state_H ).otherwise(lit(0)).alias("driver_dbus") + + val warehouseStateFactCols: Array[Column] = Array( + 'organization_id, + 'warehouse_id, + 'warehouse_name, + 'tags, + 'state_start_date, + 'unixTimeMS_state_start, + 'unixTimeMS_state_end, + 'timestamp_state_start, + 'timestamp_state_end, + 'state, + 'cluster_size, + 'current_num_clusters, + 'target_num_clusters, + 'uptime_since_restart_S, + 'uptime_in_state_S, + 'uptime_in_state_H, + 'cloud_billable, + 'databricks_billable, + 'warehouse_type, + 'state_dates, + 'days_in_state, + (workerPotentialCoreS / lit(3600)).alias("worker_potential_core_H"), + warehouseDBUs + ) + + warehousePotential + .select(warehouseStateFactCols: _*) + } + protected val clusterViewColumnMapping: String = """ |organization_id, workspace_name, cluster_id, action, unixTimeMS, timestamp, date, cluster_name, driver_node_type, @@ -1092,4 +1175,12 @@ trait GoldTransforms extends SparkSessionWrapper { |num_active_sessions,jdbc_url,unixTimeMS,date,created_by |""".stripMargin + protected val warehouseStateFactViewColumnMappings: String = + """ + |organization_id,workspace_name,warehouse_id,warehouse_name,unixTimeMS_state_start, tags, state_start_date, + |unixTimeMS_state_end, timestamp_state_start, timestamp_state_end, state, cluster_size, current_num_clusters, + |target_num_clusters, uptime_since_restart_S, uptime_in_state_S, uptime_in_state_H, cloud_billable, + |databricks_billable, warehouse_type, state_dates, days_in_state, worker_potential_core_H, driver_dbus + |""".stripMargin + } diff --git a/src/main/scala/com/databricks/labs/overwatch/pipeline/InitializerFunctions.scala b/src/main/scala/com/databricks/labs/overwatch/pipeline/InitializerFunctions.scala index 4c6ff8fb5..5d27594ae 100644 --- a/src/main/scala/com/databricks/labs/overwatch/pipeline/InitializerFunctions.scala +++ b/src/main/scala/com/databricks/labs/overwatch/pipeline/InitializerFunctions.scala @@ -221,6 +221,7 @@ trait InitializerFunctions case "accounts" => accounts case "dbsql" => dbsql case "notebookcommands" => notebookCommands + case "warehouseevents" => warehouseEvents case scope => { val supportedScopes = s"${OverwatchScope.values.mkString(", ")}, all" throw new BadConfigException(s"Scope $scope is not supported. Supported scopes include: " + diff --git a/src/main/scala/com/databricks/labs/overwatch/pipeline/Pipeline.scala b/src/main/scala/com/databricks/labs/overwatch/pipeline/Pipeline.scala index b85e1efbc..6c789a699 100644 --- a/src/main/scala/com/databricks/labs/overwatch/pipeline/Pipeline.scala +++ b/src/main/scala/com/databricks/labs/overwatch/pipeline/Pipeline.scala @@ -264,6 +264,7 @@ class Pipeline( protected def loadStaticDatasets(): this.type = { if (!BronzeTargets.cloudMachineDetail.exists(dataValidation = true)) publishInstanceDetails() if (!BronzeTargets.dbuCostDetail.exists(dataValidation = true)) publishDBUCostDetails() + if (!BronzeTargets.warehouseDbuDetail.exists(dataValidation = true)) publishWarehouseDbuDetails() else updateDBUCosts(BronzeTargets.dbuCostDetail) this } @@ -486,6 +487,27 @@ class Pipeline( ) } + private def publishWarehouseDbuDetails(): Unit = { + val logMsg = "warehouseDbuDetails does not exist and/or does not contain data for this workspace. BUILDING/APPENDING" + logger.log(Level.INFO, logMsg) + + if (getConfig.debugFlag) println(logMsg) + val warehouseDbuDetailsDF = Initializer.loadLocalCSVResource(spark, "/Warehouse_DBU_Details.csv") + .filter(lower('cloud) === s"${config.cloudProvider}".toLowerCase) + + val finalWarehouseDbuDetailsDF = warehouseDbuDetailsDF + .withColumn("organization_id", lit(config.organizationId)) + .withColumn("activeFrom", lit(primordialTime.asDTString).cast("date")) + .withColumn("activeUntil", lit(null).cast("date")) + .withColumn("activeFromEpochMillis", unix_timestamp('activeFrom) * 1000) + .withColumn("activeUntilEpochMillis", + coalesce(unix_timestamp('activeUntil) * 1000, unix_timestamp(pipelineSnapTime.asColumnTS) * 1000)) + .withColumn("isActive", lit(true)) + .coalesce(1) + + database.writeWithRetry(finalWarehouseDbuDetailsDF, BronzeTargets.warehouseDbuDetail, pipelineSnapTime.asColumnTS) + if (config.databaseName != config.consumerDatabaseName) BronzeTargets.warehouseDbuDetailViewTarget.publish("*") + } } object Pipeline { diff --git a/src/main/scala/com/databricks/labs/overwatch/pipeline/PipelineTable.scala b/src/main/scala/com/databricks/labs/overwatch/pipeline/PipelineTable.scala index 205bbebd2..1ee60e7e8 100644 --- a/src/main/scala/com/databricks/labs/overwatch/pipeline/PipelineTable.scala +++ b/src/main/scala/com/databricks/labs/overwatch/pipeline/PipelineTable.scala @@ -19,7 +19,7 @@ import org.apache.spark.sql.{AnalysisException, Column, DataFrame} // Perhaps add the strategy into the Rule definition in the Rules Engine case class PipelineTable( name: String, - private val _keys: Array[String], + private[overwatch] val _keys: Array[String], config: Config, incrementalColumns: Array[String] = Array(), format: String = "delta", // TODO -- Convert to Enum diff --git a/src/main/scala/com/databricks/labs/overwatch/pipeline/PipelineTargets.scala b/src/main/scala/com/databricks/labs/overwatch/pipeline/PipelineTargets.scala index cf9793bad..39f3adb16 100644 --- a/src/main/scala/com/databricks/labs/overwatch/pipeline/PipelineTargets.scala +++ b/src/main/scala/com/databricks/labs/overwatch/pipeline/PipelineTargets.scala @@ -236,6 +236,21 @@ abstract class PipelineTargets(config: Config) { masterSchema = Some(Schema.warehouseSnapMinimumSchema) ) + lazy private[overwatch] val warehouseDbuDetail: PipelineTable = PipelineTable( + name = "warehouseDbuDetails", + _keys = Array("driver_size"), + config, + incrementalColumns = Array("Pipeline_SnapTS"), + partitionBy = Seq("organization_id"), + masterSchema = Some(Schema.warehouseDbuDetailsMinimumSchema) + ) + + lazy private[overwatch] val warehouseDbuDetailViewTarget: PipelineView = PipelineView( + name = "warehouseDbuDetails", + warehouseDbuDetail, + config + ) + } /** @@ -400,6 +415,16 @@ abstract class PipelineTargets(config: Config) { excludedReconColumn = Array("Timestamp") //Timestamp is the pipelineSnapTs in epoc ) + lazy private[overwatch] val warehousesStateDetailTarget: PipelineTable = PipelineTable( + name = "warehouse_state_detail_silver", + _keys = Array("warehouse_id", "state", "unixTimeMS_state_start"), + config, + _mode = WriteMode.merge, + incrementalColumns = Array("state_start_date", "unixTimeMS_state_start"), + partitionBy = Seq("organization_id", "state_start_date"), + maxMergeScanDates = 30, // 1 less than warehouseStateFact + ) + } object GoldTargets { @@ -697,6 +722,23 @@ abstract class PipelineTargets(config: Config) { config ) + lazy private[overwatch] val warehouseStateFactTarget: PipelineTable = PipelineTable( + name = "warehouseStateFact_gold", + _keys = Array("warehouse_id", "state", "unixTimeMS_state_start"), + config, + _mode = WriteMode.merge, + partitionBy = Seq("organization_id", "state_start_date", "__overwatch_ctrl_noise"), + maxMergeScanDates = 31, // 1 greater than clusterStateDetail + incrementalColumns = Array("state_start_date", "unixTimeMS_state_start"), + zOrderBy = Array("warehouse_id", "unixTimeMS_state_start") + ) + + lazy private[overwatch] val warehouseStateFactViewTarget: PipelineView = PipelineView( + name = "warehouseStateFact", + warehouseStateFactTarget, + config + ) + } diff --git a/src/main/scala/com/databricks/labs/overwatch/pipeline/PipelineView.scala b/src/main/scala/com/databricks/labs/overwatch/pipeline/PipelineView.scala index c9b068038..4d28f45f9 100644 --- a/src/main/scala/com/databricks/labs/overwatch/pipeline/PipelineView.scala +++ b/src/main/scala/com/databricks/labs/overwatch/pipeline/PipelineView.scala @@ -16,10 +16,37 @@ case class PipelineView(name: String, else dbTargetOverride.getOrElse(config.consumerDatabaseName) def publish(colDefinition: String, sorted: Boolean = false, reverse: Boolean = false,workspacesAllowed: Array[String] = Array()): Unit = { + + if (spark.catalog.tableExists(s"${config.etlCatalogName}.${config.databaseName}.${dataSource.name}")) { + createView(colDefinition, sorted, reverse,workspacesAllowed) + } else { + val dataSourceName = dataSource.name.toLowerCase() + val path = s"${config.etlDataPathPrefix}/${dataSourceName}" + if (Helpers.pathExists(path)) { + val query = s"CREATE TABLE ${config.etlCatalogName}.${config.databaseName}.${dataSource.name} USING DELTA LOCATION '${path}'" + val queryStatement = query.toString() + try { + setCurrentCatalog(spark, config.etlCatalogName) + spark.sql(queryStatement) + createView(colDefinition, sorted, reverse,workspacesAllowed) + } catch { + case e: Throwable => + println(s"GOLD VIEW: CREATE VIEW FAILED: Cannot create view: ${dataSource.tableFullName} --> ${e.getMessage}") + } + } else { + val msgLog = s"GOLD VIEW: CREATE VIEW FAILED: Source path: ${path} does not exist" + logger.log(Level.INFO, msgLog) + if (config.debugFlag) println(msgLog) + } + } + } + + private def createView(colDefinition: String, sorted: Boolean = false, reverse: Boolean = false, workspacesAllowed: Array[String] = Array()): Unit = { + if (dataSource.exists) { val pubStatementSB = new StringBuilder("create or replace view ") val dataSourceName = dataSource.name.toLowerCase() - pubStatementSB.append(s"${dbTarget}.${name} as select ${colDefinition} from delta.`${config.etlDataPathPrefix}/${dataSourceName}`") + pubStatementSB.append(s"${dbTarget}.${name} as select ${colDefinition} from ${config.etlCatalogName}.${config.databaseName}.${dataSource.name}") // link partition columns if (dataSource.partitionBy.nonEmpty) { val partMap: Map[String, String] = if (partitionMapOverrides.isEmpty) { @@ -27,10 +54,10 @@ case class PipelineView(name: String, } else { partitionMapOverrides } - pubStatementSB.append(s" where ${partMap.head._1} = ${s"delta.`${config.etlDataPathPrefix}/${dataSourceName}`"}.${partMap.head._2} ") + pubStatementSB.append(s" where ${partMap.head._1} = ${config.etlCatalogName}.${config.databaseName}.${dataSource.name}.${partMap.head._2} ") if (partMap.keys.toArray.length > 1) { partMap.tail.foreach(pCol => { - pubStatementSB.append(s"and ${pCol._1} = ${s"delta.`${config.etlDataPathPrefix}/${dataSourceName}`"}.${pCol._2} ") + pubStatementSB.append(s"and ${pCol._1} = ${config.etlCatalogName}.${config.databaseName}.${dataSource.name}.${pCol._2} ") }) } if (workspacesAllowed.nonEmpty){ diff --git a/src/main/scala/com/databricks/labs/overwatch/pipeline/Schema.scala b/src/main/scala/com/databricks/labs/overwatch/pipeline/Schema.scala index 416170c39..fa7577f06 100644 --- a/src/main/scala/com/databricks/labs/overwatch/pipeline/Schema.scala +++ b/src/main/scala/com/databricks/labs/overwatch/pipeline/Schema.scala @@ -222,6 +222,7 @@ object Schema extends SparkSessionWrapper { StructField("shardName", NullType, nullable = true), StructField("orgId", NullType, nullable = true), StructField("version", NullType, nullable = true), + StructField("accountId", NullType, nullable = true), //adding schema used for photon evolution StructField("effective_spark_version", StringType, nullable = true), StructField("runtime_engine", StringType, nullable = true), @@ -1144,4 +1145,16 @@ object Schema extends SparkSessionWrapper { StructField("source", StringType, nullable = false), StructField("workspace_id", StringType, nullable = false) )) + + val warehouseDbuDetailsMinimumSchema: StructType = StructType(Seq( + StructField("organization_id", StringType, nullable = false), + StructField("workspace_name", StringType, nullable = false), + StructField("cloud", StringType, nullable = false), + StructField("cluster_size", StringType, nullable = false), + StructField("driver_size", StringType, nullable = false), + StructField("worker_count", IntegerType, nullable = false), + StructField("total_dbus", IntegerType, nullable = false), + StructField("activeFrom", DateType, nullable = false), + StructField("activeUntil", DateType, nullable = true) + )) } diff --git a/src/main/scala/com/databricks/labs/overwatch/pipeline/Silver.scala b/src/main/scala/com/databricks/labs/overwatch/pipeline/Silver.scala index 6a99e0182..719b05923 100644 --- a/src/main/scala/com/databricks/labs/overwatch/pipeline/Silver.scala +++ b/src/main/scala/com/databricks/labs/overwatch/pipeline/Silver.scala @@ -30,7 +30,8 @@ class Silver(_workspace: Workspace, _database: Database, _config: Config) SilverTargets.dbJobsStatusTarget, SilverTargets.notebookStatusTarget, SilverTargets.sqlQueryHistoryTarget, - SilverTargets.warehousesSpecTarget + SilverTargets.warehousesSpecTarget, + SilverTargets.warehousesStateDetailTarget ) } @@ -57,6 +58,7 @@ class Silver(_workspace: Workspace, _database: Database, _config: Config) Array(sqlQueryHistoryModule, warehouseSpecModule) } + case OverwatchScope.warehouseEvents => Array(warehouseStateDetailModule) case _ => Array[Module]() } } @@ -268,7 +270,11 @@ class Silver(_workspace: Workspace, _database: Database, _config: Config) ) } - lazy private[overwatch] val jobRunsModule = Module(2011, "Silver_JobsRuns", this, Array(1004, 2010, 2014), shuffleFactor = 12.0) + lazy private[overwatch] val jobRunsModule = + Module( 2011, "Silver_JobsRuns", this, Array(1004, 2010, 2014), shuffleFactor = 12.0) + .withSparkOverrides( Map( + "spark.databricks.adaptive.autoOptimizeShuffle.enabled" -> "true")) + lazy private val appendJobRunsProcess: () => ETLDefinition = { () => ETLDefinition( @@ -398,6 +404,23 @@ class Silver(_workspace: Workspace, _database: Database, _config: Config) ) } + lazy private[overwatch] val warehouseStateDetailModule = Module(2022, "Silver_WarehouseStateDetail", this, Array(1004, 1013)) + lazy private val appendWarehouseStateDetailProcess: () => ETLDefinition = { + () => + ETLDefinition( + workspace.getWarehousesEventDF(warehouseStateDetailModule.fromTime, + warehouseStateDetailModule.untilTime, + config), + Seq(buildWarehouseStateDetail( + warehouseStateDetailModule.untilTime, + BronzeTargets.auditLogsTarget.asIncrementalDF(warehouseSpecModule, BronzeTargets.auditLogsTarget.incrementalColumns,1), //Added to get the Removed Cluster, + SilverTargets.dbJobRunsTarget.asIncrementalDF(warehouseStateDetailModule, SilverTargets.dbJobRunsTarget.incrementalColumns, 30), + SilverTargets.warehousesSpecTarget + )), + append(SilverTargets.warehousesStateDetailTarget) + ) + } + private def processSparkEvents(): Unit = { executorsModule.execute(appendExecutorsProcess) @@ -429,6 +452,9 @@ class Silver(_workspace: Workspace, _database: Database, _config: Config) jobStatusModule.execute(appendJobStatusProcess) jobRunsModule.execute(appendJobRunsProcess) } + case OverwatchScope.warehouseEvents => { + warehouseStateDetailModule.execute(appendWarehouseStateDetailProcess) + } case _ => } } diff --git a/src/main/scala/com/databricks/labs/overwatch/pipeline/SilverTransforms.scala b/src/main/scala/com/databricks/labs/overwatch/pipeline/SilverTransforms.scala index fc458032f..b71535d26 100644 --- a/src/main/scala/com/databricks/labs/overwatch/pipeline/SilverTransforms.scala +++ b/src/main/scala/com/databricks/labs/overwatch/pipeline/SilverTransforms.scala @@ -5,13 +5,15 @@ import com.databricks.labs.overwatch.pipeline.WorkflowsTransforms._ import com.databricks.labs.overwatch.pipeline.DbsqlTransforms._ import com.databricks.labs.overwatch.utils._ import org.apache.log4j.{Level, Logger} -import org.apache.spark.sql.expressions.Window +import org.apache.spark.sql.expressions.{Window, WindowSpec} import org.apache.spark.sql.functions._ import org.apache.spark.sql.{Column, DataFrame} -trait SilverTransforms extends SparkSessionWrapper { +trait SilverTransforms extends SparkSessionWrapper with DataFrameSyntax[ SparkSessionWrapper] { + + import TransformationDescriber._ import spark.implicits._ private val logger: Logger = Logger.getLogger(this.getClass) @@ -487,6 +489,54 @@ trait SilverTransforms extends SparkSessionWrapper { .withColumn("gcp_attributes", SchemaTools.structToMap(clusterWithStructs, "gcp_attributes")) } + private def warehouseBase(auditLogDf: DataFrame): DataFrame = { + + val warehouse_id_gen_w = Window.partitionBy('organization_id, 'warehouse_name).orderBy('timestamp).rowsBetween(Window.currentRow, 1000) + val warehouse_name_gen_w = Window.partitionBy('organization_id, 'warehouse_id).orderBy('timestamp).rowsBetween(Window.currentRow, 1000) + val warehouse_id_gen = first('warehouse_id, true).over(warehouse_id_gen_w) + val warehouse_name_gen = first('warehouse_name, true).over(warehouse_name_gen_w) + + val warehouseSummaryCols = auditBaseCols ++ Array[Column]( + deriveWarehouseId.alias("warehouse_id"), + 'name.alias("warehouse_name"), + 'cluster_size, + 'min_num_clusters, + 'max_num_clusters, + 'auto_stop_mins, + 'spot_instance_policy, + 'enable_photon, + get_json_object('channel, "$.name").alias("channel"), + 'tags, + 'enable_serverless_compute, + 'warehouse_type + ) + + val rawAuditLogDf = auditLogDf + .filter('actionName.isin("createEndpoint", "editEndpoint", "createWarehouse", + "editWarehouse", "deleteEndpoint", "deleteWarehouse") + && responseSuccessFilter + && 'serviceName === "databrickssql") + + + if(rawAuditLogDf.isEmpty) + throw new NoNewDataException("No New Data", Level.INFO, allowModuleProgression = true) + + val auditLogDfWithStructs = rawAuditLogDf + .selectExpr("*", "requestParams.*").drop("requestParams", "Overwatch_RunID") + .select(warehouseSummaryCols: _*) + .withColumn("warehouse_id", warehouse_id_gen) + .withColumn("warehouse_name", warehouse_name_gen) + + val auditLogDfWithStructsToMap = auditLogDfWithStructs + .withColumn("tags", SchemaTools.structFromJson(spark, auditLogDfWithStructs, "tags")) + .scrubSchema + + val filteredAuditLogDf = auditLogDfWithStructsToMap + .withColumn("tags", SchemaTools.structToMap(auditLogDfWithStructsToMap, "tags")) + .withColumn("source_table",lit("audit_log_bronze")) + filteredAuditLogDf + } + protected def buildPoolsSpec( poolSnapDF: DataFrame, isFirstRun: Boolean, @@ -689,6 +739,69 @@ trait SilverTransforms extends SparkSessionWrapper { } + /** + * Function to create window spec for window functions + * @param partitionCols + * @param orderByColumn + * @param boundaryStart + * @param boundaryEnd + * @return + */ + private def createWindowSpec(partitionCols: String*) + (orderByColumn: Column) + (boundaryStart: Option[Long] = None, boundaryEnd: Option[Long] = None) + : WindowSpec = { + if (partitionCols.isEmpty) + throw new IllegalArgumentException("partitionCols cannot be empty") + if (orderByColumn.toString().isEmpty) + throw new IllegalArgumentException("orderByColumn cannot be empty") + val baseWindow = Window.partitionBy(partitionCols.map(col): _*) + if (boundaryStart.isEmpty && boundaryEnd.isEmpty) + baseWindow.orderBy(orderByColumn) + else + baseWindow.rowsBetween(boundaryStart.get, boundaryEnd.get).orderBy(orderByColumn) + } + + private def getLatestClusterSnapAsSpecSilver(df: DataFrame,deriveClusterType: Column): DataFrame = { + val latestClusterSnapW = Window.partitionBy('organization_id, 'cluster_id).orderBy('Pipeline_SnapTS.desc) + df.withColumn("rnk", rank().over(latestClusterSnapW)) + .filter('rnk === 1).drop("rnk") + .withColumn("spark_conf", to_json('spark_conf)) + .withColumn("custom_tags", to_json('custom_tags)) + .select( + 'organization_id, + 'cluster_id, + lit("clusters").alias("serviceName"), + lit("snapImpute").alias("actionName"), + 'cluster_name, + 'driver_node_type_id, + 'node_type_id, + 'num_workers, + to_json('autoscale).alias("autoscale"), + 'autotermination_minutes.cast("int").alias("autotermination_minutes"), + 'enable_elastic_disk, + 'state.alias("cluster_state"), + isAutomated('cluster_name).alias("is_automated"), + deriveClusterType, + to_json('cluster_log_conf).alias("cluster_log_conf"), + to_json('init_scripts).alias("init_scripts"), + 'custom_tags, + 'cluster_source, + 'aws_attributes, + 'azure_attributes, + to_json('spark_env_vars).alias("spark_env_vars"), + 'spark_conf, + 'driver_instance_pool_id, + 'instance_pool_id, + coalesce('effective_spark_version, 'spark_version).alias("spark_version"), + (unix_timestamp('Pipeline_SnapTS) * 1000).alias("timestamp"), + 'Pipeline_SnapTS.cast("date").alias("date"), + 'creator_user_name.alias("createdBy"), + 'runtime_engine + ) + + } + protected def buildClusterSpec( bronze_cluster_snap: PipelineTable, pools_snapshot: PipelineTable, @@ -748,45 +861,9 @@ trait SilverTransforms extends SparkSessionWrapper { .select('organization_id, 'cluster_id).distinct, Seq("organization_id", "cluster_id"), "anti" ) - val latestClusterSnapW = Window.partitionBy('organization_id, 'cluster_id).orderBy('Pipeline_SnapTS.desc) - val missingClusterBaseFromSnap = bronzeClusterSnapLatest + val missingClusterBaseSnapJoined = bronzeClusterSnapLatest .join(missingClusterIds, Seq("organization_id", "cluster_id")) - .withColumn("rnk", rank().over(latestClusterSnapW)) - .filter('rnk === 1).drop("rnk") - .withColumn("spark_conf", to_json('spark_conf)) - .withColumn("custom_tags", to_json('custom_tags)) - .select( - 'organization_id, - 'cluster_id, - lit("clusters").alias("serviceName"), - lit("snapImpute").alias("actionName"), - 'cluster_name, - 'driver_node_type_id, - 'node_type_id, - 'num_workers, - to_json('autoscale).alias("autoscale"), - 'autotermination_minutes.cast("int").alias("autotermination_minutes"), - 'enable_elastic_disk, - 'state.alias("cluster_state"), - isAutomated('cluster_name).alias("is_automated"), - deriveClusterType, - to_json('cluster_log_conf).alias("cluster_log_conf"), - to_json('init_scripts).alias("init_scripts"), - 'custom_tags, - 'cluster_source, - 'aws_attributes, - 'azure_attributes, - 'gcp_attributes, - to_json('spark_env_vars).alias("spark_env_vars"), - 'spark_conf, - 'driver_instance_pool_id, - 'instance_pool_id, - coalesce('effective_spark_version, 'spark_version).alias("spark_version"), - (unix_timestamp('Pipeline_SnapTS) * 1000).alias("timestamp"), - 'Pipeline_SnapTS.cast("date").alias("date"), - 'creator_user_name.alias("createdBy"), - 'runtime_engine - ) + val missingClusterBaseFromSnap = getLatestClusterSnapAsSpecSilver(missingClusterBaseSnapJoined,deriveClusterType) unionWithMissingAsNull(clusterBaseWMetaDF, missingClusterBaseFromSnap) } else clusterBaseWMetaDF @@ -961,7 +1038,7 @@ trait SilverTransforms extends SparkSessionWrapper { } val onlyOnceSemanticsW = Window.partitionBy('organization_id, 'cluster_id, 'actionName,'timestamp).orderBy('timestamp) - clusterBaseWithPoolsAndSnapPools + val clusterSpecSilver = clusterBaseWithPoolsAndSnapPools .select(clusterSpecBaseCols: _*) .join(creatorLookup, Seq("organization_id", "cluster_id"), "left") .join(clustersRemoved, Seq("organization_id", "cluster_id"), "left") @@ -994,20 +1071,37 @@ trait SilverTransforms extends SparkSessionWrapper { .when(!'spark_version.like("%_photon_%") && 'runtime_engine.isNull,"STANDARD") .otherwise(lit("UNKNOWN"))) .drop("userEmail", "cluster_creator_lookup", "single_user_name", "rnk", "rn") + + //Code added to fill the null columns with cluster snapshot values + val latestClusterSnapW = Window.partitionBy('organization_id, 'cluster_id).orderBy('Pipeline_SnapTS.desc) + val snapLatest = getLatestClusterSnapAsSpecSilver(bronze_cluster_snap.asDF,deriveClusterType) + val commonColumns = clusterSpecSilver.columns.intersect(snapLatest.columns) + val clusterSpecSilverOnlyColumns = clusterSpecSilver.columns.diff(snapLatest.columns) + val joinedDF = clusterSpecSilver.as("cs").join(snapLatest.as("sl"), Seq("organization_id", "cluster_id"), "left_outer") + val columnExpressions = commonColumns.map { colName => + when(col(s"cs.$colName").isNull, col(s"sl.$colName")).otherwise(col(s"cs.$colName")).as(colName) + } + val clusterSpecSilverOnlyColumnExpressions = clusterSpecSilverOnlyColumns.map { colName => + col(s"cs.$colName") + } + val resultDF = joinedDF.select(columnExpressions ++ clusterSpecSilverOnlyColumnExpressions: _*) + resultDF } + def buildClusterStateDetail( untilTime: TimeTypes, auditLogDF: DataFrame, jrsilverDF: DataFrame, clusterSpec: PipelineTable, )(clusterEventsDF: DataFrame): DataFrame = { - val stateUnboundW = Window.partitionBy('organization_id, 'cluster_id).orderBy('timestamp) - val stateFromCurrentW = Window.partitionBy('organization_id, 'cluster_id).rowsBetween(1L, 1000L).orderBy('timestamp) - val stateUntilCurrentW = Window.partitionBy('organization_id, 'cluster_id).rowsBetween(-1000L, -1L).orderBy('timestamp) - val stateUntilPreviousRowW = Window.partitionBy('organization_id, 'cluster_id).rowsBetween(Window.unboundedPreceding, -1L).orderBy('timestamp) - val uptimeW = Window.partitionBy('organization_id, 'cluster_id, 'reset_partition).orderBy('unixTimeMS_state_start) - val orderingWindow = Window.partitionBy('organization_id, 'cluster_id).orderBy(desc("timestamp")) + + val stateUnboundW = createWindowSpec("organization_id", "cluster_id")('timestamp)() + val stateFromCurrentW = createWindowSpec("organization_id", "cluster_id")('timestamp)(Some(1L), Some(1000L)) + val stateUntilCurrentW = createWindowSpec("organization_id", "cluster_id")('timestamp)(Some(-1000L), Some(-1L)) + val stateUntilPreviousRowW = createWindowSpec("organization_id", "cluster_id")('timestamp)(Some(Window.unboundedPreceding), Some(-1L)) + val uptimeW = createWindowSpec("organization_id", "cluster_id", "reset_partition")('unixTimeMS_state_start)() + val orderingWindow = createWindowSpec("organization_id", "cluster_id")(col("timestamp").desc)() val nonBillableTypes = Array( "STARTING", "TERMINATING", "CREATING", "RESTARTING" , "TERMINATING_IMPUTED" @@ -1305,6 +1399,7 @@ trait SilverTransforms extends SparkSessionWrapper { "runSucceeded", "runFailed", "runTriggered", "runNow", "runStart", "submitRun", "cancel", "repairRun" ) val optimalCacheParts = Math.min(daysToProcess * getTotalCores * 2, 1000) + val jobRunsLag30D = getJobsBase(auditLogLag30D) .filter('actionName.isin(jobRunActions: _*)) .repartition(optimalCacheParts) @@ -1313,6 +1408,15 @@ trait SilverTransforms extends SparkSessionWrapper { // eagerly force this highly reused DF into cache() jobRunsLag30D.count() + + // TODO: remove or comment out or change log level or . . . + + logger.log( Level.INFO, "Showing first 5 rows of `jobRunsLag30D`:") + + jobRunsLag30D + .showLines(5, 20, true) + .foreach( logger.log( Level.INFO, _)) + // Lookup to populate the clusterID/clusterName where missing from jobs lazy val clusterSpecNameLookup = clusterSpec.asDF .select('organization_id, 'timestamp, 'cluster_name, 'cluster_id.alias("clusterId")) @@ -1350,26 +1454,176 @@ trait SilverTransforms extends SparkSessionWrapper { to_json($"task_detail_legacy.pipeline_task").alias("pipeline_task") ) - lazy val jobSnapNameLookup = jobsSnapshot.asDF - .withColumn("timestamp", unix_timestamp('Pipeline_SnapTS) * lit(1000)) - .select('organization_id, 'timestamp, 'job_id.alias("jobId"), $"settings.name".alias("jobName")) + logger.log( Level.INFO, s"optimalCacheParts: ${optimalCacheParts}") - val jobRunsLookups = jobRunsInitializeLookups( - (clusterSpec, clusterSpecNameLookup), - (clusterSnapshot, clusterSnapNameLookup), - (jobsStatus, jobStatusMetaLookup), - (jobsSnapshot, jobSnapNameLookup) - ) + val filterAuditLog = NamedTransformation { + (df: DataFrame) => { + val cachedActions = getJobsBase( df) + .filter('actionName.isin( jobRunActions: _*)) + .repartition( optimalCacheParts) + .cache() // cached df removed at end of module run + + // eagerly force this highly reused DF into cache() + val n = cachedActions.count() + + logger.log( Level.INFO, s"jobRunsLag30D count: ${n}") + logger.log( Level.INFO, s"jobRunsLag30D.rdd.partitions.size: ${cachedActions.rdd.partitions.size}") + + cachedActions + } + } + + val jobRunsLag30D = auditLogLag30D.transformWithDescription( filterAuditLog) + + /* + * keep original usage example of `.showLines()` for reference + * + * logger.log( Level.INFO, "Showing first 5 rows of `jobRunsLag30D`:") + * jobRunsLag30D + * .showLines(5, 20, true) + * .foreach( logger.log( Level.INFO, _)) + * + */ + + + /** + * Look up the cluster_name based on id first from + * `job_status_silver`. If not present there fallback to latest + * snapshot prior to the run + */ + + val jobRunsAppendClusterName = NamedTransformation { + (df: DataFrame) => { + + val key = Seq( "organization_id", "clusterId") + + lazy val clusterSpecNameLookup = clusterSpec.asDF + .select( + 'organization_id, + 'timestamp, + 'cluster_name, + 'cluster_id.alias("clusterId")) + .filter( + 'clusterId.isNotNull + && 'cluster_name.isNotNull) + .toTSDF( "timestamp", key:_*) + + lazy val clusterSnapNameLookup = clusterSnapshot.asDF + .select( + // .withColumn("timestamp", unix_timestamp('Pipeline_SnapTS) * lit(1000)) + 'organization_id, + ( unix_timestamp('Pipeline_SnapTS) * lit(1000)).alias( "timestamp"), + 'cluster_name, + 'cluster_id.alias( "clusterId")) + .filter( + 'clusterId.isNotNull + && 'cluster_name.isNotNull) + .toTSDF( "timestamp", key:_*) + + df.toTSDF( "timestamp", key:_*) + .lookupWhen( clusterSpecNameLookup) + .lookupWhen( clusterSnapNameLookup) + .df + } + } + + /** + * looks up the job name based on id first from job_status_silver + * and if not present there fallback to latest snapshot prior to + * the run + */ + + val jobRunsAppendJobMeta = NamedTransformation { + (df: DataFrame) => { + + val key = Seq( "organization_id", "jobId") + + lazy val jobStatusMetaLookup = jobsStatus.asDF + .verifyMinimumSchema( + Schema.minimumJobStatusSilverMetaLookupSchema) + .select( + 'organization_id, + 'timestamp, + 'jobId, + 'jobName, + to_json('tags).alias("tags"), + 'schedule, + 'max_concurrent_runs, + 'run_as_user_name, + 'timeout_seconds, + 'created_by, + 'last_edited_by, + to_json( 'tasks).alias("tasks"), + to_json( 'job_clusters).alias("job_clusters"), + to_json( $"task_detail_legacy.notebook_task").alias( "notebook_task"), + to_json( $"task_detail_legacy.spark_python_task").alias( "spark_python_task"), + to_json( $"task_detail_legacy.python_wheel_task").alias( "python_wheel_task"), + to_json( $"task_detail_legacy.spark_jar_task").alias( "spark_jar_task"), + to_json( $"task_detail_legacy.spark_submit_task").alias( "spark_submit_task"), + to_json( $"task_detail_legacy.shell_command_task").alias( "shell_command_task"), + to_json( $"task_detail_legacy.pipeline_task").alias( "pipeline_task")) + .toTSDF( "timestamp", key:_*) + + lazy val jobSnapshotNameLookup = jobsSnapshot.asDF + // .withColumn("timestamp", unix_timestamp('Pipeline_SnapTS) * lit(1000)) + .select( + 'organization_id, + ( unix_timestamp( 'Pipeline_SnapTS) * lit( 1000)).alias( "timestamp"), + 'job_id.alias("jobId"), + $"settings.name".alias("jobName")) + .toTSDF( "timestamp", key:_*) + + df.toTSDF( "timestamp", key:_*) + .lookupWhen( jobStatusMetaLookup) + .lookupWhen( jobSnapshotNameLookup) + .df + .withColumn( "jobName", + coalesce('jobName, 'run_name)) + .withColumn( "tasks", + coalesce('tasks, 'submitRun_tasks)) + .withColumn( "job_clusters", + coalesce('job_clusters, 'submitRun_job_clusters)) + + // the following is only valid in Spark > 3.1.2, apparently + // (it compiles with 3.3.0) + + // .withColumns( Map( + // "jobName" + // -> coalesce('jobName, 'run_name), + // "tasks" + // -> coalesce('tasks, 'submitRun_tasks), + // "job_clusters" + // -> coalesce('job_clusters, 'submitRun_job_clusters))) + } + } + + + + // val jobRunsLookups = jobRunsInitializeLookups( + // (clusterSpec, clusterSpecNameLookup), + // (clusterSnapshot, clusterSnapNameLookup), + // (jobsStatus, jobStatusMetaLookup), + // (jobsSnapshot, jobSnapNameLookup) + // ) // caching before structifying - jobRunsDeriveRunsBase(jobRunsLag30D, etlUntilTime) - .transform(jobRunsAppendClusterName(jobRunsLookups)) - .transform(jobRunsAppendJobMeta(jobRunsLookups)) - .transform(jobRunsStructifyLookupMeta(optimalCacheParts)) - .transform(jobRunsAppendTaskAndClusterDetails) - .transform(jobRunsCleanseCreatedNestedStructures(targetKeys)) - // .transform(jobRunsRollupWorkflowsAndChildren) - .drop("timestamp") // could be duplicated to enable asOf Lookups, dropping to clean up + val cachedDF = jobRunsLag30D + .transformWithDescription( jobRunsDeriveRunsBase( etlUntilTime)) + .transformWithDescription( jobRunsAppendClusterName) + .transformWithDescription( jobRunsAppendJobMeta) + .repartition( optimalCacheParts) + .cache() // to speed up schema inference while "structifying" next + + logger.log( Level.INFO, s"cachedDF count before structifying: ${cachedDF.count()}") + logger.log( Level.INFO, s"cachedDF.rdd.partitions.size: ${cachedDF.rdd.partitions.size}") + + cachedDF + .transformWithDescription( jobRunsStructifyLookupMeta) // ( optimalCacheParts)) + .transformWithDescription( jobRunsAppendTaskAndClusterDetails) + .transformWithDescription( jobRunsCleanseCreatedNestedStructures( targetKeys)) + .drop("timestamp") + // `timestamp` could be duplicated to enable `asOf` lookups; + // dropping to clean up } protected def notebookSummary()(df: DataFrame): DataFrame = { @@ -1408,4 +1662,222 @@ trait SilverTransforms extends SparkSessionWrapper { .transform(deriveWarehouseBase()) .transform(deriveWarehouseBaseFilled(isFirstRun, bronzeWarehouseSnapLatest, silver_warehouse_spec)) } + + protected def buildWarehouseStateDetail( + untilTime: TimeTypes, + auditLogDF: DataFrame, + jrsilverDF: DataFrame, + warehousesSpec: PipelineTable, + )(warehouseEventsDF: DataFrame): DataFrame = { + + val stateUnboundW = createWindowSpec("organization_id", "warehouse_id")(col("timestamp"))() + val stateFromCurrentW = createWindowSpec("organization_id", "warehouse_id")(col("timestamp"))(Some(1L), Some(1000L)) + val stateUntilCurrentW = createWindowSpec("organization_id", "warehouse_id")(col("timestamp"))(Some(-1000L), Some(-1L)) + val stateUntilPreviousRowW = createWindowSpec("organization_id", "warehouse_id")(col("timestamp"))(Some(Window.unboundedPreceding), Some(-1L)) + val uptimeW = createWindowSpec("organization_id", "warehouse_id", "state_transition_flag_sum")(col("unixTimeMS_state_start"))() + val orderingWindow = createWindowSpec("organization_id", "warehouse_id")(col("timestamp").desc)() + + + val nonBillableTypes = Array( + "STARTING", "TERMINATING", "CREATING", "RESTARTING" , "TERMINATING_IMPUTED" + ,"STOPPING","STOPPED" //new states from warehouse_events + ) + + val runningStates = Array( + "STARTING", "INIT_SCRIPTS_STARTED", "RUNNING", "CREATING", + "RESIZING", "UPSIZE_COMPLETED", "DRIVER_HEALTHY" + ,"SCALED_UP","SCALED_DOWN" //new states from warehouse_events + ) + + val invalidEventChain = lead('runningSwitch, 1).over(stateUnboundW).isNotNull && lead('runningSwitch, 1) + .over(stateUnboundW) === lead('previousSwitch, 1).over(stateUnboundW) + + val warehouseEventsFinal = if (jrsilverDF.isEmpty || warehousesSpec.asDF.isEmpty) { + warehouseEventsDF // need to add "min_num_clusters","max_num_clusters" + .withColumn("min_num_clusters",lit(0)) + .withColumn("max_num_clusters",lit(0)) + .withColumn("timestamp", unix_timestamp($"timestamp")*1000) + }else{ + val refinedWarehouseEventsDFFiltered = warehouseEventsDF + .withColumn("row", row_number().over(orderingWindow)) + .filter('state =!= "TERMINATING" && 'row === 1) + + val jrSilverAgg= jrsilverDF + .filter('clusterType === "sqlWarehouse") + .groupBy("clusterID") + .agg(max("TaskExecutionRunTime.endTS").alias("end_run_time")) + .withColumnRenamed("clusterID","warehouseId") + .filter('end_run_time.isNotNull) + + val joined = refinedWarehouseEventsDFFiltered.join(jrSilverAgg, + refinedWarehouseEventsDFFiltered("warehouse_id") === jrSilverAgg("warehouseId"), "inner") + .withColumn("state", lit("TERMINATING_IMPUTED")) // check if STOPPING_IMPUTED can be used ? + + // Join with Cluster Spec to get filter on automated cluster + val warehousesSpecDF = warehousesSpec.asDF + .select("warehouse_id","warehouse_name","min_num_clusters","max_num_clusters") + .dropDuplicates() + + val jobClusterImputed = joined.join(warehousesSpecDF,Seq("warehouse_id"),"inner") + .drop("row","warehouseId","end_run_time","warehouse_name") + + warehouseEventsDF + .unionByName(jobClusterImputed, allowMissingColumns = true) + .withColumn("timestamp", unix_timestamp($"timestamp")*1000) // need to add "min_num_clusters","max_num_clusters" + } + + val warehouseBaseDF = warehouseBase(auditLogDF) + + val warehouseBaseDF_latest = warehouseBaseDF + .withColumn("row_num",row_number().over(stateUnboundW)) + .filter('row_num === 1).dropDupColumnByAlias("row_num") + .select("organization_id","warehouse_id","max_num_clusters","min_num_clusters") + .withColumnRenamed("max_num_clusters","warehouse_max_num_clusters") + .withColumnRenamed("min_num_clusters","warehouse_min_num_clusters") + + val warehouseEventsDerived = warehouseEventsFinal.join( + warehouseBaseDF_latest + ,Seq("organization_id","warehouse_id"),"left" + ) + .withColumn("max_num_clusters",when('max_num_clusters === 0,'warehouse_max_num_clusters).otherwise('max_num_clusters)) + + val warehouseEventsBaseline = warehouseEventsDerived //warehouseEventsFinal + .withColumn( + "runningSwitch", + when('state.isin("TERMINATING","TERMINATING_IMPUTED","STOPPING"), lit(false)) //added STOPPING + .when('state.isin("CREATING", "STARTING"), lit(true)) + .otherwise(lit(null).cast("boolean"))) + .withColumn( + "previousSwitch", + when('runningSwitch.isNotNull, last('runningSwitch, true).over(stateUntilPreviousRowW)) + ) + .withColumn( + "invalidEventChainHandler", + when(invalidEventChain, array(lit(false), lit(true))).otherwise(array(lit(false))) + ) + .selectExpr("*", "explode(invalidEventChainHandler) as imputedTerminationEvent").drop("invalidEventChainHandler") + .withColumn("state", when('imputedTerminationEvent, "STOPPING").otherwise('state)) // replaced TERMINATED with STOPPING + .withColumn("timestamp", when('imputedTerminationEvent, lag('timestamp, 1).over(stateUnboundW) + 1L).otherwise('timestamp)) + .withColumn("lastRunningSwitch", last('runningSwitch, true).over(stateUntilCurrentW)) // previous on/off switch + .withColumn("nextRunningSwitch", first('runningSwitch, true).over(stateFromCurrentW)) // next on/off switch + // given no anomaly, set on/off state to current state + // if no current state use previous state + // if no previous state found, assume opposite of next state switch + .withColumn("isRunning", coalesce( + when('imputedTerminationEvent, lit(false)).otherwise(lit(null).cast("boolean")), + 'runningSwitch, + 'lastRunningSwitch, + !'nextRunningSwitch + )) + // if isRunning still undetermined, use guaranteed events to create state anchors to identify isRunning anchors + .withColumn("isRunning", when('isRunning.isNull && 'state.isin(runningStates: _*), lit(true)).otherwise('isRunning)) + // use the anchors to fill in the null gaps between the state changes to determine if running + // if ultimately unable to be determined, assume not isRunning + .withColumn("isRunning", coalesce( + when('isRunning.isNull, last('isRunning, true).over(stateUntilCurrentW)).otherwise('isRunning), + when('isRunning.isNull, !first('isRunning, true).over(stateFromCurrentW)).otherwise('isRunning), + lit(false) + )).drop("lastRunningSwitch", "nextRunningSwitch") + .withColumn("previousIsRunning",lag($"isRunning", 1, null).over(stateUnboundW)) + .withColumn("isRunning",when(col("previousIsRunning") === "false" && col("state") === "EXPANDED_DISK",lit(false)).otherwise('isRunning)) + .drop("previousIsRunning") + .withColumn( + "current_num_clusters", + coalesce( + when(!'isRunning || 'isRunning.isNull, lit(null).cast("long")) + .otherwise( + coalesce( // get current_num_workers no matter where the value is stored based on business rules + 'cluster_count, + 'min_num_clusters, + last(coalesce( // look for the last non-null value when current value isn't present + 'cluster_count, + 'min_num_clusters + ), true).over(stateUntilCurrentW) + ) + ), + lit(0) // don't allow null returns + ) + ) + .withColumn( + "target_num_clusters", // need to check this logic and rename column + coalesce( + when(!'isRunning || 'isRunning.isNull, lit(null).cast("long")) + .when('state === "CREATING", + coalesce('min_num_clusters, 'current_num_clusters)) + .otherwise(coalesce('max_num_clusters, 'current_num_clusters)), + lit(0) // don't allow null returns + ) + ) + .select( + 'organization_id, 'warehouse_id, 'isRunning, + 'timestamp, 'state, 'current_num_clusters, 'target_num_clusters + ) + .withColumn("unixTimeMS_state_start", 'timestamp) + .withColumn("unixTimeMS_state_end", coalesce( // if state end open, use pipelineSnapTime, will be merged when state end is received + lead('timestamp, 1).over(stateUnboundW) - lit(1), // subtract 1 millis + lit(untilTime.asUnixTimeMilli) + )) + + // Start changes from here + // Get the warehouseID that has been Permenantly_Deleted + + val removedWarehouseID = warehouseBaseDF + .filter('actionName.isin("deleteEndpoint")) + .select('warehouse_id,'timestamp.alias("deletion_timestamp")).distinct() + + val warehouseEventsBaselineForRemovedCluster = warehouseEventsBaseline.join(removedWarehouseID,Seq("warehouse_id")) + + val window = Window.partitionBy('organization_id, 'warehouse_id).orderBy('timestamp.desc) + val stateBeforeRemoval = warehouseEventsBaselineForRemovedCluster + .withColumn("rnk",rank().over(window)) + .withColumn("rn", row_number().over(window)) + .withColumn("unixTimeMS_state_end",when('state.isin("STOPPING","TERMINATING_IMPUTED"),'unixTimeMS_state_end).otherwise('deletion_timestamp)) + .filter('rnk === 1 && 'rn === 1).drop("rnk", "rn") + + val stateDuringRemoval = stateBeforeRemoval + .withColumn("timestamp",when('state.isin("STOPPING","TERMINATING","TERMINATING_IMPUTED"),'unixTimeMS_state_end+1).otherwise(col("deletion_timestamp")+1)) + .withColumn("isRunning",lit(false)) + .withColumn("unixTimeMS_state_start",('timestamp)) + .withColumn("unixTimeMS_state_end",('timestamp)) + .withColumn("state",lit("PERMENANT_DELETE")) + .withColumn("current_num_clusters",lit(0)) + .withColumn("target_num_clusters",lit(0)) + .drop("deletion_timestamp") + + val columns: Array[String] = warehouseEventsBaseline.columns + val stateDuringRemovalFinal = stateBeforeRemoval.drop("deletion_timestamp") + .unionByName(stateDuringRemoval, allowMissingColumns = true) + .select(columns.map(col): _*) + + val warehouseEventsBaselineFinal = warehouseEventsBaseline.join(stateDuringRemovalFinal,Seq("warehouse_id","timestamp"),"anti") + .select(columns.map(col): _*) + .unionByName(stateDuringRemovalFinal, allowMissingColumns = true) + + warehouseEventsBaselineFinal + .withColumn("state_transition_flag", + when( + lag('state, 1).over(stateUnboundW).isin("STOPPING","TERMINATING", "RESTARTING", "EDITED","TERMINATING_IMPUTED") || + !'isRunning, lit(1) + ).otherwise(lit(0)) + ) + .withColumn("state_transition_flag_sum", sum('state_transition_flag).over(stateUnboundW)) + .withColumn("target_num_clusters", last('target_num_clusters, true).over(stateUnboundW)) + .withColumn("current_num_clusters", last('current_num_clusters, true).over(stateUnboundW)) + .withColumn("timestamp_state_start", from_unixtime('unixTimeMS_state_start.cast("double") / lit(1000)).cast("timestamp")) + .withColumn("timestamp_state_end", from_unixtime('unixTimeMS_state_end.cast("double") / lit(1000)).cast("timestamp")) // subtract 1.0 millis + .withColumn("state_start_date", 'timestamp_state_start.cast("date")) + .withColumn("uptime_in_state_S", ('unixTimeMS_state_end - 'unixTimeMS_state_start) / lit(1000)) + .withColumn("uptime_since_restart_S", + coalesce( + when('state_transition_flag === 1, lit(0)) + .otherwise(sum('uptime_in_state_S).over(uptimeW)), + lit(0) + ) + ) + .withColumn("cloud_billable", 'isRunning) + .withColumn("databricks_billable", 'isRunning && !'state.isin(nonBillableTypes: _*)) + .withColumn("uptime_in_state_H", 'uptime_in_state_S / lit(3600)) + .withColumn("state_dates", sequence('timestamp_state_start.cast("date"), 'timestamp_state_end.cast("date"))) + .withColumn("days_in_state", size('state_dates)) + } } diff --git a/src/main/scala/com/databricks/labs/overwatch/pipeline/WorkflowsTransforms.scala b/src/main/scala/com/databricks/labs/overwatch/pipeline/WorkflowsTransforms.scala index ff0471f05..bb45f25bc 100644 --- a/src/main/scala/com/databricks/labs/overwatch/pipeline/WorkflowsTransforms.scala +++ b/src/main/scala/com/databricks/labs/overwatch/pipeline/WorkflowsTransforms.scala @@ -12,6 +12,7 @@ import org.apache.spark.sql.{Column, DataFrame} object WorkflowsTransforms extends SparkSessionWrapper { + import TransformationDescriber._ import spark.implicits._ /** @@ -41,6 +42,7 @@ object WorkflowsTransforms extends SparkSessionWrapper { def getJobsBase(df: DataFrame): DataFrame = { val onlyOnceJobRecW = Window.partitionBy('organization_id, 'timestamp, 'actionName, 'requestId, $"response.statusCode", 'runId).orderBy('timestamp) df.filter(col("serviceName") === "jobs") + .verifyMinimumSchema(Schema.auditMasterSchema) .selectExpr("*", "requestParams.*").drop("requestParams") .withColumn("rnk", rank().over(onlyOnceJobRecW)) .withColumn("rn", row_number.over(onlyOnceJobRecW)) @@ -543,13 +545,13 @@ object WorkflowsTransforms extends SparkSessionWrapper { */ // val jobRunsLookups: Map[String, DataFrame] = - def jobRunsInitializeLookups(lookups: (PipelineTable, DataFrame)*): Map[String, DataFrame] = { - lookups - .filter(_._1.exists) - .map(lookup => { - (lookup._1.name, lookup._2) - }).toMap - } + // def jobRunsInitializeLookups(lookups: (PipelineTable, DataFrame)*): Map[String, DataFrame] = { + // lookups + // .filter(_._1.exists) + // .map(lookup => { + // (lookup._1.name, lookup._2) + // }).toMap + // } def jobRunsDeriveCompletedRuns(df: DataFrame, firstRunSemanticsW: WindowSpec): DataFrame = { df @@ -767,147 +769,151 @@ object WorkflowsTransforms extends SparkSessionWrapper { .agg(collect_list('repair_details).alias("repair_details")) } - def jobRunsDeriveRunsBase(df: DataFrame, etlUntilTime: TimeTypes): DataFrame = { - - val arrayStringSchema = ArrayType(StringType, containsNull = true) - val firstTaskRunSemanticsW = Window.partitionBy('organization_id, 'jobRunId, 'taskRunId).orderBy('timestamp) - val firstJobRunSemanticsW = Window.partitionBy('organization_id, 'jobRunId).orderBy('timestamp) - val firstRunSemanticsW = Window.partitionBy('organization_id, 'runId).orderBy('timestamp) - - // Completes must be >= etlStartTime as it is the driver endpoint - // All joiners to Completes may be from the past up to N days as defined in the incremental df - // Identify all completed jobs in scope for this overwatch run - val allCompletes = jobRunsDeriveCompletedRuns(df, firstTaskRunSemanticsW) - - // CancelRequests are still lookups from the driver "complete" as a cancel request is a request and still - // results in a runFailed after the cancellation - // Identify all cancelled jobs in scope for this overwatch run - val allCancellations = jobRunsDeriveCancelledRuns(df, firstRunSemanticsW) - - // DF for jobs launched with actionName == "runNow" - // Lookback 30 days for laggard starts prior to current run - // only field from runNow that we care about is the response.result.runId - val runNowStart = jobRunsDeriveRunsLaunched(df, firstJobRunSemanticsW, arrayStringSchema) - - val runTriggered = jobRunsDeriveRunsTriggered(df, firstJobRunSemanticsW) - - /** - * These are runs submitted using the "submitRun" API endpoint. These runs will have no corresponding job - * since the job was never scheduled. The entire definition of the job and the cluster must be sumitted - * in this API call. Does not reference an existing job_id present in the jobsStatus Target - */ - val runSubmitStart = jobRunsDeriveSubmittedRuns(df, firstJobRunSemanticsW) - - // DF to pull unify differing schemas from runNow and submitRun and pull all job launches into one DF - val allSubmissions = runNowStart - .unionByName(runTriggered, allowMissingColumns = true) - .unionByName(runSubmitStart, allowMissingColumns = true) - - // Find the corresponding runStart action for the completed jobs - // Lookback 30 days for laggard starts prior to current run - val runStarts = jobRunsDeriveRunStarts(df, firstTaskRunSemanticsW) - - val repairDetails = jobRunsDeriveRepairRunsDetail(df, firstRunSemanticsW) - - val jobRunsMaster = allSubmissions - .join(runStarts, Seq("organization_id", "jobRunId"), "left") - .join(allCompletes, Seq("organization_id", "jobRunId", "taskRunId"), "left") - .withColumn("runId", coalesce('taskRunId, 'jobRunId).cast("long")) - .join(allCancellations, Seq("organization_id", "runId"), "left") - .withColumn("repairId", coalesce('repairId_runStart, 'repairId_Completed).cast("long")) - .join(repairDetails, Seq("organization_id", "runId", "repairId"), "left") - .cache() // caching to speed up schema inference - - jobRunsMaster.count - - jobRunsMaster - .select( - 'organization_id, - coalesce('runStartJobId, 'completedJobId, 'submissionJobId).cast("long").alias("jobId"), - 'jobRunId.cast("long"), - 'taskRunId.cast("long"), - coalesce('taskKey_runStart, 'taskKey_Completed).alias("taskKey"), - from_json(coalesce('taskDependencies_runStart, 'taskDependencies_Completed), arrayStringSchema).alias("taskDependencies"), - 'runId, - coalesce('multitaskParentRunId_Started, 'multitaskParentRunId_Completed).cast("long").alias("multitaskParentRunId"), - coalesce('parentRunId_Started, 'parentRunId_Completed).cast("long").alias("parentRunId"), - coalesce('taskRunId, 'idInJob).cast("long").alias("idInJob"), - TransformFunctions.subtractTime( - 'startTime, - array_max(array('completionTime, 'cancellationTime)) // endTS must remain null if still open - ).alias("TaskRunTime"), // run launch time until terminal event - TransformFunctions.subtractTime( - 'startTime, - array_max(array('completionTime, 'cancellationTime)) // endTS must remain null if still open - ).alias("TaskExecutionRunTime"), // from cluster up and run begin until terminal event - 'run_name, - when(coalesce('jobClusterType_Started, 'jobClusterType_Completed).isNull and ('job_clusters.isNotNull or 'new_cluster.isNotNull), "new") - .otherwise(coalesce('jobClusterType_Started, 'jobClusterType_Completed)).alias("clusterType"), - coalesce('jobTaskType_Started, 'jobTaskType_Completed).alias("taskType"), - coalesce('jobTriggerType_Triggered,'jobTriggerType_Started, 'jobTriggerType_Completed, 'jobTriggerType_runNow).alias("jobTriggerType"), - when('cancellationRequestId.isNotNull, "Cancelled") - .otherwise('jobTerminalState) - .alias("terminalState"), - 'clusterId, - 'existing_cluster_id, - 'new_cluster, - 'tasks.alias("submitRun_tasks"), - 'job_clusters.alias("submitRun_job_clusters"), - 'libraries, - 'access_control_list, - 'git_source, - 'manual_override_params, - coalesce('workflow_context_runNow, 'workflow_context_submitRun).alias("workflow_context"), - 'notebook_task, - 'spark_python_task, - 'python_wheel_task, - 'spark_jar_task, - 'spark_submit_task, - 'shell_command_task, - 'pipeline_task, - 'repairId, - 'repair_details, - struct( - 'startTime, - 'submissionTime, - 'cancellationTime, - 'completionTime, - 'timeout_seconds - ).alias("timeDetails"), - struct( - struct( - 'submitRequestId, - 'submitResponse, - 'submitSessionId, - 'submitSourceIP, - 'submitUserAgent, - 'submittedBy - ).alias("submissionRequest"), - struct( - 'cancellationRequestId, - 'cancellationResponse, - 'cancellationSessionId, - 'cancellationSourceIP, - 'cancelledUserAgent, - 'cancelledBy - ).alias("cancellationRequest"), + val jobRunsDeriveRunsBase = (etlUntilTime: TimeTypes) => NamedTransformation { + (df: DataFrame) => { + + val arrayStringSchema = ArrayType(StringType, containsNull = true) + val firstTaskRunSemanticsW = Window.partitionBy('organization_id, 'jobRunId, 'taskRunId).orderBy('timestamp) + val firstJobRunSemanticsW = Window.partitionBy('organization_id, 'jobRunId).orderBy('timestamp) + val firstRunSemanticsW = Window.partitionBy('organization_id, 'runId).orderBy('timestamp) + + // Completes must be >= etlStartTime as it is the driver endpoint + // All joiners to Completes may be from the past up to N days as defined in the incremental df + // Identify all completed jobs in scope for this overwatch run + val allCompletes = jobRunsDeriveCompletedRuns(df, firstTaskRunSemanticsW) + + // CancelRequests are still lookups from the driver "complete" as a cancel request is a request and still + // results in a runFailed after the cancellation + // Identify all cancelled jobs in scope for this overwatch run + val allCancellations = jobRunsDeriveCancelledRuns(df, firstRunSemanticsW) + + // DF for jobs launched with actionName == "runNow" + // Lookback 30 days for laggard starts prior to current run + // only field from runNow that we care about is the response.result.runId + val runNowStart = jobRunsDeriveRunsLaunched(df, firstJobRunSemanticsW, arrayStringSchema) + + val runTriggered = jobRunsDeriveRunsTriggered(df, firstJobRunSemanticsW) + + /** + * These are runs submitted using the "submitRun" API endpoint. These runs will have no corresponding job + * since the job was never scheduled. The entire definition of the job and the cluster must be sumitted + * in this API call. Does not reference an existing job_id present in the jobsStatus Target + */ + val runSubmitStart = jobRunsDeriveSubmittedRuns(df, firstJobRunSemanticsW) + + // DF to pull unify differing schemas from runNow and submitRun and pull all job launches into one DF + val allSubmissions = runNowStart + .unionByName(runTriggered, allowMissingColumns = true) + .unionByName(runSubmitStart, allowMissingColumns = true) + + // Find the corresponding runStart action for the completed jobs + // Lookback 30 days for laggard starts prior to current run + val runStarts = jobRunsDeriveRunStarts(df, firstTaskRunSemanticsW) + + val repairDetails = jobRunsDeriveRepairRunsDetail(df, firstRunSemanticsW) + + val jobRunsMaster = allSubmissions + .join(runStarts, Seq("organization_id", "jobRunId"), "left") + .join(allCompletes, Seq("organization_id", "jobRunId", "taskRunId"), "left") + .withColumn("runId", coalesce('taskRunId, 'jobRunId).cast("long")) + .join(allCancellations, Seq("organization_id", "runId"), "left") + .withColumn("repairId", coalesce('repairId_runStart, 'repairId_Completed).cast("long")) + .join(repairDetails, Seq("organization_id", "runId", "repairId"), "left") + .cache() // caching to speed up schema inference + + jobRunsMaster.count + + jobRunsMaster + .select( + 'organization_id, + coalesce('runStartJobId, 'completedJobId, 'submissionJobId).cast("long").alias("jobId"), + 'jobRunId.cast("long"), + 'taskRunId.cast("long"), + coalesce('taskKey_runStart, 'taskKey_Completed).alias("taskKey"), + from_json(coalesce('taskDependencies_runStart, 'taskDependencies_Completed), arrayStringSchema).alias("taskDependencies"), + 'runId, + coalesce('multitaskParentRunId_Started, 'multitaskParentRunId_Completed).cast("long").alias("multitaskParentRunId"), + coalesce('parentRunId_Started, 'parentRunId_Completed).cast("long").alias("parentRunId"), + coalesce('taskRunId, 'idInJob).cast("long").alias("idInJob"), + TransformFunctions.subtractTime( + 'startTime, + array_max(array('completionTime, 'cancellationTime)) // endTS must remain null if still open + ).alias("TaskRunTime"), // run launch time until terminal event + TransformFunctions.subtractTime( + 'startTime, + array_max(array('completionTime, 'cancellationTime)) // endTS must remain null if still open + ).alias("TaskExecutionRunTime"), // from cluster up and run begin until terminal event + 'run_name, + when(coalesce('jobClusterType_Started, 'jobClusterType_Completed).isNull and ('job_clusters.isNotNull or 'new_cluster.isNotNull), "new") + .otherwise(coalesce('jobClusterType_Started, 'jobClusterType_Completed)).alias("clusterType"), + coalesce('jobTaskType_Started, 'jobTaskType_Completed).alias("taskType"), + coalesce('jobTriggerType_Triggered,'jobTriggerType_Started, 'jobTriggerType_Completed, 'jobTriggerType_runNow).alias("jobTriggerType"), + when('cancellationRequestId.isNotNull, "Cancelled") + .otherwise('jobTerminalState) + .alias("terminalState"), + 'clusterId, + 'existing_cluster_id, + 'new_cluster, + 'tasks.alias("submitRun_tasks"), + 'job_clusters.alias("submitRun_job_clusters"), + 'libraries, + 'access_control_list, + 'git_source, + 'manual_override_params, + coalesce('workflow_context_runNow, 'workflow_context_submitRun).alias("workflow_context"), + 'notebook_task, + 'spark_python_task, + 'python_wheel_task, + 'spark_jar_task, + 'spark_submit_task, + 'shell_command_task, + 'pipeline_task, + 'repairId, + 'repair_details, struct( - 'completionRequestId, - 'completionResponse - ).alias("completionRequest"), + 'startTime, + 'submissionTime, + 'cancellationTime, + 'completionTime, + 'timeout_seconds + ).alias("timeDetails"), struct( - 'startRequestId - ).alias("startRequest") - ).alias("requestDetails") - ) - .withColumn("timestamp", $"TaskRunTime.startEpochMS") // TS lookup key added for next steps (launch time) - .withColumn("startEpochMS", $"TaskRunTime.startEpochMS") // set launch time as TS key -// .scrubSchema + struct( + 'submitRequestId, + 'submitResponse, + 'submitSessionId, + 'submitSourceIP, + 'submitUserAgent, + 'submittedBy + ).alias("submissionRequest"), + struct( + 'cancellationRequestId, + 'cancellationResponse, + 'cancellationSessionId, + 'cancellationSourceIP, + 'cancelledUserAgent, + 'cancelledBy + ).alias("cancellationRequest"), + struct( + 'completionRequestId, + 'completionResponse + ).alias("completionRequest"), + struct( + 'startRequestId + ).alias("startRequest") + ).alias("requestDetails") + ) + .withColumn("timestamp", $"TaskRunTime.startEpochMS") // TS lookup key added for next steps (launch time) + .withColumn("startEpochMS", $"TaskRunTime.startEpochMS") // set launch time as TS key + // .scrubSchema + } } - def jobRunsStructifyLookupMeta(cacheParts: Int)(df: DataFrame): DataFrame = { - val dfc = df.repartition(cacheParts).cache() // caching to speed up schema inference - dfc.count() + // val jobRunsStructifyLookupMeta = (cacheParts: Int) => NamedTransformation { (df: DataFrame) => { + // val dfc = df.repartition(cacheParts).cache() // caching to speed up schema inference + // dfc.count() + + val jobRunsStructifyLookupMeta = NamedTransformation { (dfc: DataFrame) => { val colsToOverride = Array("tasks", "job_clusters", "tags").toSet val dfOrigCols = (dfc.columns.toSet -- colsToOverride).toArray map col val colsToAppend: Array[Column] = Array( @@ -945,9 +951,9 @@ object WorkflowsTransforms extends SparkSessionWrapper { ) .scrubSchema - } + }} - def jobRunsCleanseCreatedNestedStructures(keys: Array[String])(df: DataFrame): DataFrame = { + val jobRunsCleanseCreatedNestedStructures = (keys: Array[String]) => NamedTransformation { (df: DataFrame) => { val emptyKeysDF = Seq.empty[(String, Long, Long)].toDF("organization_id", "runId", "startEpochMS") val cleansedTasksDF = workflowsCleanseTasks(df, keys, emptyKeysDF, "submitRun_details.tasks") @@ -987,66 +993,48 @@ object WorkflowsTransforms extends SparkSessionWrapper { ) // cleanup temporary cleaner fields .scrubSchema - } - - /** - * looks up the cluster_name based on id first from job_status_silver and if not present there fallback to latest - * snapshot prior to the run - */ - def jobRunsAppendClusterName(lookups: Map[String, DataFrame])(df: DataFrame): DataFrame = { - - val runsWClusterNames1 = if (lookups.contains("cluster_spec_silver")) { - df.toTSDF("timestamp", "organization_id", "clusterId") - .lookupWhen( - lookups("cluster_spec_silver") - .toTSDF("timestamp", "organization_id", "clusterId") - ).df - } else df - - val runsWClusterNames2 = if (lookups.contains("clusters_snapshot_bronze")) { - runsWClusterNames1 - .toTSDF("timestamp", "organization_id", "clusterId") - .lookupWhen( - lookups("clusters_snapshot_bronze") - .toTSDF("timestamp", "organization_id", "clusterId") - ).df - } else runsWClusterNames1 - - runsWClusterNames2 - } - - /** - * looks up the job name based on id first from job_status_silver and if not present there fallback to latest - * snapshot prior to the run - */ - def jobRunsAppendJobMeta(lookups: Map[String, DataFrame])(df: DataFrame): DataFrame = { - - val runsWithJobName1 = if (lookups.contains("job_status_silver")) { - df - .toTSDF("timestamp", "organization_id", "jobId") - .lookupWhen( - lookups("job_status_silver") - .toTSDF("timestamp", "organization_id", "jobId") - ).df - } else df - - val runsWithJobName2 = if (lookups.contains("jobs_snapshot_bronze")) { - runsWithJobName1 - .toTSDF("timestamp", "organization_id", "jobId") - .lookupWhen( - lookups("jobs_snapshot_bronze") - .toTSDF("timestamp", "organization_id", "jobId") - ).df - } else df - - runsWithJobName2 - .withColumn("jobName", coalesce('jobName, 'run_name)) - .withColumn("tasks", coalesce('tasks, 'submitRun_tasks)) - .withColumn("job_clusters", coalesce('job_clusters, 'submitRun_job_clusters)) - - } + }} + + // /** + // * Look up the cluster_name based on id first from + // * `job_status_silver`. If not present there fallback to latest + // * snapshot prior to the run + // */ + + // val jobRunsAppendClusterName = NamedTransformation { + // (df: DataFrame) => { + // val key = Seq( "timestamp", "organization_id", "clusterId") + // df.toTSDF( key:_*) + // .lookupWhen( clusterSpecNameLookup.toTSDF( key:_*)) + // .lookupWhen( clusterSnapNameLookup.toTSDF( key:_*)) + // .df + // } + // } + + + // /** + // * looks up the job name based on id first from job_status_silver and if not present there fallback to latest + // * snapshot prior to the run + // */ + // val jobRunsAppendJobMeta = NamedTransformation { + // (df: DataFrame) => { + // val key = Seq( "timestamp", "organization_id", "jobId") + // df.toTSDF( key:_*) + // .lookupWhen( jobStatusMetaLookup.toTSDF( key:_*)) + // .lookupWhen( jobSnapNameLookup.toTSDF( key:_*)) + // .df + // .withColumns( Map( + // "jobName" + // -> coalesce('jobName, 'run_name), + // "tasks" + // -> coalesce('tasks, 'submitRun_tasks), + // "job_clusters" + // -> coalesce('job_clusters, 'submitRun_job_clusters))) + // } + // } + + val jobRunsAppendTaskAndClusterDetails = NamedTransformation { (df: DataFrame) => { - def jobRunsAppendTaskAndClusterDetails(df: DataFrame): DataFrame = { val computeIsSQLWarehouse = $"task_detail.sql_task.warehouse_id".isNotNull val dfHasTasks = SchemaTools.nestedColExists(df.schema, "tasks") @@ -1114,70 +1102,8 @@ object WorkflowsTransforms extends SparkSessionWrapper { .drop("tasks") } } else df // tasks not present in schema - } - - /** - * It's imperative that all nested runs be nested within the jobRun record to ensure cost accuracy downstream in - * jrcp -- without it jrcp will double count costs as both the parent and the child will have an associated cost - * - * A "workflow" in this context isa dbutils.notebook.run execution -- it does spin up a job run in the back end - * and will have a workflow_context (field) with root_run_id and parent_run_id. All of these are rolled to the root - * to avoid the need to multi-layer joining. It's up to the customer to complete this as needed as the depths can - * get very large. - * - * An multi-task job (mtj) is a job that has at least one task identified (all jobs runs after the change in 2022). - * MTJs can execute notebooks which can also run nested workflows using dbutils.notebook.run. - * - * Workflows can be launched interactively from a notebook or through an mtj; thus it's necessary to account for - * both scenarios hence the double join in the last DF. - * - * Nested runs DO NOT mean tasks inside a jobrun as these are still considered root level tasks. A nested run - * is only launched via dbutils.notebook.run either manually or through an MTJ. - * - * It may be possible to utilize a single field to report both of these as it appears there can never be a child - * without a workflow child but the reverse is not true. This can be reviewed with customer to determine if this - * is a valid assumption and these can be coalesced, but for now, for safety, they are being kept separate until - * all scenarios can be identified - */ - def jobRunsRollupWorkflowsAndChildren(df: DataFrame): DataFrame = { - - // identify root level task runs - val rootTaskRuns = df - .filter('parentRunId.isNull && get_json_object('workflow_context, "$.root_run_id").isNull) - - // pull only workflow children as defined by having a workflow_context.root_run_id - val workflowChildren = df - .filter(get_json_object('workflow_context, "$.root_run_id").isNotNull) - - // prepare the nesting by getting keys and the entire record as a nested record - val workflowChildrenForNesting = workflowChildren - .withColumn("parentRunId", get_json_object('workflow_context, "$.root_run_id").cast("long")) - .withColumn("workflowChild", struct(workflowChildren.schema.fieldNames map col: _*)) - .groupBy('organization_id, 'parentRunId) - .agg(collect_list('workflowChild).alias("workflow_children")) - - // get all the children identified as having a parentRunId as they need to be rolled up - val children = df - .filter('parentRunId.isNotNull) - .join(workflowChildrenForNesting, Seq("organization_id", "parentRunId"), "left") - - // prepare the nesting by getting keys and the entire record as a nested record - val childrenForNesting = children - .withColumn("child", struct(children.schema.fieldNames map col: _*)) - .groupBy('organization_id, 'parentRunId) - .agg(collect_list('child).alias("children")) - .withColumnRenamed("parentRunId", "taskRunId") // for simple joining - - // deliver root task runs with workflows and children nested within the root - rootTaskRuns - .join(childrenForNesting, Seq("organization_id", "taskRunId"), "left") // workflows in mtjs - .join( - workflowChildrenForNesting.withColumnRenamed("parentRunId", "taskRunId"), // for simple joining - Seq("organization_id", "taskRunId"), - "left" - ) + }} - } /** * BEGIN JRCP Transforms diff --git a/src/main/scala/com/databricks/labs/overwatch/utils/Config.scala b/src/main/scala/com/databricks/labs/overwatch/utils/Config.scala index 054695eac..0ec4a61c0 100644 --- a/src/main/scala/com/databricks/labs/overwatch/utils/Config.scala +++ b/src/main/scala/com/databricks/labs/overwatch/utils/Config.scala @@ -160,7 +160,8 @@ class Config() { private[overwatch] def orderedOverwatchScope: Seq[OverwatchScope.Value] = { import OverwatchScope._ // jobs, clusters, clusterEvents, sparkEvents, pools, audit, passthrough, profiles - Seq(audit, notebooks, accounts, pools, clusters, clusterEvents, sparkEvents, jobs, dbsql,notebookCommands) + Seq(audit, notebooks, accounts, pools, clusters, clusterEvents, sparkEvents, jobs, + dbsql, warehouseEvents, notebookCommands) } def overwatchScope: Seq[OverwatchScope.Value] = _overwatchScope diff --git a/src/main/scala/com/databricks/labs/overwatch/utils/DataFrameSyntax.scala b/src/main/scala/com/databricks/labs/overwatch/utils/DataFrameSyntax.scala new file mode 100644 index 000000000..e713b06a2 --- /dev/null +++ b/src/main/scala/com/databricks/labs/overwatch/utils/DataFrameSyntax.scala @@ -0,0 +1,261 @@ +package com.databricks.labs.overwatch.utils + +import org.apache.log4j.{Level,Logger} +import org.apache.spark.sql.Dataset +import java.io.ByteArrayOutputStream +import scala.util.{ Try, Success, Failure} + +trait DataFrameSyntax[ SPARK <: SparkSessionWrapper] { + + self: SPARK => + + /** + * Spark's native `df.show()` produces formatted, tabular output on + * the console but provides no ability to capture that output for + * any programmatic manipulation. This trait implements two + * additional extension methods for Spark `Dataset[T]`s (including + * `DataFrame`s): + * + * - `df.showLines()` returns an `Iterator[String]` suitable for + * any programmatic manipulation + * + * - `df.log()` uses `.showLines()` internally to redirect the + * formatted output to a logger + * + * These methods are overloaded in way that closely mimics the + * behavior of Spark's built-in `Dataset.show()`. The exception is + * that `df.log()` defaults to vertical output, i.e. one line per + * field per row. Regular tabular output of wide + * `DataFrame`/`Dataset`s are likely to overwhelm the logger and/or + * impose severe performance penalties on the Overwatch + * application. + * + * Examples: + * + * Given: + * {{{ + * val debugDF: DataFrame = ??? + * }}} + * + * This produces "vertical" output on the console using Spark's native method: + * }}} + * debugDF.show( debugDF.count.toInt, 0, true) + * }}} + * + * {{{ + * -RECORD 0--------------------------------- + * organization_id | 2753962522174656 + * jobId | 903015066329560 + * fromMS | 1709419775381 + * . . . + * -RECORD 1--------------------------------- + * . . . + * }}} + * + * This captures the same text in an in-memory data structure: + * {{{ + * val lines: Iterator[String] = + * debugDF.showLines( debugDF.count.toInt, 0, true) + * }}} + * + * + * This, when called from within the `overwatch.pipelines.Silver` + * class, produces log entries according to the logging level + * specified by the Spark conf: + * + * {{{ + * spark.conf.set( "overwatch.dataframelogger.level", "OFF") + * + * debugDF.log( debugDF.count.toInt, 0, true) + * }}} + * + * {{{ + * + * }}} + * + * {{{ + * spark.conf.set( "overwatch.dataframelogger.level", "INFO") + * + * debugDF.log( debugDF.count.toInt, 0, true) + * }}} + * 24/05/04 00:18:16 INFO Silver: -RECORD 0--------------------------------- + * 24/05/04 00:18:16 INFO Silver: organization_id | 2753962522174656 + * 24/05/04 00:18:16 INFO Silver: jobId | 903015066329560 + * 24/05/04 00:18:16 INFO Silver: fromMS | 1709419775381 + * . . . + * 24/05/04 00:18:16 INFO Silver: -RECORD 1--------------------------------- + * . . . + * }}} + * + * This feature respects the logging level of the existing logger + * in the calling class. The value of + * `overwatch.dataframelogger.level` must be less than or equal to + * (in Log4J 1.x terms) that of the effective level of the logger + * in scope, otherwise the logs will only contain an informative + * message and no Spark action will occur. See the test suite + * implemntation below and log output for details. + * + */ + + private val logger: Logger = Logger.getLogger(this.getClass) + + implicit class DataFrameShower[T]( val df: Dataset[T]) { + + def showLines(): Iterator[String] = + showLines( 20) + + def showLines( numRows: Int): Iterator[String] = + showLines( numRows, truncate = true) + + def showLines( truncate: Boolean): Iterator[String] = + showLines( 20, truncate) + + def showLines( numRows: Int, truncate: Boolean): Iterator[String] = + showLines( numRows, if( truncate) 20 else 0, vertical= false) + + def showLines( numRows: Int, truncate: Int): Iterator[String] = + showLines( numRows, truncate, vertical= false) + + def showLines( + numRows: Int = 20, + truncate: Int = 20, + vertical: Boolean = false + ): Iterator[String] = { + val out = new ByteArrayOutputStream + Console.withOut( out) { + df.show( numRows, truncate, vertical) } + val lines = out.toString.linesIterator + out.reset() + lines + + } + } + + val dataFrameLoggerSparkConfKey = "overwatch.dataframelogger.level" + + def getDataFrameLoggerSparkConfValue(): Option[ String] = + spark.conf.getOption( dataFrameLoggerSparkConfKey) + + def getDataFrameLoggerLevel(): Level = Try { + getDataFrameLoggerSparkConfValue match { + case Some( v) => + Level.toLevel( v) + case None => + logger.log( Level.WARN, + s"No ${dataFrameLoggerSparkConfKey} set in Spark conf;" + + " default is OFF.") + Level.OFF + } + } match { + case Success( l) => { + logger.log( Level.DEBUG, s"DataFrameLogger Level is ${l}") + l + } + case Failure(_) => { + logger.log( Level.WARN, + s"${dataFrameLoggerSparkConfKey} value of ${getDataFrameLoggerSparkConfValue.get}" + + " is not a valid logger level;" + + " default is OFF.") + Level.OFF + } + } + + + implicit class DataFrameLogger[T]( val df: Dataset[T]) { + + // def log()(implicit level: Level): Unit = + // log( 20) + + def log( numRows: Int): Unit = //( implicit level: Level): Unit = + log( numRows= numRows, truncate= true) + + def log( truncate: Boolean): Unit = //( implicit level: Level): Unit = + log( truncate= if( truncate) 20 else 0) + + def log( numRows: Int, truncate: Boolean): Unit = //( implicit level: Level): Unit = + log( numRows= numRows, truncate= if( truncate) 20 else 0) + + def log( numRows: Int, truncate: Int): Unit = //( implicit level: Level): Unit = + log( numRows, truncate, vertical= true) + + // Not providing any signature with a `Level` for now. + // Developers must get or set the Spark conf option above. + // + // def log( level: Level, numRows: Int = 20, truncate: Int = 20, vertical: Boolean = true) = ??? + + def log( + numRows : Int = 20, + truncate: Int = 20, + vertical: Boolean = true + ): Unit = { // ( implicit level: Level): Unit = { + + // import DataFrameLogger._ + + // @transient lazy val level = getDataFrameLoggerLevel() + + logger.log( Level.INFO, + s"""| Overwatch DataFrame logger Spark configuration: + | `'${dataFrameLoggerSparkConfKey}'` -> + | `'${getDataFrameLoggerSparkConfValue.getOrElse("")}'`""" + .stripMargin.linesIterator.mkString) + + logger.log( Level.INFO, + s"""| Overwatch DataFrame logger parameters: + | `level= ${getDataFrameLoggerLevel()}, + | numRows= ${numRows}, + | truncate= ${truncate}, + | vertical= ${vertical}`""" + .stripMargin.linesIterator.mkString) + + + + // level + getDataFrameLoggerLevel() match { + + case Level.OFF => + + logger.log( Level.INFO, + s"""| Overwatch DataFrame logging is disabled; either set Spark + | conf option "${dataFrameLoggerSparkConfKey}" to a defined logging level (e.g. + | "WARN", "INFO", etc.), less than or equal to "${logger.getLevel}" + | set in class `${this.getClass.getSimpleName}` or use Spark's + | native `DataFrame.show()` method to produce console output.""" + .stripMargin.linesIterator.mkString) + + + case l if logger.getLevel.isGreaterOrEqual( l) => { + + if( truncate == 0) logger.log( Level.WARN, + """| Overwatch DataFrame logging is enabled but without value truncation; + | long `STRING` columns and large/nested `MAP` or `STRUCT` columns are likely + | to produce excessive logger output. Please consider using truncation. The + | default is 20 characters per column.""" + .stripMargin.linesIterator.mkString) + + if( !vertical) logger.log( Level.WARN, + """| Overwatch `DataFrame` logging is enabled but without vertical formatting; + | tabular output of > 5 columns is likely to produce excessive logger output. + | Please consider using vertical formatting (the default).""" + .stripMargin.linesIterator.mkString) + + df.showLines( numRows, truncate, vertical) + .foreach( logger.log( l, _)) + + } + + case l => + + logger.log( logger.getLevel, + s"""| Overwatch DataFrame logging is enabled and set to ${l} but the + | logger level ${logger.getLevel} set in class ${this.getClass.getSimpleName} + | is higher; no records will appear in the logger output.""" + .stripMargin.linesIterator.mkString) + + + } + + } + + } + +} diff --git a/src/main/scala/com/databricks/labs/overwatch/utils/PipelineValidationHelper.scala b/src/main/scala/com/databricks/labs/overwatch/utils/PipelineValidationHelper.scala new file mode 100644 index 000000000..ae1941fbb --- /dev/null +++ b/src/main/scala/com/databricks/labs/overwatch/utils/PipelineValidationHelper.scala @@ -0,0 +1,712 @@ +package com.databricks.labs.overwatch.utils + +import com.databricks.labs.overwatch.pipeline.Pipeline +import org.apache.log4j.{Level, Logger} +import org.apache.spark.sql.functions._ +import com.databricks.labs.overwatch.pipeline._ +import com.databricks.labs.overwatch.pipeline.TransformFunctions._ +import com.databricks.labs.validation._ +import org.apache.spark.sql.{Column, DataFrame} +import org.apache.spark.sql.expressions.Window + +import java.time.LocalDateTime +import scala.collection.mutable.ArrayBuffer + +/** + * This class contains the utility functions for PipelineValidation.scala. + */ + +class PipelineValidationHelper(_etlDB: String ,_allRun: Boolean = true) extends SparkSessionWrapper { + + import spark.implicits._ + + private val logger: Logger = Logger.getLogger(this.getClass) + + private var _pipelineSnapTime: Long = _ + + private val _healthCheck_id: String = java.util.UUID.randomUUID.toString + private val _quarantine_id: String = java.util.UUID.randomUUID.toString + + private def healthCheckID: String = _healthCheck_id + + def quarantineID: String = _quarantine_id + + def etlDB: String = _etlDB + val allRun: Boolean = _allRun + + private var _validations: ArrayBuffer[HealthCheckReport] = new ArrayBuffer[HealthCheckReport]() + private var _quarantine: ArrayBuffer[QuarantineReport] = new ArrayBuffer[QuarantineReport]() + + def validations: ArrayBuffer[HealthCheckReport] = _validations + + def quarantine: ArrayBuffer[QuarantineReport] = _quarantine + + val _isOverwatchDB: Boolean = spark.sessionState.catalog.getDatabaseMetadata(etlDB).properties.getOrElse("OVERWATCHDB", "FALSE").toBoolean + + private def isOverwatchDB = _isOverwatchDB + + private val workSpace = if (isOverwatchDB) { + println(s"$etlDB is Overwatch Database and Suitable for Pipeline Validation") + Helpers.getWorkspaceByDatabase(etlDB) + } else { + val errMsg = s"${etlDB} is Not Overwatch Database.Pipeline Validation can only work on Overwatch Database Tables. Validation aborted!!!" + throw new BadConfigException(errMsg) + } + + val validPrefixes = List("dbfs:", "s3", "abfss", "gs") + val storagePrefix: String = workSpace.getConfig.etlDataPathPrefix + var healthCheckBasePath = storagePrefix.replace("global_share", "") + "healthCheck" + if (!validPrefixes.exists(healthCheckBasePath.startsWith)) { + healthCheckBasePath = s"dbfs:$healthCheckBasePath" + } + + val healthCheckReportPath = s"""$healthCheckBasePath/heathCheck_report""" + val quarantineReportPath = s"""$healthCheckBasePath/quarantine_report""" + + private val All_Overwatch_RunID = if (spark.catalog.tableExists(s"$etlDB.pipeline_report")) { + spark.read.table(s"$etlDB.pipeline_report") + .select("Overwatch_RunID").distinct().collect().map(_.getString(0)) + }else{ + val errMsg = s"pipeline_report is not present in $etlDB. To proceed with pipeline_validation , pipeline_report table needs to be present in the database.Pipeline Validation aborted!!!" + throw new BadConfigException(errMsg) + } + + val Overwatch_RunIDs :Array[String] = if (allRun) { + println("All Runs are getting validated") + All_Overwatch_RunID + }else { + if (Helpers.pathExists(healthCheckReportPath)) { + val healthCheckDF = spark.read.load(healthCheckReportPath) + val Validated_Overwatch_RunIDs = healthCheckDF.select("Overwatch_RunID").distinct().collect().map(_.getString(0)) + All_Overwatch_RunID.diff(Validated_Overwatch_RunIDs) + } else { + All_Overwatch_RunID + } + } + + private val gold = Gold(workSpace) + private val goldTargets = gold.GoldTargets + + + val jrcpKey: Array[String] = getKeysAndPartitionBy(goldTargets.jobRunCostPotentialFactTarget) + val clsfKey: Array[String] = getKeysAndPartitionBy(goldTargets.clusterStateFactTarget) + val jobRunKey: Array[String] = getKeysAndPartitionBy(goldTargets.jobRunTarget) + val nbkey: Array[String] = getKeysAndPartitionBy(goldTargets.notebookTarget) + val nbcmdkey: Array[String] = getKeysAndPartitionBy(goldTargets.notebookCommandsTarget) + val clusterKey: Array[String] = getKeysAndPartitionBy(goldTargets.clusterTarget) + val sparkJobKey: Array[String] = getKeysAndPartitionBy(goldTargets.sparkJobTarget) + val sqlQueryHistKey: Array[String] = getKeysAndPartitionBy(goldTargets.sqlQueryHistoryTarget) + val jobKey: Array[String] = getKeysAndPartitionBy(goldTargets.jobTarget) + + + val jrcpTable: String = goldTargets.jobRunCostPotentialFactTarget.name + val clsfTable: String = goldTargets.clusterStateFactTarget.name + val jobRunTable: String = goldTargets.jobRunTarget.name + val nbTable: String = goldTargets.notebookTarget.name + val nbcmdTable: String = goldTargets.notebookCommandsTarget.name + val clusterTable: String = goldTargets.clusterTarget.name + val sparkJobTable: String = goldTargets.sparkJobTarget.name + val sqlQueryHistTable: String = goldTargets.sqlQueryHistoryTarget.name + val jobTable: String = goldTargets.jobTarget.name + + val filterCondition = 'Overwatch_RunID.isin(Overwatch_RunIDs:_*) + val jrcpDF: DataFrame = getTableDF(etlDB, jrcpTable, filterCondition) + val clsfDF: DataFrame = getTableDF(etlDB, clsfTable, filterCondition) + val jobRunDF: DataFrame = getTableDF(etlDB, jobRunTable, filterCondition) + val nbDF: DataFrame = getTableDF(etlDB, nbTable, filterCondition) + val nbcmdDF: DataFrame = getTableDF(etlDB, nbcmdTable, filterCondition) + val clusterDF: DataFrame = getTableDF(etlDB, clusterTable, filterCondition) + val sparkJobDF: DataFrame = getTableDF(etlDB, sparkJobTable, filterCondition) + val sqlQueryHistDF: DataFrame = getTableDF(etlDB, sqlQueryHistTable, filterCondition) + val jobDF: DataFrame = getTableDF(etlDB, jobTable, filterCondition) + + private[overwatch] def pipelineSnapTime: TimeTypes = { + Pipeline.createTimeDetail(_pipelineSnapTime) + } + + private[overwatch] def setPipelineSnapTime(): this.type = { + _pipelineSnapTime = LocalDateTime.now(Pipeline.systemZoneId).toInstant(Pipeline.systemZoneOffset).toEpochMilli + logger.log(Level.INFO, s"INIT: Pipeline Snap TS: ${pipelineSnapTime.asUnixTimeMilli}-${pipelineSnapTime.asTSString}") + this + } + + def validateNotNull(ruleName: String, configColumns: String): Rule = { + Rule(ruleName, col(configColumns).isNotNull) + } + + def validateGreaterThanZero(ruleName: String, configColumns: String): Rule = { + Rule(ruleName, col(configColumns) > lit(0)) + } + + def validateLEQOne(ruleName: String, configColumns: String): Rule = { + Rule(ruleName, col(configColumns) <= lit(1)) + } + + def checkRunningDays(ruleName: String, configColumns: String): Rule = { + Rule(ruleName, col(configColumns) === 2) + } + + def checkColumnInValues(ruleName: String, configColumns: String, value: Array[String]): Rule = { + Rule(ruleName, col(configColumns).isin(value: _*)) + } + + def getKeysAndPartitionBy(pipelineTable: PipelineTable): Array[String] = { + pipelineTable._keys ++ pipelineTable.partitionBy + } + + def getTableDF(etlDB: String, tableName: String, filterCondition: Column): DataFrame = { + if (spark.catalog.tableExists(s"$etlDB.$tableName")) { + spark.read.table(s"$etlDB.$tableName").filter(filterCondition) + } else { + spark.emptyDataFrame + } + } + + def validateRuleAndUpdateStatus( + validateNullRuleSet: RuleSet, + table_name: String, + keys: Array[String], + validationStatus: ArrayBuffer[HealthCheckReport], + quarantineStatus: ArrayBuffer[QuarantineReport], + validationType: String = "", + Overwatch_RunID:String + ): (ArrayBuffer[HealthCheckReport], ArrayBuffer[QuarantineReport]) = { + + val vStatus: ArrayBuffer[HealthCheckReport] = new ArrayBuffer[HealthCheckReport]() + val qStatus: ArrayBuffer[QuarantineReport] = new ArrayBuffer[QuarantineReport]() + + val validation = validateNullRuleSet.validate() + + val completeReportDF = validation.completeReport + + validateNullRuleSet.getRules.foreach(elem => { + val colName = elem.inputColumn.toString.split("\\(")(1).split("\\)")(0).split(" ")(0) + val healthCheckRuleColumn = elem.ruleName + val dfWithNegativeValidation = if (validationType == "validate_leq_one") { + completeReportDF.filter((col(s"$healthCheckRuleColumn.passed") === false)).select(keys.map(col): _*) + }else{ + completeReportDF.filter((col(s"$healthCheckRuleColumn.passed") === false) || + col(s"$healthCheckRuleColumn.passed").isNull).select(keys.map(col): _*) + } + + val countOfNegativeValidation = dfWithNegativeValidation.count() + if (validationType.toLowerCase() == "validate_not_required"){ + val healthCheckMsg = s"Validation is not required for ${table_name} for Overwatch_RunID ${Overwatch_RunID}. The Table doesn't contain any data" + vStatus.append(HealthCheckReport(etlDB, table_name, healthCheckRuleColumn,"Single_Table_Validation", Some(healthCheckMsg), Overwatch_RunID)) + } + else if (countOfNegativeValidation == 0) { + val healthCheckMsg = "Success" + vStatus.append(HealthCheckReport(etlDB, table_name, healthCheckRuleColumn,"Single_Table_Validation", Some(healthCheckMsg), Overwatch_RunID)) + } else { + val (healthCheckMsg: String, healthCheckType: String) = validationType.toLowerCase() match { + case "validate_greater_than_zero" => + (s"HealthCheck Failed: got $countOfNegativeValidation ${colName}s which are not greater than zero or is NULL", "Failure") + case "validate_not_null" => + (s"HealthCheck Failed: got $countOfNegativeValidation ${colName}s which are null", "Failure") + case "validate_leq_one" => + (s"HealthCheck Failed: got $countOfNegativeValidation ${colName}s which are greater than 1", "Failure") + case "validate_values_in_between" => + (s"HealthCheck Warning: got $countOfNegativeValidation ${colName}s which are not in between expected values", "Warning") + case _ => + (s"HealthCheck Warning : got $countOfNegativeValidation ${colName}s which are greater than 2", "Warning") + } + vStatus.append(HealthCheckReport(etlDB, table_name, healthCheckRuleColumn,"Single_Table_Validation",Some(healthCheckMsg), Overwatch_RunID)) + dfWithNegativeValidation.toJSON.collect().foreach(jsonString => { + qStatus.append(QuarantineReport(etlDB, table_name, healthCheckRuleColumn,"Single_Table_Validation", healthCheckType, jsonString)) + }) + } + }) + validationStatus ++= vStatus + quarantineStatus ++= qStatus + + (validationStatus, quarantineStatus) + } + + /** + * Function to Validate relation between 2 OW tables. Check whether we have proper data consistency between 2 tables. + * @param source : Source OW Table + * @param target : Target OW Table + * @param sourceDF : Dataframe created from Source OW Table + * @param targetDF : Dataframe created from Target OW Table + * @param column : Column on which Data consistency would be validated between 2 tables. + * @param key : Key Column in Source Tables. Would be used for Reporting Purpose. + * @param validationStatus : Validation Status Array for Validation Status Report + * @param quarantineStatus : Quarantine Status Array for Quarantine Report + * @return + */ + def validateColumnBetweenMultipleTable( + source: PipelineTable, + target: PipelineTable, + sourceDF: DataFrame, + targetDF: DataFrame, + column: String, + key: Array[String], + validationStatus: ArrayBuffer[HealthCheckReport], + quarantineStatus: ArrayBuffer[QuarantineReport], + Overwatch_RunID:String + ): (ArrayBuffer[HealthCheckReport], ArrayBuffer[QuarantineReport]) = { + + val vStatus: ArrayBuffer[HealthCheckReport] = new ArrayBuffer[HealthCheckReport]() + val qStatus: ArrayBuffer[QuarantineReport] = new ArrayBuffer[QuarantineReport]() + val sourceTable = source.name + val targetTable = target.name + + if (spark.catalog.tableExists(s"$etlDB.$sourceTable") && spark.catalog.tableExists(s"$etlDB.$targetTable")){ + val ruleName = s"${column.toUpperCase()}_Present_In_${sourceTable}_But_Not_In_$targetTable" + if (sourceDF.count() == 0 || targetDF.count() == 0) { + val msg = s"Cross table validation between source $sourceTable and target $targetTable is not possible for Overwatch_RunID $Overwatch_RunID as either of them doesn't contain any data" + vStatus.append(HealthCheckReport(etlDB, targetTable, ruleName,"Cross_Table_Validation", Some(msg), Overwatch_RunID)) + logger.log(Level.WARN,msg) + return (validationStatus, quarantineStatus) + }else{ + // In Case of NotebookCommands Table we should only consider the workspaces where verbose auditlog is enabled + val joinedDF = if (source.name == nbcmdTable || target.name == nbcmdTable){ + val organizationID_list : Array[String] = spark.sql(s"select distinct organization_id from $etlDB.$nbcmdTable").collect().map(_.getString(0)) + sourceDF.filter('organization_id.isin(organizationID_list:_*)).join(targetDF.filter('organization_id.isin(organizationID_list:_*)), Seq(column), "anti").select(key.map(col): _*) + }else{ + sourceDF.join(targetDF, Seq(column), "anti").select(key.map(col): _*) + } + val joinedDFCount = joinedDF.count() + if (joinedDFCount == 0) { + val healthCheckMsg = s"HealthCheck Success: There are $joinedDFCount ${column}s that are present in $sourceTable but not in $targetTable" + vStatus.append(HealthCheckReport(etlDB, targetTable, ruleName,"Cross_Table_Validation", Some(healthCheckMsg), Overwatch_RunID)) + } else { + val healthCheckMsg = s"HealthCheck Warning: There are $joinedDFCount ${column}s that are present in $sourceTable but not in $targetTable" + vStatus.append(HealthCheckReport(etlDB, targetTable, ruleName, "Cross_Table_Validation", Some(healthCheckMsg), Overwatch_RunID)) + joinedDF.toJSON.collect().foreach(jsonString => { + qStatus.append(QuarantineReport(etlDB, targetTable, ruleName, "Cross_Table_Validation", "Warning", jsonString)) + }) + } + } + (validationStatus ++= vStatus, quarantineStatus ++= qStatus) + }else{ + val msg = s"Cross table validation between source $sourceTable and target $targetTable is not possible as either of them doesn't exist in the database" + println(msg) + logger.log(Level.WARN,msg) + (validationStatus, quarantineStatus) + } + } + + private[overwatch] def validateCLSF(): (ArrayBuffer[HealthCheckReport], ArrayBuffer[QuarantineReport]) = { + + var validationStatus: ArrayBuffer[HealthCheckReport] = new ArrayBuffer[HealthCheckReport]() + var quarantineStatus: ArrayBuffer[QuarantineReport] = new ArrayBuffer[QuarantineReport]() + + val tableName = clsfTable + val key = clsfKey + + val validateRules = Seq[Rule]( + validateNotNull("Cluster_ID_Should_Not_be_NULL", "cluster_id"), + validateNotNull("Driver_Node_Type_ID_Should_Not_be_NULL", "driver_node_type_id"), + validateNotNull("Node_Type_ID_Should_Not_be_NULL_for_Multi_Node_Cluster", "node_type_id"), + validateGreaterThanZero("DBU_Rate_Should_Be_Greater_Than_Zero_for_Runtime_Engine_is_Standard_Or_Photon", "dbu_rate"), + validateGreaterThanZero("Total_Cost_Should_Be_Greater_Than_Zero_for_Databricks_Billable", "total_cost"), + checkRunningDays("Check_Whether_Any_Single_Cluster_State_is_Running_For_Multiple_Days", "days_in_state") + ) + + Overwatch_RunIDs.foreach(Overwatch_RunID =>{ + val clsf_df : DataFrame= clsfDF.filter('Overwatch_RunID === Overwatch_RunID) + if (clsf_df.count() == 0) { + (validationStatus, quarantineStatus) == validateRuleAndUpdateStatus( + RuleSet(clsf_df).add(validateRules), + tableName, key, validationStatus, quarantineStatus, "validate_not_required", Overwatch_RunID) + }else { + println(s"${tableName} is getting validated for Overwatch_RunID ${Overwatch_RunID}") + (validationStatus, quarantineStatus) == validateRuleAndUpdateStatus( + RuleSet(clsf_df).add(validateRules.take(2)), + tableName, key, validationStatus, quarantineStatus, "validate_not_null", Overwatch_RunID) + + (validationStatus, quarantineStatus) == validateRuleAndUpdateStatus( + RuleSet(clsf_df.where("target_num_workers != 0")).add(validateRules(2)), + tableName, key, validationStatus, quarantineStatus, "validate_not_null", Overwatch_RunID) + + (validationStatus, quarantineStatus) == validateRuleAndUpdateStatus( + RuleSet(clsf_df.where("runtime_engine IN ('STANDARD','PHOTON')")).add(validateRules(3)), + tableName, key, validationStatus, quarantineStatus, "validate_greater_than_zero", Overwatch_RunID) + + (validationStatus, quarantineStatus) == validateRuleAndUpdateStatus( + RuleSet(clsf_df.where("cluster_name is not null") + .where("databricks_billable is true") + .where("custom_tags not like '%SqlEndpointId%'") + .where("unixTimeMS_state_end > unixTimeMS_state_start")).add(validateRules(4)), + tableName, key, validationStatus, quarantineStatus, "validate_greater_than_zero", Overwatch_RunID) + + (validationStatus, quarantineStatus) == validateRuleAndUpdateStatus( + RuleSet(clsf_df).add(validateRules(5)), + tableName, key, validationStatus, quarantineStatus, "", Overwatch_RunID) + } + }) + (validations ++= validationStatus, quarantine ++= quarantineStatus) + } + + private[overwatch] def validateJRCP(): (ArrayBuffer[HealthCheckReport], ArrayBuffer[QuarantineReport]) = { + var validationStatus: ArrayBuffer[HealthCheckReport] = new ArrayBuffer[HealthCheckReport]() + var quarantineStatus: ArrayBuffer[QuarantineReport] = new ArrayBuffer[QuarantineReport]() + + val tableName = jrcpTable + val key = jrcpKey + + val validateRules = Seq[Rule]( + validateNotNull("Job_ID_Should_Not_be_NULL", "job_id"), + validateNotNull("Driver_Node_Type_ID_Should_Not_be_NULL", "driver_node_type_id"), + validateLEQOne("Job_Run_Cluster_Util_value_Should_Not_Be_More_Than_One", "Job_run_cluster_util"), + checkRunningDays("Check_Whether_Any_Job_is_Running_For_Multiple_Days", "days_in_running") + ) + + Overwatch_RunIDs.foreach(Overwatch_RunID =>{ + val jrcp_df = jrcpDF.filter('Overwatch_RunID === Overwatch_RunID) + .withColumn("days_in_running", size(col("running_days"))) + if (jrcp_df.count() == 0) { + (validationStatus, quarantineStatus) == validateRuleAndUpdateStatus( + RuleSet(jrcp_df).add(validateRules), + tableName, key, validationStatus, quarantineStatus, "validate_not_required", Overwatch_RunID) + }else { + println(s"${tableName} is getting validated for Overwatch_RunID ${Overwatch_RunID}") + (validationStatus, quarantineStatus) == validateRuleAndUpdateStatus( + RuleSet(jrcp_df).add(validateRules.take(2)), + tableName, key, validationStatus, quarantineStatus, "validate_not_null", Overwatch_RunID) + + (validationStatus, quarantineStatus) == validateRuleAndUpdateStatus( + RuleSet(jrcp_df).add(validateRules(2)), + tableName, key, validationStatus, quarantineStatus, "validate_leq_one", Overwatch_RunID) + + (validationStatus, quarantineStatus) == validateRuleAndUpdateStatus( + RuleSet(jrcp_df).add(validateRules(3)), + tableName, key, validationStatus, quarantineStatus, "", Overwatch_RunID) + } + }) + (validations ++= validationStatus, quarantine ++= quarantineStatus) + } + + private[overwatch] def validateCluster(): (ArrayBuffer[HealthCheckReport], ArrayBuffer[QuarantineReport]) = { + var validationStatus: ArrayBuffer[HealthCheckReport] = new ArrayBuffer[HealthCheckReport]() + var quarantineStatus: ArrayBuffer[QuarantineReport] = new ArrayBuffer[QuarantineReport]() + + val tableName = clusterTable + val key = clusterKey + + val validateRules = Seq[Rule]( + validateNotNull("Cluster_ID_Should_Not_be_NULL", "cluster_id"), + validateNotNull("Driver_Node_Type_ID_Should_Not_be_NULL", "driver_node_type"), + validateNotNull("Node_Type_ID_Should_Not_be_NULL_for_Multi_Node_Cluster", "node_type"), + checkColumnInValues("Cluster_Type_Should_be_In_Between_Serverless_SQL-Analytics_Single-Node_Standard_High-Concurrency", "cluster_type" + , Array("Serverless", "SQL Analytics", "Single Node", "Standard", "High-Concurrency")) + ) + + Overwatch_RunIDs.foreach(Overwatch_RunID =>{ + val cluster_df = clusterDF.filter('Overwatch_RunID === Overwatch_RunID) + if (cluster_df.count() == 0) { + (validationStatus, quarantineStatus) == validateRuleAndUpdateStatus( + RuleSet(cluster_df).add(validateRules), + tableName, key, validationStatus, quarantineStatus, "validate_not_required", Overwatch_RunID) + } else { + println(s"${tableName} is getting validated for Overwatch_RunID ${Overwatch_RunID}") + (validationStatus, quarantineStatus) == validateRuleAndUpdateStatus( + RuleSet(cluster_df).add(validateRules.take(2)), + tableName, key, validationStatus, quarantineStatus, "validate_not_null", Overwatch_RunID) + + (validationStatus, quarantineStatus) == validateRuleAndUpdateStatus( + RuleSet(cluster_df.where("num_workers != 0")).add(validateRules(2)), + tableName, key, validationStatus, quarantineStatus, "validate_not_null", Overwatch_RunID) + + (validationStatus, quarantineStatus) == validateRuleAndUpdateStatus( + RuleSet(cluster_df).add(validateRules(3)), + tableName, key, validationStatus, quarantineStatus, "validate_values_in_between", Overwatch_RunID) + } + }) + (validations ++= validationStatus, quarantine ++= quarantineStatus) + } + + private[overwatch] def validateSparkJob(): (ArrayBuffer[HealthCheckReport], ArrayBuffer[QuarantineReport]) = { + var validationStatus: ArrayBuffer[HealthCheckReport] = new ArrayBuffer[HealthCheckReport]() + var quarantineStatus: ArrayBuffer[QuarantineReport] = new ArrayBuffer[QuarantineReport]() + + val tableName = sparkJobTable + val key = sparkJobKey + + val validateRules = Seq[Rule]( + validateNotNull("Cluster_ID_Should_Not_be_NULL", "cluster_id"), + validateNotNull("Job_ID_Should_Not_be_NULL", "job_id"), + validateNotNull("db_id_in_job_Should_Not_be_NULL_When_db_Job_Id_is_Not_NULL", "db_id_in_job") + ) + + Overwatch_RunIDs.foreach(Overwatch_RunID => { + val sparkJob_df = sparkJobDF.filter('Overwatch_RunID === Overwatch_RunID) + if (sparkJob_df.count() == 0) { + (validationStatus, quarantineStatus) == validateRuleAndUpdateStatus( + RuleSet(sparkJob_df).add(validateRules), + tableName, key, validationStatus, quarantineStatus, "validate_not_required", Overwatch_RunID) + } else { + println(s"${tableName} is getting validated for Overwatch_RunID ${Overwatch_RunID}") + (validationStatus, quarantineStatus) == validateRuleAndUpdateStatus( + RuleSet(sparkJob_df).add(validateRules.take(2)), + tableName, key, validationStatus, quarantineStatus, "validate_not_null", Overwatch_RunID) + + (validationStatus, quarantineStatus) == validateRuleAndUpdateStatus( + RuleSet(sparkJob_df.where("db_job_id is not NULL")).add(validateRules(2)), + tableName, key, validationStatus, quarantineStatus, "validate_not_null", Overwatch_RunID) + } + }) + (validations ++= validationStatus, quarantine ++= quarantineStatus) + } + + private[overwatch] def validateSqlQueryHist(): (ArrayBuffer[HealthCheckReport], ArrayBuffer[QuarantineReport]) = { + var validationStatus: ArrayBuffer[HealthCheckReport] = new ArrayBuffer[HealthCheckReport]() + var quarantineStatus: ArrayBuffer[QuarantineReport] = new ArrayBuffer[QuarantineReport]() + + val tableName = sqlQueryHistTable + + val validateRules = Seq[Rule]( + validateNotNull("Warehouse_ID_Should_Not_be_NULL", "warehouse_id"), + validateNotNull("Query_ID_Should_Not_be_NULL", "query_id") + ) + + Overwatch_RunIDs.foreach(Overwatch_RunID => { + val sqlQueryHist_df = sqlQueryHistDF.filter('Overwatch_RunID === Overwatch_RunID) + if (sqlQueryHist_df.count() == 0) { + (validationStatus, quarantineStatus) == validateRuleAndUpdateStatus( + RuleSet(sqlQueryHist_df).add(validateRules), + tableName, sqlQueryHistKey, validationStatus, quarantineStatus, "validate_not_required", Overwatch_RunID) + } else { + println(s"${tableName} is getting validated for Overwatch_RunID ${Overwatch_RunID}") + (validationStatus, quarantineStatus) == validateRuleAndUpdateStatus( + RuleSet(sqlQueryHist_df).add(validateRules), + tableName, sqlQueryHistKey, validationStatus, quarantineStatus, "validate_not_null", Overwatch_RunID) + } + }) + (validations ++= validationStatus, quarantine ++= quarantineStatus) + } + + private[overwatch] def validateJobRun(): (ArrayBuffer[HealthCheckReport], ArrayBuffer[QuarantineReport]) = { + var validationStatus: ArrayBuffer[HealthCheckReport] = new ArrayBuffer[HealthCheckReport]() + var quarantineStatus: ArrayBuffer[QuarantineReport] = new ArrayBuffer[QuarantineReport]() + + val tableName = jobRunTable + val key = jobRunKey + + val validateRules = Seq[Rule]( + validateNotNull("Job_ID_Should_Not_be_NULL", "job_id"), + validateNotNull("Run_ID_Should_Not_be_NULL", "run_id"), + validateNotNull("Job_Run_ID_Should_Not_be_NULL", "job_run_id"), + validateNotNull("Task_Run_ID_Should_Not_be_NULL", "task_run_id"), + validateNotNull("Cluster_ID_Should_Not_be_NULL", "cluster_id"), + ) + Overwatch_RunIDs.foreach(Overwatch_RunID => { + val jobRun_df = jobRunDF.filter('Overwatch_RunID === Overwatch_RunID) + if (jobRun_df.count() == 0) { + (validationStatus, quarantineStatus) == validateRuleAndUpdateStatus( + RuleSet(jobRun_df).add(validateRules), + tableName, key, validationStatus, quarantineStatus, "validate_not_required", Overwatch_RunID) + } else { + println(s"${tableName} is getting validated for Overwatch_RunID ${Overwatch_RunID}") + (validationStatus, quarantineStatus) == validateRuleAndUpdateStatus( + RuleSet(jobRun_df).add(validateRules.take(4)), + tableName, key, validationStatus, quarantineStatus, "validate_not_null", Overwatch_RunID) + + (validationStatus, quarantineStatus) == validateRuleAndUpdateStatus( + RuleSet(jobRun_df.filter(!'task_type.isin("sqlalert", "sqldashboard", "pipeline"))).add(validateRules(4)), + tableName, key, validationStatus, quarantineStatus, "validate_not_null", Overwatch_RunID) + } + }) + (validations ++= validationStatus, quarantine ++= quarantineStatus) + } + + private[overwatch] def validateJob(): (ArrayBuffer[HealthCheckReport], ArrayBuffer[QuarantineReport]) = { + var validationStatus: ArrayBuffer[HealthCheckReport] = new ArrayBuffer[HealthCheckReport]() + var quarantineStatus: ArrayBuffer[QuarantineReport] = new ArrayBuffer[QuarantineReport]() + + val tableName = jobTable + val key = jobKey + + val validateRules = Seq[Rule]( + validateNotNull("Job_ID_Should_Not_be_NULL", "job_id"), + checkColumnInValues("Action_Should_be_In_Between_snapimpute_create_reset_update_delete_resetJobAcl_changeJobAcl", "action" + , Array("snapimpute", "create", "reset", "update", "delete", "resetJobAcl", "changeJobAcl")) + ) + + Overwatch_RunIDs.foreach(Overwatch_RunID => { + val job_df = jobDF.filter('Overwatch_RunID === Overwatch_RunID) + if (job_df.count() == 0) { + (validationStatus, quarantineStatus) == validateRuleAndUpdateStatus( + RuleSet(job_df).add(validateRules), + tableName, key, validationStatus, quarantineStatus, "validate_not_required", Overwatch_RunID) + } else { + println(s"${tableName} is getting validated for Overwatch_RunID ${Overwatch_RunID}") + (validationStatus, quarantineStatus) == validateRuleAndUpdateStatus( + RuleSet(job_df).add(validateRules.head), + tableName, key, validationStatus, quarantineStatus, "validate_not_null", Overwatch_RunID) + + (validationStatus, quarantineStatus) == validateRuleAndUpdateStatus( + RuleSet(job_df).add(validateRules(1)), + tableName, key, validationStatus, quarantineStatus, "validate_values_in_between", Overwatch_RunID) + } + }) + (validations ++= validationStatus, quarantine ++= quarantineStatus) + } + + def checkPipelineModules ( + resultDF: DataFrame, + table_name: String, + validationStatus: ArrayBuffer[HealthCheckReport], + quarantineStatus: ArrayBuffer[QuarantineReport], + Overwatch_RunID: String, + status: String + ) : (ArrayBuffer[HealthCheckReport], ArrayBuffer[QuarantineReport]) = { + + val vStatus: ArrayBuffer[HealthCheckReport] = new ArrayBuffer[HealthCheckReport]() + val qStatus: ArrayBuffer[QuarantineReport] = new ArrayBuffer[QuarantineReport]() + + val healthCheckRuleColumn = f"Check If Any Module is ${status}" + + val dfWithNegativeValidation = resultDF.filter('status.startsWith(status)) + val countOfNegativeValidation = dfWithNegativeValidation.count() + if (countOfNegativeValidation == 0) { + val healthCheckMsg = "Success" + vStatus.append(HealthCheckReport(etlDB, table_name, healthCheckRuleColumn, "Pipeline_Report_Validation", Some(healthCheckMsg), Overwatch_RunID)) + }else{ + val healthCheckMsg = s"HealthCheck Warning: got $countOfNegativeValidation ${status} Module" + vStatus.append(HealthCheckReport(etlDB, table_name, healthCheckRuleColumn, "Pipeline_Report_Validation", Some(healthCheckMsg), Overwatch_RunID)) + dfWithNegativeValidation.toJSON.collect().foreach(jsonString => { + qStatus.append(QuarantineReport(etlDB, table_name, healthCheckRuleColumn, "Pipeline_Report_Validation", "Warning", jsonString)) + }) + } + validationStatus ++= vStatus + quarantineStatus ++= qStatus + + (validationStatus, quarantineStatus) + } + + private[overwatch] def validatePipelineTable(): (ArrayBuffer[HealthCheckReport], ArrayBuffer[QuarantineReport]) ={ + var validationStatus: ArrayBuffer[HealthCheckReport] = new ArrayBuffer[HealthCheckReport]() + var quarantineStatus: ArrayBuffer[QuarantineReport] = new ArrayBuffer[QuarantineReport]() + + Overwatch_RunIDs.foreach(Overwatch_RunID => { + val pipeline_df = spark.table(s"$etlDB.pipeline_report").filter('Overwatch_RunID === Overwatch_RunID) + val windowSpec = Window.partitionBy("organization_id","moduleID","Overwatch_RunID").orderBy('Pipeline_SnapTS.desc) + val resultDF = pipeline_df.select( + col("organization_id"), + col("workspace_name"), + col("moduleID"), + col("moduleName"), + col("fromTS"), + col("untilTS"), + col("Overwatch_RunID"), + substring(col("status"), 0, 200).alias("status"), + col("Pipeline_SnapTS"), + rank().over(windowSpec).alias("rank1") + ) + + (validationStatus, quarantineStatus) == checkPipelineModules(resultDF,"pipeline_report",validationStatus,quarantineStatus,Overwatch_RunID,"EMPTY") + (validationStatus, quarantineStatus) == checkPipelineModules(resultDF,"pipeline_report",validationStatus,quarantineStatus,Overwatch_RunID,"FAILED") + }) + + (validations ++= validationStatus, quarantine ++= quarantineStatus) + } + + private[overwatch] def validateCrossTable(): (ArrayBuffer[HealthCheckReport], ArrayBuffer[QuarantineReport]) = { + + var validationStatus: ArrayBuffer[HealthCheckReport] = new ArrayBuffer[HealthCheckReport]() + var quarantineStatus: ArrayBuffer[QuarantineReport] = new ArrayBuffer[QuarantineReport]() + + Overwatch_RunIDs.foreach(Overwatch_RunID => { + val jobRun_df = if (jobRunDF.count == 0)spark.emptyDataFrame else jobRunDF.filter('Overwatch_RunID === Overwatch_RunID) + val jrcp_df = if (jrcpDF.count == 0)spark.emptyDataFrame else jrcpDF.filter('Overwatch_RunID === Overwatch_RunID) + val nb_df = if (nbDF.count == 0)spark.emptyDataFrame else nbDF.filter('Overwatch_RunID === Overwatch_RunID) + val nbcmd_df = if (nbcmdDF.count == 0)spark.emptyDataFrame else nbcmdDF.filter('Overwatch_RunID === Overwatch_RunID) + val clsf_df = if (clsfDF.count == 0)spark.emptyDataFrame else clsfDF.filter('Overwatch_RunID === Overwatch_RunID) + val cluster_df = if (clusterDF.count == 0)spark.emptyDataFrame else clusterDF.filter('Overwatch_RunID === Overwatch_RunID) + + //Job_ID_Present_In_JobRun_Gold_But_Not_In_JobRunCostPotentialFact_Gold + (validationStatus, quarantineStatus) == validateColumnBetweenMultipleTable(goldTargets.jobRunTarget, goldTargets.jobRunCostPotentialFactTarget, + jobRun_df, jrcp_df, "job_id", jobRunKey, validationStatus, quarantineStatus,Overwatch_RunID) + + //Job_ID_Present_In_JobRunCostPotentialFact_Gold_But_Not_In_JobRun_Gold + (validationStatus, quarantineStatus) == validateColumnBetweenMultipleTable(goldTargets.jobRunCostPotentialFactTarget, goldTargets.jobRunTarget, + jrcp_df, jobRun_df, "job_id", jrcpKey, validationStatus, quarantineStatus,Overwatch_RunID) + + //Cluster_ID_Present_In_JobRun_Gold_But_Not_In_JobRunCostPotentialFact_Gold + (validationStatus, quarantineStatus) == validateColumnBetweenMultipleTable(goldTargets.jobRunTarget, goldTargets.jobRunCostPotentialFactTarget, + jobRun_df, jrcp_df, "cluster_id", jobRunKey, validationStatus, quarantineStatus,Overwatch_RunID) + + //Cluster_ID_Present_In_JobRunCostPotentialFact_Gold_But_Not_In_JobRun_Gold + (validationStatus, quarantineStatus) == validateColumnBetweenMultipleTable(goldTargets.jobRunCostPotentialFactTarget, goldTargets.jobRunTarget, + jrcp_df, jobRun_df, "cluster_id", jrcpKey, validationStatus, quarantineStatus,Overwatch_RunID) + + //Notebook_Id_Present_In_Notebook_gold_But_Not_In_NotebookCommands_gold + (validationStatus, quarantineStatus) == validateColumnBetweenMultipleTable(goldTargets.notebookTarget, goldTargets.notebookCommandsTarget, + nb_df, nbcmd_df, "notebook_id", nbkey, validationStatus, quarantineStatus,Overwatch_RunID) + + //Notebook_Id_Present_In_NotebookCommands_Gold_But_Not_In_Notebook_Gold + (validationStatus, quarantineStatus) == validateColumnBetweenMultipleTable(goldTargets.notebookCommandsTarget, goldTargets.notebookTarget, + nbcmd_df, nb_df, "notebook_id", nbcmdkey, validationStatus, quarantineStatus,Overwatch_RunID) + + //Cluster_ID_Present_In_ClusterStateFact_Gold_But_Not_In_JobRunCostPotentialFact_Gold + (validationStatus, quarantineStatus) == validateColumnBetweenMultipleTable(goldTargets.clusterStateFactTarget, goldTargets.jobRunCostPotentialFactTarget, + clsf_df, jrcp_df, "cluster_id", clsfKey, validationStatus, quarantineStatus,Overwatch_RunID) + + //Cluster_ID_Present_In_JobRunCostPotentialFact_Gold_But_Not_In_ClusterStateFact_Gold + (validationStatus, quarantineStatus) == validateColumnBetweenMultipleTable(goldTargets.jobRunCostPotentialFactTarget, goldTargets.clusterStateFactTarget, + jrcp_df, clsf_df, "cluster_id", jrcpKey, validationStatus, quarantineStatus,Overwatch_RunID) + + //Cluster_ID_Present_In_Cluster_Gold_But_Not_In_ClusterStateFact_Gold + (validationStatus, quarantineStatus) == validateColumnBetweenMultipleTable(goldTargets.clusterTarget, goldTargets.clusterStateFactTarget, + cluster_df, clsf_df, "cluster_id", clusterKey, validationStatus, quarantineStatus,Overwatch_RunID) + + //Cluster_ID_Present_In_ClusterStateFact_Gold_But_Not_In_Cluster_Gold + (validationStatus, quarantineStatus) == validateColumnBetweenMultipleTable(goldTargets.clusterStateFactTarget, goldTargets.clusterTarget, + clsf_df, cluster_df, "cluster_id", clsfKey, validationStatus, quarantineStatus,Overwatch_RunID) + }) + (validations ++= validationStatus, quarantine ++= quarantineStatus) + + } + + private[overwatch] def snapShotHealthCheck(validationArray: Array[HealthCheckReport], healthCheckReportPath: String): Unit = { + + validationArray.toSeq.toDS().toDF() + .withColumn("healthcheck_id", lit(healthCheckID)) + .withColumn("snapTS", lit(pipelineSnapTime.asTSString)) + .withColumn("quarantine_id", lit(quarantineID)) + .moveColumnsToFront("healthcheck_id") + .write.format("delta") + .option("mergeSchema", "true") + .mode("append") + .save(healthCheckReportPath) + println("Validation report has been saved to " + s"""$healthCheckReportPath""") + } + + private[overwatch] def snapShotQuarantine(quarantineArray: Array[QuarantineReport], quarantineReportPath: String): Unit = { + + quarantineArray.toSeq.toDS().toDF() + .withColumn("quarantine_id", lit(quarantineID)) + .withColumn("snapTS", lit(pipelineSnapTime.asTSString)) + .moveColumnsToFront("quarantine_id") + .write + .partitionBy("quarantine_id") + .format("delta") + .option("mergeSchema", "true") + .mode("append") + .save(quarantineReportPath) + println("Quarantine report has been saved to " + s"""$quarantineReportPath""") + } + + private[overwatch] def handleValidation( + tableName: String, + df: DataFrame, + validationMethod: () => (ArrayBuffer[HealthCheckReport], ArrayBuffer[QuarantineReport]), + validations: ArrayBuffer[HealthCheckReport], + quarantine: ArrayBuffer[QuarantineReport]): (ArrayBuffer[HealthCheckReport], ArrayBuffer[QuarantineReport]) = { + if (spark.catalog.tableExists(s"$etlDB.$tableName")) { + if (df.count() != 0) { + validationMethod() + }else{ + println(s"Validation is not required for ${tableName}. The Table doesn't contain any data") + (validations, quarantine) + } + } else { + println(s"Validation is not possible for ${tableName} as it doesn't exist in the database") + (validations, quarantine) + } + } + +} + diff --git a/src/main/scala/com/databricks/labs/overwatch/utils/SparkSessionTestWrapper.scala b/src/main/scala/com/databricks/labs/overwatch/utils/SparkSessionTestWrapper.scala new file mode 100644 index 000000000..8fc77c2f1 --- /dev/null +++ b/src/main/scala/com/databricks/labs/overwatch/utils/SparkSessionTestWrapper.scala @@ -0,0 +1,29 @@ +package com.databricks.labs.overwatch.utils + +import org.apache.spark.SparkContext +import org.apache.spark.sql.SparkSession + +trait SparkSessionTestWrapper extends SparkSessionWrapper { + + override lazy protected val _envInit: Boolean = true + + override private[overwatch] def spark( globalSession: Boolean = true): SparkSession = { + SparkSession + .builder() + .master("local") + .appName("SparkSessionTestWrapper") + .config("spark.sql.shuffle.partitions", "1") + .getOrCreate() + } + + @transient override lazy val spark: SparkSession = spark(true) + + override lazy val sc: SparkContext = spark.sparkContext + + override def envInit( logLevel: String): Boolean = { + sc.setLogLevel( logLevel) + true + + } + +} diff --git a/src/main/scala/com/databricks/labs/overwatch/utils/Structures.scala b/src/main/scala/com/databricks/labs/overwatch/utils/Structures.scala index 2b0b85a55..2037015f3 100644 --- a/src/main/scala/com/databricks/labs/overwatch/utils/Structures.scala +++ b/src/main/scala/com/databricks/labs/overwatch/utils/Structures.scala @@ -160,6 +160,8 @@ case class ReconReport( deviationPercentage: Option[Double] = None, sourceQuery: Option[String] = None, targetQuery: Option[String] = None, + sourceOnlyColumns: Option[String] = None, + targetOnlyColumns: Option[String] = None, errorMsg: Option[String] = None ) @@ -340,6 +342,24 @@ case class MultiWSDeploymentReport( deploymentId: Option[String] ) +case class HealthCheckReport( + etl_database: String, + table_name: String, + healthCheck_rule: String, + rule_type : String, + healthCheckMsg: Option[String] = None, + Overwatch_RunID: String + ) + +case class QuarantineReport( + etl_database: String, + table_name : String, + healthcheck_rule_failed : String, + rule_type : String, + healthcheck_type : String, + keys : String + ) + case class WorkspaceMetastoreRegistrationReport(workspaceDataset: WorkspaceDataset, registerStatement: String, status: String) @@ -382,8 +402,10 @@ case class SanitizeFieldException(field: StructField, rules: List[SanitizeRule], object OverwatchScope extends Enumeration { type OverwatchScope = Value - val jobs, clusters, clusterEvents, sparkEvents, audit, notebooks, accounts, dbsql, pools, notebookCommands = Value + val jobs, clusters, clusterEvents, sparkEvents, audit, notebooks, accounts, + dbsql, pools, notebookCommands, warehouseEvents = Value // Todo Issue_77 + def toArray: Array[String] = values.map(_.toString).toArray } object WriteMode extends Enumeration { diff --git a/src/main/scala/com/databricks/labs/overwatch/utils/TransformationDescriber.scala b/src/main/scala/com/databricks/labs/overwatch/utils/TransformationDescriber.scala new file mode 100644 index 000000000..10c976547 --- /dev/null +++ b/src/main/scala/com/databricks/labs/overwatch/utils/TransformationDescriber.scala @@ -0,0 +1,58 @@ +package com.databricks.labs.overwatch.utils + +import org.apache.spark.sql.Dataset + +// TODO: implement this as a `trait`. Initial attempts would not +// compile because of the dependencies among other `trait`s and +// `object`s that would have to be refactored. + +object TransformationDescriber { + + + class NamedTransformation[T,U]( + val transformation: Dataset[T] => Dataset[U])( + implicit _name: sourcecode.Name) { + + final val name: String = _name.value + + override def toString = s"${_name.value}: NamedTransformation" + + } + + + object NamedTransformation { + + def apply[T,U]( + transformation: Dataset[T] => Dataset[U])( + implicit name: sourcecode.Name) = + new NamedTransformation( transformation)( name) + + } + + + implicit class TransformationDescriber[T,U]( ds: Dataset[T]) { + + def transformWithDescription[U]( + namedTransformation: NamedTransformation[T,U])( + implicit + // enclosing: sourcecode.Enclosing, + name: sourcecode.Name, + fileName: sourcecode.FileName, + line: sourcecode.Line + ): Dataset[U] = { + + // println( s"Inside TransformationDescriber.transformWithDescription: $enclosing") + + val callSite = s"${name.value} at ${fileName.value}:${line.value}" + + val sc = ds.sparkSession.sparkContext + sc.setJobDescription( namedTransformation.toString) + sc.setCallSite( callSite) + + ds.transform( namedTransformation.transformation) + + } + + } + +} diff --git a/src/main/scala/com/databricks/labs/overwatch/validation/DataReconciliation.scala b/src/main/scala/com/databricks/labs/overwatch/validation/DataReconciliation.scala index 7407949fe..53e7402f2 100644 --- a/src/main/scala/com/databricks/labs/overwatch/validation/DataReconciliation.scala +++ b/src/main/scala/com/databricks/labs/overwatch/validation/DataReconciliation.scala @@ -6,7 +6,8 @@ import com.databricks.labs.overwatch.utils.Helpers.getAllPipelineTargets import com.databricks.labs.overwatch.utils.{Helpers, ReconReport, SparkSessionWrapper} import org.apache.log4j.Logger import org.apache.spark.sql.DataFrame -import org.apache.spark.sql.functions.{col, hash, lit} +import org.apache.spark.sql.functions.{col, concat_ws, hash, lit, sha2} + import java.time.LocalDateTime import scala.collection.mutable.ArrayBuffer @@ -36,10 +37,11 @@ object DataReconciliation extends SparkSessionWrapper { performBasicRecon(sourceOrgIDArr,targetOrgIDArr) val sourceWorkspace = getConfig(sourceEtl,sourceOrgIDArr(0)) val targetWorkspace = getConfig(targetEtl,targetOrgIDArr(0)) - val targets = getAllPipelineTargets(sourceWorkspace) - println("Number of tables for recon: "+targets.length) - println(targets.foreach(t => println(t.name))) - val report = runRecon(targets, sourceEtl, sourceOrgIDArr, targetEtl) + val sourceTargets = getAllPipelineTargets(sourceWorkspace) + val targetTargets = getAllPipelineTargets(targetWorkspace) + println("Number of tables for recon: "+sourceTargets.length) + println(sourceTargets.foreach(t => println(t.name))) + val report = runRecon(sourceTargets,targetTargets, sourceEtl, sourceOrgIDArr, targetEtl) val reconRunId: String = java.util.UUID.randomUUID.toString val etlStoragePrefix = targetWorkspace.getConfig.etlDataPathPrefix.substring(0, targetWorkspace.getConfig.etlDataPathPrefix.length - 13) saveReconReport(report, etlStoragePrefix, "ReconReport", reconRunId) @@ -59,18 +61,27 @@ object DataReconciliation extends SparkSessionWrapper { * @param targetEtl * @return */ - private def hashValidation(target: PipelineTable,orgId: String, sourceEtl:String,targetEtl:String ):ReconReport ={ + private def hashValidation(sourceTarget: PipelineTable,targetTarget: PipelineTable,orgId: String, sourceEtl:String,targetEtl:String ):ReconReport ={ val reconType = "Validation by hashing" try { - val sourceQuery = getQuery(s"""${target.tableFullName}""", orgId) - val sourceTable = hashAllColumns(getTableDF(sourceQuery,target)) - val targetQuery = getQuery(s"""${target.tableFullName.replaceAll(sourceEtl,targetEtl)}""", orgId) - val targetTable = hashAllColumns(getTableDF(targetQuery,target)) - val sourceCount = sourceTable.count() - val targetCount = targetTable.count() - val missingSourceCount = targetTable.exceptAll(sourceTable).count() - val missingTargetCount = sourceTable.exceptAll(targetTable).count() - val commonDataCount = sourceTable.intersectAll(targetTable).count() + val sourceQuery = getQuery(s"""${sourceTarget.tableFullName}""", orgId) + val targetQuery = getQuery(s"""${targetTarget.tableFullName}""", orgId) + val sourceDF = sourceTarget.asDF.filter(col("organization_id") === orgId) + val targetDF = targetTarget.asDF.filter(col("organization_id") === orgId) + val commonColumns = sourceDF.columns.intersect(targetDF.columns) + val colInSourceNotInTarget = sourceDF.columns.diff(targetDF.columns).mkString(",") + val colInTargetNotInSource = targetDF.columns.diff(sourceDF.columns).mkString(",") + val sourceTable = hashAllColumns(getTableDF(sourceDF.select(commonColumns.map(col):_*),sourceTarget)) + val targetTable = hashAllColumns(getTableDF(targetDF.select(commonColumns.map(col):_*),targetTarget)) + val sumSource = sourceTable.withColumn("hash", sha2(concat_ws("||", sourceTable.columns.map(col): _*), 256)).select("hash") + val sumTarget = targetTable.withColumn("hash", sha2(concat_ws("||", targetTable.columns.map(col): _*), 256)).select("hash") + sourceTable.unpersist() + targetTable.unpersist() + val sourceCount = sumSource.count() + val targetCount = sumTarget.count() + val missingSourceCount = sumTarget.exceptAll(sumSource).count() + val missingTargetCount = sumSource.exceptAll(sumTarget).count() + val commonDataCount = sumSource.intersectAll(sumTarget).count() val deviationFactor = { if ((missingSourceCount + missingTargetCount) == 0) { 1 @@ -100,7 +111,7 @@ object DataReconciliation extends SparkSessionWrapper { reconType = reconType, sourceDB = sourceEtl, targetDB = targetEtl, - tableName = target.name, + tableName = targetTarget.name, sourceCount = Some(sourceCount), targetCount = Some(targetCount), missingInSource = Some(missingSourceCount), @@ -109,6 +120,8 @@ object DataReconciliation extends SparkSessionWrapper { deviationPercentage = Some(deviation), sourceQuery = Some(sourceQuery), targetQuery = Some(targetQuery), + sourceOnlyColumns = Some(colInSourceNotInTarget), + targetOnlyColumns = Some(colInTargetNotInSource), errorMsg = Some("")) } catch { @@ -120,7 +133,7 @@ object DataReconciliation extends SparkSessionWrapper { reconType = reconType, sourceDB = sourceEtl, targetDB = targetEtl, - tableName = target.tableFullName, + tableName = targetTarget.tableFullName, errorMsg = Some(fullMsg) ) } @@ -129,6 +142,7 @@ object DataReconciliation extends SparkSessionWrapper { } + /** * This method runs the reconciliation for all targets in parallel. * @@ -138,7 +152,8 @@ object DataReconciliation extends SparkSessionWrapper { * @param targetEtl : ETL name of the current version of OW * @return Array of ReconReport */ - private[overwatch] def runRecon(targets: ParArray[PipelineTable] , + private[overwatch] def runRecon(sourceTargets: ParArray[PipelineTable] , + targetTargets: ParArray[PipelineTable], sourceEtl:String, sourceOrgIDArr: Array[String], targetEtl:String, @@ -146,8 +161,26 @@ object DataReconciliation extends SparkSessionWrapper { spark.conf.set("spark.sql.legacy.allowHashOnMapType","true") val reconStatus: ArrayBuffer[ReconReport] = new ArrayBuffer[ReconReport]() sourceOrgIDArr.foreach(orgId=> { - targets.foreach(target => { - reconStatus.append(hashValidation(target, orgId, sourceEtl, targetEtl)) + sourceTargets.foreach(st => { + println("performing recon for"+st.name) + var tableFound = false + targetTargets.foreach(tt => + if(tt.name == st.name) + { + tableFound = true + reconStatus.append(hashValidation(st,tt, orgId, sourceEtl, targetEtl)) + } + ) + if(!tableFound){ + reconStatus.append(ReconReport( + workspaceId = orgId, + reconType = "Validation by hashing", + sourceDB = sourceEtl, + targetDB = targetEtl, + tableName = st.name, + errorMsg = Some("Table not found in target") + )) + } }) } ) @@ -204,15 +237,15 @@ object DataReconciliation extends SparkSessionWrapper { * @param target : PipelineTable object * @return DataFrame */ - private def getTableDF(query: String,target: PipelineTable):DataFrame = { + private def getTableDF(df: DataFrame,target: PipelineTable):DataFrame = { try{ val excludedCol = target.excludedReconColumn val dropCol = excludedCol ++ Array("Overwatch_RunID", "Pipeline_SnapTS", "__overwatch_ctrl_noise") - val filterDF = spark.sql(query).drop(dropCol: _ *) + val filterDF = df.drop(dropCol: _ *) filterDF }catch { case exception: Exception => - println(s"""Exception: Unable to run the query ${query}"""+exception.getMessage) + println(s"""Exception: Unable filter dataframe ${target.tableFullName}"""+exception.getMessage) spark.emptyDataFrame } diff --git a/src/main/scala/com/databricks/labs/overwatch/validation/PipelineValidation.scala b/src/main/scala/com/databricks/labs/overwatch/validation/PipelineValidation.scala new file mode 100644 index 000000000..40a7f1a14 --- /dev/null +++ b/src/main/scala/com/databricks/labs/overwatch/validation/PipelineValidation.scala @@ -0,0 +1,107 @@ +package com.databricks.labs.overwatch.validation + +import org.apache.log4j.{Level, Logger} +import com.databricks.labs.overwatch.utils.SparkSessionWrapper +import com.databricks.labs.overwatch.utils.PipelineValidationHelper +import org.apache.spark.sql.DataFrame + +/** + * The Purpose of this object is to validate the health of the tables in the overwatch database. + * @param ETLDB ETL Database for on which validation need to be done(Mandatory) + * @param table Table for which validation need to be done(Optional). If not provided all tables + * which are in scope of validation framework would be validated + * @param crossTableValidation Cross Table Validation need to be done(Optional). Default value is True. If it is false then only + * single table validation would be performed. + */ +object PipelineValidation extends SparkSessionWrapper { + + def apply(etlDB : String , allRun: Boolean) :Unit = { + new PipelineValidation(etlDB,allRun) + .setPipelineSnapTime() + .process() + } + + def apply(etlDB : String, allRun: Boolean, table : Array[String]) :Unit = { + new PipelineValidation(etlDB,allRun) + .setPipelineSnapTime() + .process(table) + } + + def apply(etlDB : String, allRun: Boolean, table : Array[String],crossTableValidation : Boolean) :Unit = { + new PipelineValidation(etlDB,allRun) + .setPipelineSnapTime() + .process(table,crossTableValidation) + } + +} + +class PipelineValidation (_etlDB: String, _allRun: Boolean) extends PipelineValidationHelper(_etlDB,_allRun) with SparkSessionWrapper { + + import spark.implicits._ + + private val logger: Logger = Logger.getLogger(this.getClass) + + def process(tableArray: Array[String] = Array(), crossTableValidation: Boolean = true): Unit = { + + val processingStartTime = System.currentTimeMillis() + + println("By Default Pipeline_Report would be Validated") + (validations, quarantine) == validatePipelineTable() + + if (crossTableValidation) { + println("Cross Table Validation has been Configured") + (validations, quarantine) == validateCrossTable() + } else { + println("Cross Table Validation has not been Configured") + logger.log(Level.INFO, "Cross Table Validation is Disabled") + } + + tableArray.length match { + case 0 => + println(s"By Default Single Table Validation has been configured for ${clsfTable},${jrcpTable},${clusterTable},${sparkJobTable}," + + s"${sqlQueryHistTable},${jobRunTable},${jobTable}") + (validations, quarantine) == handleValidation(clsfTable, clsfDF, validateCLSF, validations, quarantine) + (validations, quarantine) == handleValidation(jrcpTable, jrcpDF, validateJRCP, validations, quarantine) + (validations, quarantine) == handleValidation(clusterTable, clusterDF, validateCluster, validations, quarantine) + (validations, quarantine) == handleValidation(sparkJobTable, sparkJobDF, validateSparkJob, validations, quarantine) + (validations, quarantine) == handleValidation(sqlQueryHistTable, sqlQueryHistDF, validateSqlQueryHist, validations, quarantine) + (validations, quarantine) == handleValidation(jobRunTable, jobRunDF, validateJobRun, validations, quarantine) + (validations, quarantine) == handleValidation(jobTable, jobDF, validateJob, validations, quarantine) + + case _ => + println(s"Single Table Validation has been configured for ${tableArray.mkString(",")}") + tableArray.map(_.toLowerCase).foreach { + case tableName@"clusterstatefact_gold" => + (validations, quarantine) == handleValidation(clsfTable, clsfDF, validateCLSF, validations, quarantine) + case tableName@"jobruncostpotentialfact_gold" => + (validations, quarantine) == handleValidation(jrcpTable, jrcpDF, validateJRCP, validations, quarantine) + case tableName@"cluster_gold" => + (validations, quarantine) == handleValidation(clusterTable, clusterDF, validateCluster, validations, quarantine) + case tableName@"sparkjob_gold" => + (validations, quarantine) == handleValidation(sparkJobTable, sparkJobDF, validateSparkJob, validations, quarantine) + case tableName@"sql_query_history_gold" => + (validations, quarantine) == handleValidation(sqlQueryHistTable, sqlQueryHistDF, validateSqlQueryHist, validations, quarantine) + case tableName@"jobrun_gold" => + (validations, quarantine) == handleValidation(jobRunTable, jobRunDF, validateJobRun, validations, quarantine) + case tableName@"job_gold" => + (validations, quarantine) == handleValidation(jobTable, jobDF, validateJob, validations, quarantine) + case tableName => println(s"Table $tableName is not recognized or not supported.") + } + } + + val notValidatedCount = validations.toDS().toDF().filter(!'healthCheckMsg.contains("Success")).count() + + snapShotHealthCheck(validations.toArray, healthCheckReportPath) + snapShotQuarantine(quarantine.toArray, quarantineReportPath) + + val processingEndTime = System.currentTimeMillis() + val msg = + s"""*********** HealthCheck Report Details ******************* + |Total healthcheck count: ${validations.length} + |Failed healthcheck count:$notValidatedCount + |Report run duration in sec : ${(processingEndTime - processingStartTime) / 1000} + |""".stripMargin + + println(msg) + } +} \ No newline at end of file diff --git a/src/test/scala/com/databricks/labs/overwatch/SparkSessionTestWrapper.scala b/src/test/scala/com/databricks/labs/overwatch/SparkSessionTestWrapper.scala deleted file mode 100644 index 681ae9e99..000000000 --- a/src/test/scala/com/databricks/labs/overwatch/SparkSessionTestWrapper.scala +++ /dev/null @@ -1,17 +0,0 @@ -package com.databricks.labs.overwatch - -import org.apache.spark.SparkContext -import org.apache.spark.sql.SparkSession -trait SparkSessionTestWrapper { - lazy val spark: SparkSession = { - SparkSession - .builder() - .master("local") - .appName("spark session") - .config("spark.sql.shuffle.partitions", "1") - .getOrCreate() - } - - lazy val sc: SparkContext = spark.sparkContext - -} diff --git a/src/test/scala/com/databricks/labs/overwatch/pipeline/InitializeTest.scala b/src/test/scala/com/databricks/labs/overwatch/pipeline/InitializeTest.scala index ec683d900..2036f6d19 100644 --- a/src/test/scala/com/databricks/labs/overwatch/pipeline/InitializeTest.scala +++ b/src/test/scala/com/databricks/labs/overwatch/pipeline/InitializeTest.scala @@ -1,6 +1,6 @@ package com.databricks.labs.overwatch.pipeline -import com.databricks.labs.overwatch.SparkSessionTestWrapper +import com.databricks.labs.overwatch.utils.SparkSessionTestWrapper import com.databricks.labs.overwatch.env.Database import com.databricks.labs.overwatch.utils.OverwatchScope._ import com.databricks.labs.overwatch.utils.{BadConfigException, Config, OverwatchScope, TokenSecret} @@ -315,4 +315,4 @@ class InitializeTest extends AnyFunSpec with DataFrameComparer with SparkSession } } -} \ No newline at end of file +} diff --git a/src/test/scala/com/databricks/labs/overwatch/pipeline/InitializerFunctionsTest.scala b/src/test/scala/com/databricks/labs/overwatch/pipeline/InitializerFunctionsTest.scala index 235238f27..def9e1622 100644 --- a/src/test/scala/com/databricks/labs/overwatch/pipeline/InitializerFunctionsTest.scala +++ b/src/test/scala/com/databricks/labs/overwatch/pipeline/InitializerFunctionsTest.scala @@ -1,5 +1,5 @@ package com.databricks.labs.overwatch.pipeline -import com.databricks.labs.overwatch.SparkSessionTestWrapper +import com.databricks.labs.overwatch.utils.SparkSessionTestWrapper import com.databricks.labs.overwatch.utils.Config import com.github.mrpowers.spark.fast.tests.DataFrameComparer import org.scalatest.{Ignore, color} @@ -35,4 +35,4 @@ class InitializerFunctionsTest extends AnyFunSpec with DataFrameComparer with Sp assertResult(0)(df.count()) } } -} \ No newline at end of file +} diff --git a/src/test/scala/com/databricks/labs/overwatch/pipeline/PipelineFunctionsTest.scala b/src/test/scala/com/databricks/labs/overwatch/pipeline/PipelineFunctionsTest.scala index 3c96318e4..7b239747e 100644 --- a/src/test/scala/com/databricks/labs/overwatch/pipeline/PipelineFunctionsTest.scala +++ b/src/test/scala/com/databricks/labs/overwatch/pipeline/PipelineFunctionsTest.scala @@ -1,6 +1,6 @@ package com.databricks.labs.overwatch.pipeline -import com.databricks.labs.overwatch.SparkSessionTestWrapper +import com.databricks.labs.overwatch.utils.SparkSessionTestWrapper import com.databricks.labs.overwatch.utils.{BadConfigException, Config} import com.github.mrpowers.spark.fast.tests.DataFrameComparer import io.delta.tables.DeltaTable diff --git a/src/test/scala/com/databricks/labs/overwatch/pipeline/SchemaTest.scala b/src/test/scala/com/databricks/labs/overwatch/pipeline/SchemaTest.scala index 625827cc5..28d3a03a4 100644 --- a/src/test/scala/com/databricks/labs/overwatch/pipeline/SchemaTest.scala +++ b/src/test/scala/com/databricks/labs/overwatch/pipeline/SchemaTest.scala @@ -1,6 +1,6 @@ package com.databricks.labs.overwatch.pipeline -import com.databricks.labs.overwatch.SparkSessionTestWrapper +import com.databricks.labs.overwatch.utils.SparkSessionTestWrapper import com.github.mrpowers.spark.fast.tests.DataFrameComparer import org.scalatest.funspec.AnyFunSpec diff --git a/src/test/scala/com/databricks/labs/overwatch/pipeline/TransformFunctionsTest.scala b/src/test/scala/com/databricks/labs/overwatch/pipeline/TransformFunctionsTest.scala index 4f3edfdda..22a007928 100644 --- a/src/test/scala/com/databricks/labs/overwatch/pipeline/TransformFunctionsTest.scala +++ b/src/test/scala/com/databricks/labs/overwatch/pipeline/TransformFunctionsTest.scala @@ -1,6 +1,6 @@ package com.databricks.labs.overwatch.pipeline -import com.databricks.labs.overwatch.SparkSessionTestWrapper +import com.databricks.labs.overwatch.utils.SparkSessionTestWrapper import com.databricks.labs.overwatch.pipeline.TransformFunctions._ import com.github.mrpowers.spark.fast.tests.DataFrameComparer import org.apache.spark.sql.Row diff --git a/src/test/scala/com/databricks/labs/overwatch/utils/DataFrameSyntaxTest.scala b/src/test/scala/com/databricks/labs/overwatch/utils/DataFrameSyntaxTest.scala new file mode 100644 index 000000000..275c0d59b --- /dev/null +++ b/src/test/scala/com/databricks/labs/overwatch/utils/DataFrameSyntaxTest.scala @@ -0,0 +1,212 @@ +package com.databricks.labs.overwatch.utils + +// import com.databricks.labs.overwatch.SparkSessionTestWrapper +import org.apache.log4j.{Level,Logger} +import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.functions.length +import org.scalatest.funspec.AnyFunSpec +import org.scalatest.GivenWhenThen +import java.io.ByteArrayOutputStream +import org.scalatest.BeforeAndAfterEach + +class DataFrameSyntaxTest + extends AnyFunSpec + with GivenWhenThen + with BeforeAndAfterEach + with SparkSessionTestWrapper + with DataFrameSyntax[ SparkSessionTestWrapper] { + + + import spark.implicits._ + + private val logger: Logger = Logger.getLogger(this.getClass) + + override def beforeEach(): Unit = { + envInit( "WARN") + // in case of any stackable traits: + super.beforeEach() + } + + Given( "any Spark `Dataset` (including `DataFrame`s)") + + val df: DataFrame = Seq( + ( "1xxx5xxx10xxx15xxx20xxx25", + "BARx5" * 5)) + .toDF( + "foo", + "bar") + .withColumn( "length_foo", length( $"foo")) + .withColumn( "length_bar", length( $"bar")) + + info( "df.schema.treeString =>") + + df.schema.treeString.linesIterator.foreach( info(_)) + + + describe( "implicit class `DataFrameShower`") { + + it( "implements `df.showLines()`") { + + info( "with defaults of `(numRows, truncate, vertical)` => `(20, 20, false)`") + + df.showLines().foreach( info(_)) + + // assert( ???) + + } + + it( "implements `df.showLines( numRows, truncate, vertical)`") { + + df.showLines( 1, 2, true).foreach( info(_)) + + // assert( ???) + + } + + } + + + describe( "implicit class `DataFrameLogger`") { + + Given( "option `'overwatch.dataframelogger.level'` is set in Spark's runtime configuration to a level greater than or equal to the logger of the calling class") + + spark.conf.set( "overwatch.dataframelogger.level", "INFO") + + lazy val v = getDataFrameLoggerSparkConfValue.get + lazy val l = getDataFrameLoggerLevel + logger.setLevel( l) + + info( s"`'overwatch.dataframelogger.level'` is `'${v}'`") + info( s"`log4j.Level` used by `DataFrameLogger` is `${l}`") + info( s"`logger.getLevel` is `${logger.getLevel}`") + + assert( l.isGreaterOrEqual( logger.getLevel)) + + + it( "implements `df.log()` which produces truncated vertical output by default") { + + // info( df.showLines( vertical= true).mkString("\n")) + + df.log() + + // assert( ???) + + } + + it( "implements `df.log( false)`") { + + info( "which produces vertical output without value truncation and a warning") + + // info( df.showLines( false).mkString("\n")) + + df.log( false) + + // assert( ???) + + } + + it( "implements `df.log( truncate= 0, vertical= false)`") { + + info( "which produces tabular output without value truncation accompanied by two warnings") + + // info( df.showLines( 1, 0, false).mkString("\n")) + + df.log( truncate= 0, vertical= false) + + // assert( ???) + + } + + it( "implements `df.log( numRows, truncate, vertical)`") { + + val (numRows, truncate, vertical) = ( 1, 10, true) + + info( s"(numRows, truncate, vertical) => ${(numRows, truncate, vertical)}") + + // info( df.showLines( numRows, truncate, vertical).mkString("\n")) + + df.log( numRows= numRows, truncate= truncate, vertical= vertical) + + // assert( ???) + + } + + + ignore( "treats unrecognized logger levels as `OFF`") { + + spark.conf.unset( dataFrameLoggerSparkConfKey) + + spark.conf.set( dataFrameLoggerSparkConfKey, "FOO") + + assert( spark.sparkContext.appName == "SparkSessionTestWrapper") + + logger.setLevel( Level.INFO) + + val v = getDataFrameLoggerSparkConfValue().get + val l = getDataFrameLoggerLevel() + + info( s"`'overwatch.dataframelogger.level'` is `'${v}'`") + info( s"`logger.getLevel()` in `${this.getClass.getSimpleName}` is `${logger.getLevel}`") + info( s"`DataFrameLogger` `Level` is `${l}`") + + assert( v == "FOO") + assert( l == Level.OFF) + + df.log() + + + } + + + it( "treats unset logger level as `OFF`") { + + spark.conf.unset( dataFrameLoggerSparkConfKey) + + val v = getDataFrameLoggerSparkConfValue.getOrElse( "not set") + val l = getDataFrameLoggerLevel() + + assert( v == "not set") + + info( s"`'overwatch.dataframelogger.level'` is ${v}") + info( s"`logger.getLevel()` in `${this.getClass.getSimpleName}` is `${logger.getLevel}`") + info( s"`DataFrameLogger` `Level` is `${l}`") + + assert( l == Level.OFF) + + + df.log() + + } + + + it( "respects the log level of the calling class") { + + spark.conf.set( "overwatch.dataframelogger.level", "DEBUG") + + // val l: Level = Level.toLevel( spark.conf.get( "overwatch.dataframelogger.level")) + + lazy val l = getDataFrameLoggerLevel() + + logger.setLevel( Level.INFO) + + val out = + s"""| `'overwatch.dataframelogger.level'` is `'${l}'` (`${l.toInt}`); + | `logger.getLevel()` in `${this.getClass.getSimpleName}` + | is `${logger.getLevel}` (`${logger.getLevel.toInt}`)""" + .stripMargin.linesIterator.mkString + + info( out) + + logger.info( out) + + assert( logger.getLevel.isGreaterOrEqual( getDataFrameLoggerLevel)) + + df.log() + + } + + + } + + +} diff --git a/src/test/scala/com/databricks/labs/overwatch/utils/SchemaToolsTest.scala b/src/test/scala/com/databricks/labs/overwatch/utils/SchemaToolsTest.scala index 299808e1a..0d1e1ee8d 100644 --- a/src/test/scala/com/databricks/labs/overwatch/utils/SchemaToolsTest.scala +++ b/src/test/scala/com/databricks/labs/overwatch/utils/SchemaToolsTest.scala @@ -1,6 +1,6 @@ package com.databricks.labs.overwatch.utils -import com.databricks.labs.overwatch.SparkSessionTestWrapper +// import com.databricks.labs.overwatch.SparkSessionTestWrapper import com.databricks.labs.overwatch.pipeline.TransformFunctions._ import org.apache.spark.sql.Column import org.apache.spark.sql.functions.{col, lit, size, struct} diff --git a/src/test/scala/com/databricks/labs/overwatch/utils/TSDFTest.scala b/src/test/scala/com/databricks/labs/overwatch/utils/TSDFTest.scala index 5f274dd9c..8c51f7afc 100644 --- a/src/test/scala/com/databricks/labs/overwatch/utils/TSDFTest.scala +++ b/src/test/scala/com/databricks/labs/overwatch/utils/TSDFTest.scala @@ -1,6 +1,6 @@ package com.databricks.labs.overwatch.utils -import com.databricks.labs.overwatch.SparkSessionTestWrapper +// import com.databricks.labs.overwatch.SparkSessionTestWrapper import org.scalatest.GivenWhenThen import org.scalatest.funspec.AnyFunSpec import com.databricks.labs.overwatch.pipeline.TransformFunctions._ diff --git a/src/test/scala/com/databricks/labs/overwatch/utils/TransformationDescriberTest.scala b/src/test/scala/com/databricks/labs/overwatch/utils/TransformationDescriberTest.scala new file mode 100644 index 000000000..870717a8a --- /dev/null +++ b/src/test/scala/com/databricks/labs/overwatch/utils/TransformationDescriberTest.scala @@ -0,0 +1,101 @@ +package com.databricks.labs.overwatch.utils + +import org.apache.spark.sql.DataFrame +import org.scalatest.funspec.AnyFunSpec +import org.scalatest.GivenWhenThen +import java.io.ByteArrayOutputStream + +class TransformationDescriberTest + extends AnyFunSpec + with GivenWhenThen + with SparkSessionTestWrapper { + + import TransformationDescriber._ + import spark.implicits._ + spark.conf.set("spark.sql.session.timeZone", "UTC") + + val t = (df: DataFrame) => df.select( $"foo") + + val nt = NamedTransformation( t) + + // TODO: replace use of `s` and `Console.withOut` with an abstraction + + val s = new ByteArrayOutputStream + + describe( "A NamedTransformation") { + + it( "wraps a function literal") { + + info( s"nt.transformation: ${nt.transformation}") + + assert( nt.transformation === t) + + } + + it( "knows its own name") { + + info( s"`nt.name`: ${nt.name}") + info( s"`nt.toString`: ${nt.toString}") + + assert( nt.name === "nt") + assert( nt.toString === "nt: NamedTransformation") + + } + + Given( "a Spark `Dataset` (including `DataFrame`s)") + + val in = Seq( ("foo", "bar")).toDF( "foo", "bar") + + Console.withOut( s) { + in.show(numRows= 1, truncate= 0, vertical= true) + } + // info( s.toString) + s.toString.linesIterator.foreach( info(_)) + s.reset + + When( "a `NamedTransformation` is applied") + + val out = in.transformWithDescription( nt) + + // val s = new ByteArrayOutputStream + Console.withOut( s) { + out.show(numRows= 1, truncate= 0, vertical= true) + } + // info( s.toString) + s.toString.linesIterator.foreach( info(_)) + + + + Then( "the resulting Spark jobs have a matching description (pending)") + + // info( s"""spark.jobGroup.id: ${out.sparkSession.sparkContext.getLocalProperty( "spark.jobGroup.id")}""") + + val sjd = out.sparkSession.sparkContext.getLocalProperty( "spark.job.description") + + info( s"spark.job.description: ${sjd}") + + assert( sjd === "nt: NamedTransformation") + + // info( s"""spark.callSite.short: ${out.sparkSession.sparkContext.getLocalProperty( "spark.callSite.short")}""") + // info( s"""spark.callSite.long: ${out.sparkSession.sparkContext.getLocalProperty( "spark.callSite.long")}""") + + + + + + + And( "the result of the transformation is correct") + + assertResult( "`foo` STRING") { + out.schema.toDDL + } + + assertResult( "foo") { + out.first.getString(0) + } + + + } + + +}