-
Notifications
You must be signed in to change notification settings - Fork 1.7k
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
Changes from 5 commits
f94cc4c
a063653
94fe218
1ef15e9
646f53a
01a4d98
013bd67
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -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 | ||
|
@@ -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. | ||
|
@@ -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, | ||
additionalConstraints: Seq[Constraint]): Seq[FileAction] = { | ||
hasWritten = true | ||
|
||
|
@@ -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() | ||
|
||
|
@@ -449,4 +469,24 @@ trait TransactionalWrite extends DeltaLogging { self: OptimisticTransactionImpl | |
|
||
resultFiles.toSeq ++ committer.changeFiles | ||
} | ||
|
||
/** | ||
* Optimized writes can be enabled/disabled through the following order: | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe 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 There was a problem hiding this comment. Choose a reason for hiding this commentThe 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) | ||
} | ||
} |
There was a problem hiding this comment.
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 ?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Added some comments.