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

Use conf.useCompression config to decide if we should be compressing the cache [databricks] #4989

Merged
merged 4 commits into from
Mar 23, 2022
Merged
Show file tree
Hide file tree
Changes from 1 commit
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
Original file line number Diff line number Diff line change
Expand Up @@ -327,6 +327,7 @@ protected class ParquetCachedBatchSerializer extends GpuCachedBatchSerializer wi
conf: SQLConf): RDD[CachedBatch] = {

val rapidsConf = new RapidsConf(conf)
val useCompression = conf.useCompression
val bytesAllowedPerBatch = getBytesAllowedPerBatch(conf)
val (schemaWithUnambiguousNames, _) = getSupportedSchemaFromUnsupported(schema)
val structSchema = schemaWithUnambiguousNames.toStructType
Expand All @@ -349,7 +350,7 @@ protected class ParquetCachedBatchSerializer extends GpuCachedBatchSerializer wi
} else {
withResource(putOnGpuIfNeeded(batch)) { gpuCB =>
compressColumnarBatchWithParquet(gpuCB, structSchema, schema.toStructType,
bytesAllowedPerBatch)
bytesAllowedPerBatch, useCompression)
}
}
})
Expand All @@ -367,7 +368,8 @@ protected class ParquetCachedBatchSerializer extends GpuCachedBatchSerializer wi
oldGpuCB: ColumnarBatch,
schema: StructType,
origSchema: StructType,
bytesAllowedPerBatch: Long): List[ParquetCachedBatch] = {
bytesAllowedPerBatch: Long,
useCompression: Boolean): List[ParquetCachedBatch] = {
val estimatedRowSize = scala.Range(0, oldGpuCB.numCols()).map { idx =>
oldGpuCB.column(idx).asInstanceOf[GpuColumnVector]
.getBase.getDeviceMemorySize / oldGpuCB.numRows()
Expand Down Expand Up @@ -418,7 +420,7 @@ protected class ParquetCachedBatchSerializer extends GpuCachedBatchSerializer wi

for (i <- splitVectors.head.indices) {
withResource(makeTableForIndex(i)) { table =>
val buffer = writeTableToCachedBatch(table, schema)
val buffer = writeTableToCachedBatch(table, schema, useCompression)
buffers += ParquetCachedBatch(buffer)
}
}
Expand All @@ -427,7 +429,7 @@ protected class ParquetCachedBatchSerializer extends GpuCachedBatchSerializer wi
}
} else {
withResource(GpuColumnVector.from(gpuCB)) { table =>
val buffer = writeTableToCachedBatch(table, schema)
val buffer = writeTableToCachedBatch(table, schema, useCompression)
buffers += ParquetCachedBatch(buffer)
}
}
Expand All @@ -437,12 +439,17 @@ protected class ParquetCachedBatchSerializer extends GpuCachedBatchSerializer wi

private def writeTableToCachedBatch(
table: Table,
schema: StructType): ParquetBufferConsumer = {
schema: StructType,
useCompression: Boolean): ParquetBufferConsumer = {
val buffer = new ParquetBufferConsumer(table.getRowCount.toInt)
val opts = SchemaUtils
.writerOptionsFromSchema(ParquetWriterOptions.builder(), schema, writeInt96 = false)
.withStatisticsFrequency(StatisticsFrequency.ROWGROUP).build()
withResource(Table.writeParquetChunked(opts, buffer)) { writer =>
val optsBuilder = SchemaUtils
.writerOptionsFromSchema(ParquetWriterOptions.builder(), schema, writeInt96 = false)
.withCompressionType(CompressionType.NONE)
.withStatisticsFrequency(StatisticsFrequency.ROWGROUP)
if (useCompression) {
optsBuilder.withCompressionType(CompressionType.SNAPPY)
jlowe marked this conversation as resolved.
Show resolved Hide resolved
}
withResource(Table.writeParquetChunked(optsBuilder.build(), buffer)) { writer =>
writer.write(table)
}
buffer
Expand Down Expand Up @@ -1454,6 +1461,7 @@ protected class ParquetCachedBatchSerializer extends GpuCachedBatchSerializer wi
conf: SQLConf): RDD[CachedBatch] = {

val rapidsConf = new RapidsConf(conf)
val useCompression = conf.useCompression
val bytesAllowedPerBatch = getBytesAllowedPerBatch(conf)
val (schemaWithUnambiguousNames, _) = getSupportedSchemaFromUnsupported(schema)
if (rapidsConf.isSqlEnabled && rapidsConf.isSqlExecuteOnGPU &&
Expand All @@ -1465,7 +1473,7 @@ protected class ParquetCachedBatchSerializer extends GpuCachedBatchSerializer wi
})
columnarBatchRdd.flatMap(cb => {
withResource(cb)(cb => compressColumnarBatchWithParquet(cb, structSchema,
schema.toStructType, bytesAllowedPerBatch))
schema.toStructType, bytesAllowedPerBatch, useCompression))
})
} else {
val broadcastedConf = SparkSession.active.sparkContext.broadcast(conf.getAllConfs)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -149,7 +149,8 @@ class Spark310ParquetWriterSuite extends SparkQueryCompareTestSuite {
Array(StructField("empty", ByteType, false),
StructField("empty", ByteType, false),
StructField("empty", ByteType, false)))
ser.compressColumnarBatchWithParquet(cb, dummySchema, dummySchema, BYTES_ALLOWED_PER_BATCH)
ser.compressColumnarBatchWithParquet(cb, dummySchema, dummySchema,
BYTES_ALLOWED_PER_BATCH, false)
theTableMock.close()
}
}
Expand Down