Skip to content

Commit 802eb36

Browse files
committed
Disables page-level CRC checksums by default.
1 parent c9b4792 commit 802eb36

File tree

2 files changed

+4
-4
lines changed

2 files changed

+4
-4
lines changed

sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala

Lines changed: 3 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -127,6 +127,9 @@ class ParquetFileFormat
127127
conf.setEnum(ParquetOutputFormat.JOB_SUMMARY_LEVEL, JobSummaryLevel.NONE)
128128
}
129129

130+
// PARQUET-1580: Disables page-level CRC checksums by default.
131+
conf.setBooleanIfUnset(ParquetOutputFormat.PAGE_WRITE_CHECKSUM_ENABLED, false)
132+
130133
if (ParquetOutputFormat.getJobSummaryLevel(conf) != JobSummaryLevel.NONE
131134
&& !classOf[ParquetOutputCommitter].isAssignableFrom(committerClass)) {
132135
// output summary is requested, but the class is not a Parquet Committer

sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala

Lines changed: 1 addition & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -29,7 +29,6 @@ import scala.util.control.ControlThrowable
2929
import com.google.common.util.concurrent.UncheckedExecutionException
3030
import org.apache.commons.io.FileUtils
3131
import org.apache.hadoop.conf.Configuration
32-
import org.apache.parquet.hadoop.ParquetOutputFormat
3332
import org.scalatest.time.SpanSugar._
3433

3534
import org.apache.spark.{SparkConf, SparkContext, TaskContext, TestUtils}
@@ -226,9 +225,7 @@ class StreamSuite extends StreamTest {
226225

227226
val df = spark.readStream.format(classOf[FakeDefaultSource].getName).load()
228227
Seq("", "parquet").foreach { useV1Source =>
229-
withSQLConf(
230-
SQLConf.USE_V1_SOURCE_LIST.key -> useV1Source,
231-
ParquetOutputFormat.PAGE_WRITE_CHECKSUM_ENABLED -> "false") {
228+
withSQLConf(SQLConf.USE_V1_SOURCE_LIST.key -> useV1Source) {
232229
assertDF(df)
233230
assertDF(df)
234231
}

0 commit comments

Comments
 (0)