ConeyLiu commented on code in PR #8313:
URL: https://github.com/apache/iceberg/pull/8313#discussion_r1292909162


##########
spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkWriteConf.java:
##########
@@ -418,4 +418,64 @@ 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();

Review Comment:
   We can not get an optional value because we have set the default value, 
right?



##########
spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkWriteConf.java:
##########
@@ -418,4 +418,64 @@ 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();

Review Comment:
   Sames here



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to