Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
13 changes: 8 additions & 5 deletions .github/workflows/build.yml
Original file line number Diff line number Diff line change
@@ -1,4 +1,4 @@
name: Build
name: Build3

on:
pull_request:
Expand All @@ -17,12 +17,15 @@ jobs:
spark: 3
name: Scala ${{ matrix.scala }}, Spark ${{ matrix.spark }}
steps:
- uses: actions/checkout@v2
- uses: actions/checkout@v4
with:
persist-credentials: false
- name: Set up JDK 1.8
uses: actions/setup-java@v1
uses: actions/setup-java@v2
with:
java-version: 1.8
- uses: actions/cache@v2
distribution: 'adopt'
java-version: 8
- uses: actions/cache@v4
with:
path: ~/.m2/repository
key: ${{ runner.os }}-${{ matrix.scala }}-${{ matrix.spark }}-${{ hashFiles('**/pom.xml') }}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,9 +17,10 @@ package za.co.absa.hyperdrive.compatibility.api

import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.execution.streaming.MetadataLogFileIndex
import org.apache.spark.sql.types.StructType

trait CompatibleSparkUtil {
def createMetadataLogFileIndex(spark: SparkSession, destination: String): MetadataLogFileIndex
def createMetadataLogFileIndex(spark: SparkSession, destination: String, userSpecifiedSchema: Option[StructType]): MetadataLogFileIndex
def hasMetadata(spark: SparkSession, destination: String): Boolean
def jsonStringToObject(jsonString: String): Object
def objectToJsonString(obj: Object): Option[String]
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,12 +17,13 @@ package za.co.absa.hyperdrive.compatibility.provider

import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.execution.streaming.MetadataLogFileIndex
import org.apache.spark.sql.types.StructType
import za.co.absa.hyperdrive.compatibility.api.CompatibleSparkUtil
import za.co.absa.hyperdrive.compatibility.impl.SparkUtil

object CompatibleSparkUtilProvider extends CompatibleSparkUtil {
def createMetadataLogFileIndex(spark: SparkSession, destination: String): MetadataLogFileIndex =
SparkUtil.createMetadataLogFileIndex(spark, destination)
def createMetadataLogFileIndex(spark: SparkSession, destination: String, userSpecifiedSchema: Option[StructType]): MetadataLogFileIndex =
SparkUtil.createMetadataLogFileIndex(spark, destination, userSpecifiedSchema)

def hasMetadata(spark: SparkSession, destination: String): Boolean =
SparkUtil.hasMetadata(spark, destination)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,15 +20,16 @@ import org.apache.avro.util.internal.JacksonUtils
import org.apache.hadoop.fs.Path
import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.execution.streaming.{FileStreamSink, MetadataLogFileIndex}
import org.apache.spark.sql.types.StructType
import za.co.absa.hyperdrive.compatibility.api.CompatibleSparkUtil

import java.io.ByteArrayOutputStream

object SparkUtil extends CompatibleSparkUtil {
private lazy val objectMapper = new ObjectMapper()

override def createMetadataLogFileIndex(spark: SparkSession, destination: String): MetadataLogFileIndex =
new MetadataLogFileIndex(spark, new Path(destination), Map.empty, None)
override def createMetadataLogFileIndex(spark: SparkSession, destination: String, userSpecifiedSchema: Option[StructType]): MetadataLogFileIndex =
new MetadataLogFileIndex(spark, new Path(destination), Map.empty, userSpecifiedSchema)

override def hasMetadata(spark: SparkSession, destination: String): Boolean =
FileStreamSink.hasMetadata(Seq(destination), spark.sparkContext.hadoopConfiguration, spark.sessionState.conf)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,16 +16,21 @@
package za.co.absa.hyperdrive.ingestor.implementation.transformer.dateversion

import org.apache.commons.configuration2.Configuration
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.execution.streaming.MetadataLogFileIndex
import org.slf4j.LoggerFactory
import org.apache.spark.sql.functions.{lit, to_date}
import org.apache.spark.sql.types.StructType
import org.apache.spark.sql.{DataFrame, SparkSession}
import za.co.absa.hyperdrive.compatibility.provider.CompatibleSparkUtilProvider
import za.co.absa.hyperdrive.ingestor.api.transformer.{StreamTransformer, StreamTransformerFactory}
import za.co.absa.hyperdrive.ingestor.api.utils.ConfigUtils.getOrThrow
import za.co.absa.hyperdrive.ingestor.implementation.writer.parquet.ParquetStreamWriter
import org.apache.spark.sql.types.{DateType, StructField}

import java.time.LocalDate
import java.time.format.DateTimeFormatter
import org.apache.spark.sql.types._

private[transformer] class AddDateVersionTransformer(val reportDate: String, val destination: String) extends StreamTransformer {

Expand All @@ -44,22 +49,31 @@ private[transformer] class AddDateVersionTransformer(val reportDate: String, val
if (noCommittedParquetFilesExist(spark)) {
initialVersion
} else {
import spark.implicits._
val df = spark.read.parquet(destination)
val versions = df.select(df(ColumnVersion))
.filter(df(ColumnDate) === lit(reportDate))
.distinct()
.as[Int]
.collect().toList

val versions = getVersions(spark, ColumnDate, ColumnVersion, reportDate)
if (versions.nonEmpty) versions.max + 1 else initialVersion
}
}

private def noCommittedParquetFilesExist(spark: SparkSession): Boolean = {
val fileCatalog = CompatibleSparkUtilProvider.createMetadataLogFileIndex(spark, destination)
val fileCatalog = CompatibleSparkUtilProvider.createMetadataLogFileIndex(spark, destination, None)
!CompatibleSparkUtilProvider.hasMetadata(spark, destination) || fileCatalog.allFiles().isEmpty
}

private def getVersions(spark: SparkSession, ColumnDate: String, ColumnVersion: String, reportDate: String): Seq[Int] = {
val fileCatalog: MetadataLogFileIndex = CompatibleSparkUtilProvider.createMetadataLogFileIndex(spark, destination, Some(StructType(Seq(
StructField(ColumnDate, StringType, nullable = true),
StructField(ColumnVersion, IntegerType, nullable = true)
))))

fileCatalog.partitionSpec().partitions
.map { partition =>
val row: InternalRow = partition.values
(row.getString(0), row.getInt(1))
}
.filter { case (date, _) => date == reportDate }
.map { case (_, version) => version }
.toList
}
}

object AddDateVersionTransformer extends StreamTransformerFactory with AddDateVersionTransformerAttributes {
Expand Down
Loading