|
| 1 | +/* |
| 2 | + * Copyright 2018 ABSA Group Limited |
| 3 | + * |
| 4 | + * Licensed under the Apache License, Version 2.0 (the "License"); |
| 5 | + * you may not use this file except in compliance with the License. |
| 6 | + * You may obtain a copy of the License at |
| 7 | + * http://www.apache.org/licenses/LICENSE-2.0 |
| 8 | + * |
| 9 | + * Unless required by applicable law or agreed to in writing, software |
| 10 | + * distributed under the License is distributed on an "AS IS" BASIS, |
| 11 | + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
| 12 | + * See the License for the specific language governing permissions and |
| 13 | + * limitations under the License. |
| 14 | + */ |
| 15 | + |
| 16 | +package za.co.absa.hyperdrive.compatibility.impl.writer.cdc |
| 17 | + |
| 18 | +import org.apache.hadoop.fs.FileSystem |
| 19 | +import org.apache.spark.sql.expressions.Window |
| 20 | +import org.apache.spark.sql.functions._ |
| 21 | +import org.apache.spark.sql.types.{BooleanType, StructField, StructType, TimestampType} |
| 22 | +import org.apache.spark.sql.{DataFrame, SparkSession} |
| 23 | +import za.co.absa.hyperdrive.shared.utils.FileUtils |
| 24 | + |
| 25 | +import java.net.URI |
| 26 | + |
| 27 | +object CDCUtil { |
| 28 | + |
| 29 | + private val StartDateColumn = "_start_date" |
| 30 | + private val EndDateColumn = "_end_date" |
| 31 | + private val IsCurrentColumn = "_is_current" |
| 32 | + private val IsOldDataColumn = "_is_old_data" |
| 33 | + private val SortFieldPrefix = "_tmp_hyperdrive_" |
| 34 | + private val OldData = "_old_data" |
| 35 | + private val NewData = "_new_data" |
| 36 | + private val SortFieldCustomOrderColumn = "_tmp_hyperdrive_sort_field_custom_order_" |
| 37 | + |
| 38 | + private[hyperdrive] case class SCD2Fields(keyColumn: String, |
| 39 | + timestampColumn: String, |
| 40 | + operationColumn: String, |
| 41 | + operationDeleteValues: Seq[String], |
| 42 | + precombineColumns: Seq[String], |
| 43 | + precombineColumnsCustomOrder: Map[String, Seq[String]]) |
| 44 | + |
| 45 | + private[hyperdrive] def getStagedDataForSCD2(history: DataFrame, input: DataFrame, scd2Fields: SCD2Fields): DataFrame = { |
| 46 | + val uniqueChangesForEachKeyAndTimestamp = removeDuplicates(input, scd2Fields) |
| 47 | + val previousEvents = getPreviousEvents(history, uniqueChangesForEachKeyAndTimestamp, scd2Fields) |
| 48 | + val nextEvents = getNextEvents(history, uniqueChangesForEachKeyAndTimestamp, scd2Fields) |
| 49 | + |
| 50 | + val union = previousEvents.union(nextEvents).distinct().union( |
| 51 | + uniqueChangesForEachKeyAndTimestamp |
| 52 | + .withColumn(StartDateColumn, col(scd2Fields.timestampColumn)) |
| 53 | + .withColumn(EndDateColumn, lit(null)) |
| 54 | + .withColumn(IsCurrentColumn, lit(false)) |
| 55 | + .withColumn(IsOldDataColumn, lit(false)) |
| 56 | + .selectExpr( |
| 57 | + Seq(StartDateColumn, EndDateColumn, IsCurrentColumn) ++ |
| 58 | + uniqueChangesForEachKeyAndTimestamp.columns ++ |
| 59 | + Seq(IsOldDataColumn): _* |
| 60 | + ) |
| 61 | + ) |
| 62 | + |
| 63 | + val uniqueEvents = removeDuplicates(union, scd2Fields) |
| 64 | + setSCD2Fields(uniqueEvents, scd2Fields).drop(IsOldDataColumn) |
| 65 | + } |
| 66 | + |
| 67 | + private[hyperdrive] def getDataFrameWithSortColumns(dataFrame: DataFrame, sortFieldsPrefix: String, precombineColumns: Seq[String], precombineColumnsCustomOrder: Map[String, Seq[String]]): DataFrame = { |
| 68 | + precombineColumns.foldLeft(dataFrame) { (df, precombineColumn) => |
| 69 | + val order = precombineColumnsCustomOrder.getOrElse(precombineColumn, Seq.empty[String]) |
| 70 | + order match { |
| 71 | + case o if o.isEmpty => |
| 72 | + df.withColumn(s"$sortFieldsPrefix$precombineColumn", col(precombineColumn)) |
| 73 | + case o => |
| 74 | + df |
| 75 | + .withColumn(SortFieldCustomOrderColumn, lit(o.toArray)) |
| 76 | + .withColumn( |
| 77 | + s"$sortFieldsPrefix$precombineColumn", |
| 78 | + expr(s"""array_position($SortFieldCustomOrderColumn,$precombineColumn)""") |
| 79 | + ).drop(SortFieldCustomOrderColumn) |
| 80 | + } |
| 81 | + } |
| 82 | + } |
| 83 | + |
| 84 | + private[hyperdrive] def getSchemaWithSCD2Fields(input: DataFrame): StructType = { |
| 85 | + StructType( |
| 86 | + Seq( |
| 87 | + StructField(StartDateColumn, TimestampType, nullable = false), |
| 88 | + StructField(EndDateColumn, TimestampType, nullable = true), |
| 89 | + StructField(IsCurrentColumn, BooleanType, nullable = false) |
| 90 | + ).toArray ++ input.schema.fields |
| 91 | + ) |
| 92 | + } |
| 93 | + |
| 94 | + private[hyperdrive] def isDirEmptyOrDoesNotExist(spark: SparkSession, destination: String): Boolean = { |
| 95 | + implicit val fs: FileSystem = FileSystem.get(new URI(destination), spark.sparkContext.hadoopConfiguration) |
| 96 | + if (FileUtils.exists(destination)) { |
| 97 | + if (FileUtils.isDirectory(destination)) { |
| 98 | + FileUtils.isEmpty(destination) |
| 99 | + } else { |
| 100 | + false |
| 101 | + } |
| 102 | + } else { |
| 103 | + true |
| 104 | + } |
| 105 | + } |
| 106 | + |
| 107 | + private def removeDuplicates(input: DataFrame, scd2Fields: SCD2Fields): DataFrame = { |
| 108 | + val dataFrameWithSortColumns = getDataFrameWithSortColumns(input, SortFieldPrefix, scd2Fields.precombineColumns, scd2Fields.precombineColumnsCustomOrder) |
| 109 | + val sortColumnsWithPrefix = dataFrameWithSortColumns.schema.fieldNames.filter(_.startsWith(SortFieldPrefix)) |
| 110 | + val window = Window |
| 111 | + .partitionBy(s"${scd2Fields.keyColumn}", s"${scd2Fields.timestampColumn}") |
| 112 | + .orderBy(sortColumnsWithPrefix.map(col(_).desc): _*) |
| 113 | + dataFrameWithSortColumns |
| 114 | + .withColumn("rank", row_number().over(window)) |
| 115 | + .where("rank == 1") |
| 116 | + .drop("rank") |
| 117 | + .drop(sortColumnsWithPrefix: _*) |
| 118 | + } |
| 119 | + |
| 120 | + private def getPreviousEvents(history: DataFrame, uniqueChangesForEachKeyAndTimestamp: DataFrame, scd2Fields: SCD2Fields): DataFrame = { |
| 121 | + history.as(OldData).join( |
| 122 | + uniqueChangesForEachKeyAndTimestamp.as(NewData), |
| 123 | + col(s"$NewData.${scd2Fields.keyColumn}").equalTo(col(s"$OldData.${scd2Fields.keyColumn}")) |
| 124 | + .and(col(s"$NewData.${scd2Fields.timestampColumn}").>=(col(s"$OldData.$StartDateColumn"))) |
| 125 | + .and(col(s"$NewData.${scd2Fields.timestampColumn}").<=(col(s"$OldData.$EndDateColumn"))) |
| 126 | + .or( |
| 127 | + col(s"$NewData.${scd2Fields.keyColumn}").equalTo(col(s"$OldData.${scd2Fields.keyColumn}")) |
| 128 | + .and(col(s"$NewData.${scd2Fields.timestampColumn}").>=(col(s"$OldData.$StartDateColumn"))) |
| 129 | + .and(col(s"$OldData.$IsCurrentColumn").equalTo(true)) |
| 130 | + ) |
| 131 | + ).select(s"$OldData.*").withColumn(s"$IsOldDataColumn", lit(true)) |
| 132 | + } |
| 133 | + |
| 134 | + private def getNextEvents(history: DataFrame, uniqueChangesForEachKeyAndTimestamp: DataFrame, scd2Fields: SCD2Fields): DataFrame = { |
| 135 | + val window = Window |
| 136 | + .partitionBy(col(s"$OldData.${scd2Fields.keyColumn}"), col(s"$NewData.${scd2Fields.timestampColumn}")) |
| 137 | + .orderBy(col(s"$OldData.$StartDateColumn").asc, col(s"$OldData.${scd2Fields.timestampColumn}").asc) |
| 138 | + |
| 139 | + history.as(OldData).join( |
| 140 | + uniqueChangesForEachKeyAndTimestamp.as(NewData), |
| 141 | + col(s"$NewData.${scd2Fields.keyColumn}").equalTo(col(s"$OldData.${scd2Fields.keyColumn}")) |
| 142 | + .and(col(s"$NewData.${scd2Fields.timestampColumn}").<(col(s"$OldData.$StartDateColumn"))) |
| 143 | + ).select(s"$OldData.*", s"$NewData.${scd2Fields.timestampColumn}") |
| 144 | + .withColumn("rank", row_number().over(window)) |
| 145 | + .where("rank == 1") |
| 146 | + .drop("rank") |
| 147 | + .select(s"$OldData.*") |
| 148 | + .withColumn(s"$IsOldDataColumn", lit(true)) |
| 149 | + } |
| 150 | + |
| 151 | + private def setSCD2Fields(dataFrame: DataFrame, scd2Fields: SCD2Fields): DataFrame = { |
| 152 | + val idWindowDesc = org.apache.spark.sql.expressions.Window |
| 153 | + .partitionBy(scd2Fields.keyColumn) |
| 154 | + .orderBy(col(scd2Fields.timestampColumn).desc, col(IsOldDataColumn).desc) |
| 155 | + dataFrame |
| 156 | + .withColumn( |
| 157 | + EndDateColumn, |
| 158 | + when( |
| 159 | + col(IsOldDataColumn).equalTo(true).and( |
| 160 | + lag(scd2Fields.keyColumn, 1, null).over(idWindowDesc).isNull |
| 161 | + ), |
| 162 | + col(EndDateColumn) |
| 163 | + ).when( |
| 164 | + col(IsOldDataColumn).equalTo(true).and( |
| 165 | + lag(IsOldDataColumn, 1, false).over(idWindowDesc).equalTo(true) |
| 166 | + ), |
| 167 | + col(EndDateColumn) |
| 168 | + ).otherwise( |
| 169 | + lag(StartDateColumn, 1, null).over(idWindowDesc) |
| 170 | + ) |
| 171 | + ) |
| 172 | + .withColumn( |
| 173 | + EndDateColumn, |
| 174 | + when(col(scd2Fields.operationColumn).isInCollection(scd2Fields.operationDeleteValues), col(StartDateColumn)) |
| 175 | + .when(!col(scd2Fields.operationColumn).isInCollection(scd2Fields.operationDeleteValues), col(EndDateColumn)) |
| 176 | + .otherwise(null) |
| 177 | + ) |
| 178 | + .withColumn( |
| 179 | + IsCurrentColumn, |
| 180 | + when(col(EndDateColumn).isNull, lit(true)).otherwise(lit(false)) |
| 181 | + ) |
| 182 | + } |
| 183 | + |
| 184 | +} |
0 commit comments