-
Notifications
You must be signed in to change notification settings - Fork 3.2k
Spark 3.4: Add write options to override the compression properties of the table #8313
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
Changes from all commits
bdf359c
8b8affc
49063cb
d08226b
e2ac0c3
bb11538
86458e9
923e03a
b607d7a
083e84f
102c119
faf8f5d
371ac02
9321807
c6218f7
91a88a9
f0e295a
2c0308c
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 |
|---|---|---|
|
|
@@ -21,6 +21,12 @@ | |
| import static org.apache.iceberg.DistributionMode.HASH; | ||
| import static org.apache.iceberg.DistributionMode.NONE; | ||
| import static org.apache.iceberg.DistributionMode.RANGE; | ||
| import static org.apache.iceberg.TableProperties.AVRO_COMPRESSION; | ||
| import static org.apache.iceberg.TableProperties.AVRO_COMPRESSION_LEVEL; | ||
| import static org.apache.iceberg.TableProperties.ORC_COMPRESSION; | ||
| import static org.apache.iceberg.TableProperties.ORC_COMPRESSION_STRATEGY; | ||
| import static org.apache.iceberg.TableProperties.PARQUET_COMPRESSION; | ||
| import static org.apache.iceberg.TableProperties.PARQUET_COMPRESSION_LEVEL; | ||
|
|
||
| import java.util.Locale; | ||
| import java.util.Map; | ||
|
|
@@ -418,4 +424,96 @@ public String branch() { | |
|
|
||
| return branch; | ||
| } | ||
|
|
||
| public String parquetCompressionCodec() { | ||
| return confParser | ||
| .stringConf() | ||
| .option(SparkWriteOptions.COMPRESSION_CODEC) | ||
| .sessionConf(SparkSQLProperties.COMPRESSION_CODEC) | ||
| .tableProperty(TableProperties.PARQUET_COMPRESSION) | ||
| .defaultValue(TableProperties.PARQUET_COMPRESSION_DEFAULT) | ||
| .parse(); | ||
| } | ||
|
|
||
| public String parquetCompressionLevel() { | ||
| return confParser | ||
| .stringConf() | ||
| .option(SparkWriteOptions.COMPRESSION_LEVEL) | ||
| .sessionConf(SparkSQLProperties.COMPRESSION_LEVEL) | ||
| .tableProperty(TableProperties.PARQUET_COMPRESSION_LEVEL) | ||
| .defaultValue(TableProperties.PARQUET_COMPRESSION_LEVEL_DEFAULT) | ||
| .parseOptional(); | ||
| } | ||
|
|
||
| public String avroCompressionCodec() { | ||
| return confParser | ||
| .stringConf() | ||
| .option(SparkWriteOptions.COMPRESSION_CODEC) | ||
| .sessionConf(SparkSQLProperties.COMPRESSION_CODEC) | ||
| .tableProperty(TableProperties.AVRO_COMPRESSION) | ||
| .defaultValue(TableProperties.AVRO_COMPRESSION_DEFAULT) | ||
| .parse(); | ||
| } | ||
|
|
||
| public String avroCompressionLevel() { | ||
| return confParser | ||
| .stringConf() | ||
| .option(SparkWriteOptions.COMPRESSION_LEVEL) | ||
| .sessionConf(SparkSQLProperties.COMPRESSION_LEVEL) | ||
| .tableProperty(TableProperties.AVRO_COMPRESSION_LEVEL) | ||
| .defaultValue(TableProperties.AVRO_COMPRESSION_LEVEL_DEFAULT) | ||
| .parseOptional(); | ||
|
roryqi marked this conversation as resolved.
|
||
| } | ||
|
|
||
| public String orcCompressionCodec() { | ||
| return confParser | ||
| .stringConf() | ||
| .option(SparkWriteOptions.COMPRESSION_CODEC) | ||
| .sessionConf(SparkSQLProperties.COMPRESSION_CODEC) | ||
| .tableProperty(TableProperties.ORC_COMPRESSION) | ||
| .defaultValue(TableProperties.ORC_COMPRESSION_DEFAULT) | ||
| .parse(); | ||
| } | ||
|
|
||
| public String orcCompressionStrategy() { | ||
| return confParser | ||
| .stringConf() | ||
| .option(SparkWriteOptions.COMPRESSION_STRATEGY) | ||
| .sessionConf(SparkSQLProperties.COMPRESSION_STRATEGY) | ||
| .tableProperty(TableProperties.ORC_COMPRESSION_STRATEGY) | ||
| .defaultValue(TableProperties.ORC_COMPRESSION_STRATEGY_DEFAULT) | ||
| .parse(); | ||
| } | ||
|
|
||
| public Map<String, String> writeProperties(FileFormat format) { | ||
| Map<String, String> writeProperties = Maps.newHashMap(); | ||
|
|
||
| switch (format) { | ||
| case PARQUET: | ||
| writeProperties.put(PARQUET_COMPRESSION, parquetCompressionCodec()); | ||
| String parquetCompressionLevel = parquetCompressionLevel(); | ||
| if (parquetCompressionLevel != null) { | ||
| writeProperties.put(PARQUET_COMPRESSION_LEVEL, parquetCompressionLevel); | ||
|
Member
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.
Contributor
Author
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. Sorry for this mistake. I will check how to valid the correctness of the case. |
||
| } | ||
| break; | ||
|
|
||
| case AVRO: | ||
| writeProperties.put(AVRO_COMPRESSION, avroCompressionCodec()); | ||
| String avroCompressionLevel = avroCompressionLevel(); | ||
| if (avroCompressionLevel != null) { | ||
| writeProperties.put(AVRO_COMPRESSION_LEVEL, avroCompressionLevel()); | ||
| } | ||
| break; | ||
|
|
||
| case ORC: | ||
| writeProperties.put(ORC_COMPRESSION, orcCompressionCodec()); | ||
| writeProperties.put(ORC_COMPRESSION_STRATEGY, orcCompressionStrategy()); | ||
| break; | ||
|
|
||
| default: | ||
| // skip | ||
| } | ||
|
|
||
| return writeProperties; | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -104,6 +104,7 @@ class SparkPositionDeltaWrite implements DeltaWrite, RequiresDistributionAndOrde | |
| private final Context context; | ||
|
|
||
| private boolean cleanupOnAbort = true; | ||
| private final Map<String, String> writeProperties; | ||
|
Contributor
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. Minor: This variable should be co-located with other final variables above. It is super minor but let's do this in a follow-up.
Contributor
Author
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. OK, I have raised a follow-up pr #8421. The pr has been merged. |
||
|
|
||
| SparkPositionDeltaWrite( | ||
| SparkSession spark, | ||
|
|
@@ -126,6 +127,7 @@ class SparkPositionDeltaWrite implements DeltaWrite, RequiresDistributionAndOrde | |
| this.extraSnapshotMetadata = writeConf.extraSnapshotMetadata(); | ||
| this.writeRequirements = writeConf.positionDeltaRequirements(command); | ||
| this.context = new Context(dataSchema, writeConf, info, writeRequirements); | ||
| this.writeProperties = writeConf.writeProperties(context.dataFileFormat); | ||
| } | ||
|
|
||
| @Override | ||
|
|
@@ -155,7 +157,7 @@ public DeltaWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) { | |
| // broadcast the table metadata as the writer factory will be sent to executors | ||
| Broadcast<Table> tableBroadcast = | ||
| sparkContext.broadcast(SerializableTableWithSize.copyOf(table)); | ||
| return new PositionDeltaWriteFactory(tableBroadcast, command, context); | ||
| return new PositionDeltaWriteFactory(tableBroadcast, command, context, writeProperties); | ||
| } | ||
|
|
||
| @Override | ||
|
|
@@ -326,11 +328,17 @@ private static class PositionDeltaWriteFactory implements DeltaWriterFactory { | |
| private final Broadcast<Table> tableBroadcast; | ||
| private final Command command; | ||
| private final Context context; | ||
| private final Map<String, String> writeProperties; | ||
|
|
||
| PositionDeltaWriteFactory(Broadcast<Table> tableBroadcast, Command command, Context context) { | ||
| PositionDeltaWriteFactory( | ||
| Broadcast<Table> tableBroadcast, | ||
| Command command, | ||
| Context context, | ||
| Map<String, String> writeProperties) { | ||
| this.tableBroadcast = tableBroadcast; | ||
| this.command = command; | ||
| this.context = context; | ||
| this.writeProperties = writeProperties; | ||
| } | ||
|
|
||
| @Override | ||
|
|
@@ -356,6 +364,7 @@ public DeltaWriter<InternalRow> createWriter(int partitionId, long taskId) { | |
| .dataSparkType(context.dataSparkType()) | ||
| .deleteFileFormat(context.deleteFileFormat()) | ||
| .positionDeleteSparkType(context.deleteSparkType()) | ||
| .writeProperties(writeProperties) | ||
| .build(); | ||
|
|
||
| if (command == DELETE) { | ||
|
|
||

Uh oh!
There was an error while loading. Please reload this page.