Skip to content

Commit

Permalink
[Spark][Version Checksum] Enable checksum writes by default (#3919)
Browse files Browse the repository at this point in the history
<!--
Thanks for sending a pull request!  Here are some tips for you:
1. If this is your first time, please read our contributor guidelines:
https://github.com/delta-io/delta/blob/master/CONTRIBUTING.md
2. If the PR is unfinished, add '[WIP]' in your PR title, e.g., '[WIP]
Your PR title ...'.
  3. Be sure to keep the PR description updated to reflect all changes.
  4. Please write your PR title to summarize what this PR proposes.
5. If possible, provide a concise example to reproduce the issue for a
faster review.
6. If applicable, include the corresponding issue number in the PR title
and link it in the body.
-->

#### Which Delta project/connector is this regarding?
<!--
Please add the component selected below to the beginning of the pull
request title
For example: [Spark] Title of my pull request
-->

- [X] Spark
- [ ] Standalone
- [ ] Flink
- [ ] Kernel
- [ ] Other (fill in here)

## Description

<!--
- Describe what this PR changes.
- Describe why we need the change.
 
If this PR resolves an issue be sure to include "Resolves #XXX" to
correctly link and close the issue upon merge.
-->
Version Checksum can now be incrementally computed without triggering a
full state reconstruction (see
#3899,
#3895). This PR enables writing
the version checksum by default.

## How was this patch tested?

<!--
If tests were added, say they were added here. Please make sure to test
the changes thoroughly including negative and positive cases if
possible.
If the changes were tested in any way other than unit tests, please
clarify how you tested step by step (ideally copy and paste-able, so
that other reviewers can test and check, and descendants can verify in
the future).
If the changes were not tested, please explain why.
-->
Existing tests in DeltaLogSuite and ChecksumSuite should cover this
change.

## Does this PR introduce _any_ user-facing changes?

<!--
If yes, please clarify the previous behavior and the change this PR
proposes - provide the console output, description and/or an example to
show the behavior difference if possible.
If possible, please also clarify if this is a user-facing change
compared to the released Delta Lake versions or within the unreleased
branches such as master.
If no, write 'No'.
-->
No
  • Loading branch information
dhruvarya-db authored Dec 4, 2024
1 parent 4aecba5 commit 09e7523
Show file tree
Hide file tree
Showing 2 changed files with 18 additions and 3 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -1035,7 +1035,7 @@ trait DeltaSQLConfBase {
buildConf("writeChecksumFile.enabled")
.doc("Whether the checksum file can be written.")
.booleanConf
.createWithDefault(false)
.createWithDefault(true)

val INCREMENTAL_COMMIT_ENABLED =
buildConf("incremental.commit.enabled")
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@

package org.apache.spark.sql.delta

import org.apache.spark.sql.delta.sources.DeltaSQLConf
import org.apache.spark.sql.delta.test.DeltaSQLCommandTest
import org.apache.spark.sql.delta.test.DeltaTestImplicits._
import org.apache.spark.sql.delta.util.{FileNames, JsonUtils}
Expand Down Expand Up @@ -89,7 +90,14 @@ class EvolvabilitySuite extends EvolvabilitySuiteBase with DeltaSQLCommandTest {

test("transaction log schema evolvability - batch change data read") {
withTempDir { dir =>
withSQLConf(DeltaConfigs.CHANGE_DATA_FEED.defaultTablePropertyKey -> "true") {
withSQLConf(
DeltaConfigs.CHANGE_DATA_FEED.defaultTablePropertyKey -> "true",
// All files verification will always fail in this test since we the extra column
// will not be present in the `allFiles` of the CRC.
DeltaSQLConf.DELTA_ALL_FILES_IN_CRC_VERIFICATION_MODE_ENABLED.key -> "false",
DeltaSQLConf.DELTA_ALL_FILES_IN_CRC_FORCE_VERIFICATION_MODE_FOR_NON_UTC_ENABLED.key ->
"false"
) {
EvolvabilitySuiteBase.generateTransactionLogWithExtraColumn(spark, dir.getAbsolutePath)
spark.sql(s"UPDATE delta.`${dir.getAbsolutePath}` SET value = 10")
spark.read.format("delta").option("readChangeFeed", "true")
Expand All @@ -104,7 +112,14 @@ class EvolvabilitySuite extends EvolvabilitySuiteBase with DeltaSQLCommandTest {

test("transaction log schema evolvability - streaming change data read") {
withTempDir { dir =>
withSQLConf(DeltaConfigs.CHANGE_DATA_FEED.defaultTablePropertyKey -> "true") {
withSQLConf(
DeltaConfigs.CHANGE_DATA_FEED.defaultTablePropertyKey -> "true",
// All files verification will always fail in this test since we the extra column
// will not be present in the `allFiles` of the CRC.
DeltaSQLConf.DELTA_ALL_FILES_IN_CRC_VERIFICATION_MODE_ENABLED.key -> "false",
DeltaSQLConf.DELTA_ALL_FILES_IN_CRC_FORCE_VERIFICATION_MODE_FOR_NON_UTC_ENABLED.key ->
"false"
) {
EvolvabilitySuiteBase.generateTransactionLogWithExtraColumn(spark, dir.getAbsolutePath)
spark.sql(s"UPDATE delta.`${dir.getAbsolutePath}` SET value = 10")
val query = spark.readStream.format("delta")
Expand Down

0 comments on commit 09e7523

Please sign in to comment.