diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/sources/DeltaSQLConf.scala b/spark/src/main/scala/org/apache/spark/sql/delta/sources/DeltaSQLConf.scala index 9b288ced4e0..1c46473a56b 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/sources/DeltaSQLConf.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/sources/DeltaSQLConf.scala @@ -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") diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/EvolvabilitySuite.scala b/spark/src/test/scala/org/apache/spark/sql/delta/EvolvabilitySuite.scala index 4d5a8dde2ed..47b801aafa7 100644 --- a/spark/src/test/scala/org/apache/spark/sql/delta/EvolvabilitySuite.scala +++ b/spark/src/test/scala/org/apache/spark/sql/delta/EvolvabilitySuite.scala @@ -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} @@ -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") @@ -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")