Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Revert to unsorted Dataframe for Delta and Hudi #109

Merged
merged 1 commit into from
Jan 14, 2025
Merged
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
Original file line number Diff line number Diff line change
Expand Up @@ -155,7 +155,7 @@ object LakeWriter {

def commit(viewName: String): F[Unit] =
for {
df <- SparkUtils.prepareFinalDataFrame(spark, viewName, writerParallelism)
df <- SparkUtils.prepareFinalDataFrame(spark, viewName, writerParallelism, w.expectsSortedDataframe)
_ <- mutex.lock
.surround {
w.write(df)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -103,15 +103,18 @@ private[processing] object SparkUtils {
def prepareFinalDataFrame[F[_]: Sync](
spark: SparkSession,
viewName: String,
writerParallelism: Int
writerParallelism: Int,
writerExpectsSortedDataframe: Boolean
): F[DataFrame] =
Sync[F].delay {
spark
.table(viewName)
.repartitionByRange(writerParallelism, col("event_name"), col("event_id"))
.sortWithinPartitions("event_name")
.withColumn("load_tstamp", current_timestamp())
}
for {
df <- Sync[F].pure(spark.table(viewName))
df <- Sync[F].pure {
// Create equally-balanced partitions, for which events with similar event_name are likely to be in the same partition.
// This maximizes output file sizes, for a lake which is partitioned by event_name.
if (writerParallelism > 1) df.repartitionByRange(writerParallelism, col("event_name"), col("event_id")) else df.coalesce(1)
}
df <- Sync[F].pure(if (writerExpectsSortedDataframe) df.sortWithinPartitions("event_name") else df)
} yield df.withColumn("load_tstamp", current_timestamp())

def dropView[F[_]: Sync](spark: SparkSession, viewName: String): F[Unit] =
Logger[F].info(s"Removing Spark data frame $viewName from local disk...") >>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -111,4 +111,5 @@ class DeltaWriter(config: Config.Delta) extends Writer {
*/
override def toleratesAsyncDelete: Boolean = true

override def expectsSortedDataframe: Boolean = false
}
Original file line number Diff line number Diff line change
Expand Up @@ -94,4 +94,6 @@ class HudiWriter(config: Config.Hudi) extends Writer {
* steps must happen in order.
*/
override def toleratesAsyncDelete: Boolean = false

override def expectsSortedDataframe: Boolean = false
}
Original file line number Diff line number Diff line change
Expand Up @@ -112,4 +112,10 @@ class IcebergWriter(config: Config.Iceberg) extends Writer {
* re-writes a file that was previously deleted
*/
override def toleratesAsyncDelete: Boolean = true

/**
* Iceberg writer requires the Dataframe to be sorted, because we set the iceberg write option
* `distribution-mode = none`
*/
override def expectsSortedDataframe: Boolean = true
}
Original file line number Diff line number Diff line change
Expand Up @@ -36,4 +36,10 @@ trait Writer {
* If tolerated, then we use our customized `LakeLoaderFileSystem`.
*/
def toleratesAsyncDelete: Boolean

/**
* Whether this writer expects the DataFrame to be sorted by the partition column, i.e. by
* event_name
*/
def expectsSortedDataframe: Boolean
}
Loading