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

[Spark] Implement optimized write. #2145

Closed
wants to merge 7 commits into from
Closed
Show file tree
Hide file tree
Changes from 5 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
12 changes: 12 additions & 0 deletions spark/src/main/scala/org/apache/spark/sql/delta/DeltaConfig.scala
Original file line number Diff line number Diff line change
Expand Up @@ -684,6 +684,18 @@ trait DeltaConfigsBase extends DeltaLogging {
_ => true,
"needs to be a boolean."
)

/**
* Enable optimized writes into a Delta table. Optimized writes adds an adaptive shuffle before
* the write to write compacted files into a Delta table during a write.
*/
val OPTIMIZE_WRITE = buildConfig[Option[Boolean]](
"autoOptimize.optimizeWrite",
null,
v => Option(v).map(_.toBoolean),
_ => true,
"needs to be a boolean."
)
}

object DeltaConfigs extends DeltaConfigsBase
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,7 @@ import org.apache.spark.sql.delta.metric.IncrementMetric
import org.apache.spark.sql.delta._
import org.apache.spark.sql.delta.actions.{AddFile, FileAction}
import org.apache.spark.sql.delta.commands.merge.{MergeIntoMaterializeSource, MergeIntoMaterializeSourceReason, MergeStats}
import org.apache.spark.sql.delta.files.{TahoeBatchFileIndex, TahoeFileIndex}
import org.apache.spark.sql.delta.files.{TahoeBatchFileIndex, TahoeFileIndex, TransactionalWrite}
import org.apache.spark.sql.delta.metering.DeltaLogging
import org.apache.spark.sql.delta.schema.{ImplicitMetadataOperation, SchemaUtils}
import org.apache.spark.sql.delta.sources.DeltaSQLConf
Expand Down Expand Up @@ -228,7 +228,11 @@ abstract class MergeIntoCommandBase extends LeafRunnableCommand
txn: OptimisticTransaction,
outputDF: DataFrame): Seq[FileAction] = {
val partitionColumns = txn.metadata.partitionColumns
if (partitionColumns.nonEmpty && spark.conf.get(DeltaSQLConf.MERGE_REPARTITION_BEFORE_WRITE)) {
// If the write will be an optimized write, which shuffles the data anyway, then don't
// repartition. Optimized writes can handle both splitting very large tasks and coalescing
// very small ones.
if (partitionColumns.nonEmpty && spark.conf.get(DeltaSQLConf.MERGE_REPARTITION_BEFORE_WRITE)
&& !TransactionalWrite.shouldOptimizeWrite(txn.metadata, spark.sessionState.conf)) {
txn.writeFiles(outputDF.repartition(partitionColumns.map(col): _*))
} else {
txn.writeFiles(outputDF)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -412,7 +412,7 @@ class OptimizeExecutor(
sparkSession.sparkContext.getLocalProperty(SPARK_JOB_GROUP_ID),
description)

val addFiles = txn.writeFiles(repartitionDF).collect {
val addFiles = txn.writeFiles(repartitionDF, None, isOptimize = true, Nil).collect {
case a: AddFile =>
a.copy(dataChange = false)
case other =>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,15 +16,14 @@

package org.apache.spark.sql.delta.files

import java.net.URI

import scala.collection.mutable.ListBuffer

import org.apache.spark.sql.delta._
import org.apache.spark.sql.delta.actions._
import org.apache.spark.sql.delta.commands.cdc.CDCReader
import org.apache.spark.sql.delta.constraints.{Constraint, Constraints, DeltaInvariantCheckerExec}
import org.apache.spark.sql.delta.metering.DeltaLogging
import org.apache.spark.sql.delta.perf.DeltaOptimizedWriterExec
import org.apache.spark.sql.delta.schema._
import org.apache.spark.sql.delta.sources.DeltaSQLConf
import org.apache.spark.sql.delta.sources.DeltaSQLConf.DELTA_COLLECT_STATS_USING_TABLE_SCHEMA
Expand Down Expand Up @@ -228,6 +227,13 @@ trait TransactionalWrite extends DeltaLogging { self: OptimisticTransactionImpl
writeFiles(data, Nil)
}

def writeFiles(
data: Dataset[_],
deltaOptions: Option[DeltaOptions],
additionalConstraints: Seq[Constraint]): Seq[FileAction] = {
writeFiles(data, deltaOptions, isOptimize = false, additionalConstraints)
}

/**
* Returns a tuple of (data, partition schema). For CDC writes, a `__is_cdc` column is added to
* the data and `__is_cdc=true/false` is added to the front of the partition schema.
Expand Down Expand Up @@ -344,10 +350,16 @@ trait TransactionalWrite extends DeltaLogging { self: OptimisticTransactionImpl
/**
* Writes out the dataframe after performing schema validation. Returns a list of
* actions to append these files to the reservoir.
*
* @param inputData Data to write out.
* @param writeOptions Options to decide how to write out the data.
* @param isOptimize Whether the operation writing this is Optimize or not.
* @param additionalConstraints Additional constraints on the write.
*/
def writeFiles(
inputData: Dataset[_],
writeOptions: Option[DeltaOptions],
isOptimize: Boolean,
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

can you clarify why we need this new isOptimize flag now ?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Added some comments.

additionalConstraints: Seq[Constraint]): Seq[FileAction] = {
hasWritten = true

Expand Down Expand Up @@ -379,7 +391,15 @@ trait TransactionalWrite extends DeltaLogging { self: OptimisticTransactionImpl

val empty2NullPlan = convertEmptyToNullIfNeeded(queryExecution.executedPlan,
partitioningColumns, constraints)
val physicalPlan = DeltaInvariantCheckerExec(empty2NullPlan, constraints)
val checkInvariants = DeltaInvariantCheckerExec(empty2NullPlan, constraints)
// No need to plan optimized write if the write command is OPTIMIZE, which aims to produce
// evenly-balanced data files already.
val physicalPlan = if (!isOptimize &&
shouldOptimizeWrite(writeOptions, spark.sessionState.conf)) {
DeltaOptimizedWriterExec(checkInvariants, metadata.partitionColumns, deltaLog)
} else {
checkInvariants
}

val statsTrackers: ListBuffer[WriteJobStatsTracker] = ListBuffer()

Expand Down Expand Up @@ -449,4 +469,24 @@ trait TransactionalWrite extends DeltaLogging { self: OptimisticTransactionImpl

resultFiles.toSeq ++ committer.changeFiles
}

/**
* Optimized writes can be enabled/disabled through the following order:
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

this makes sense, I guess the intention is to have an explicit way to turn on/off for every write/table/session in that order

Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

thanks for changing the precendence

* - Through DataFrameWriter options
* - Through SQL configuration
* - Through the table parameter
*/
private def shouldOptimizeWrite(
writeOptions: Option[DeltaOptions], sessionConf: SQLConf): Boolean = {
writeOptions.flatMap(_.optimizeWrite)
.getOrElse(TransactionalWrite.shouldOptimizeWrite(metadata, sessionConf))
}
}

object TransactionalWrite {
def shouldOptimizeWrite(metadata: Metadata, sessionConf: SQLConf): Boolean = {
sessionConf.getConf(DeltaSQLConf.DELTA_OPTIMIZE_WRITE_ENABLED)
.orElse(DeltaConfigs.OPTIMIZE_WRITE.fromMetaData(metadata))
.getOrElse(false)
}
}
Loading
Loading