1
0

[HUDI-2526] Make spark.sql.parquet.writeLegacyFormat configurable (#3917)

This commit is contained in:
Sagar Sumit
2021-11-05 22:33:41 +05:30
committed by GitHub
parent 844346c3ab
commit 08c35a55b3
5 changed files with 51 additions and 16 deletions

View File

@@ -115,6 +115,18 @@ public class HoodieStorageConfig extends HoodieConfig {
.defaultValue(true)
.withDocumentation("Whether to use dictionary encoding");
public static final ConfigProperty<String> PARQUET_WRITE_LEGACY_FORMAT_ENABLED = ConfigProperty
.key("hoodie.parquet.writeLegacyFormat.enabled")
.defaultValue("false")
.withDocumentation("Sets spark.sql.parquet.writeLegacyFormat. If true, data will be written in a way of Spark 1.4 and earlier. "
+ "For example, decimal values will be written in Parquet's fixed-length byte array format which other systems such as Apache Hive and Apache Impala use. "
+ "If false, the newer format in Parquet will be used. For example, decimals will be written in int-based format.");
public static final ConfigProperty<String> PARQUET_OUTPUT_TIMESTAMP_TYPE = ConfigProperty
.key("hoodie.parquet.outputTimestampType")
.defaultValue("TIMESTAMP_MILLIS")
.withDocumentation("Sets spark.sql.parquet.outputTimestampType. Parquet timestamp type to use when Spark writes data to Parquet files.");
public static final ConfigProperty<String> HFILE_COMPRESSION_ALGORITHM_NAME = ConfigProperty
.key("hoodie.hfile.compression.algorithm")
.defaultValue("GZ")
@@ -312,6 +324,16 @@ public class HoodieStorageConfig extends HoodieConfig {
return this;
}
public Builder parquetWriteLegacyFormat(String parquetWriteLegacyFormat) {
storageConfig.setValue(PARQUET_WRITE_LEGACY_FORMAT_ENABLED, parquetWriteLegacyFormat);
return this;
}
public Builder parquetOutputTimestampType(String parquetOutputTimestampType) {
storageConfig.setValue(PARQUET_OUTPUT_TIMESTAMP_TYPE, parquetOutputTimestampType);
return this;
}
public Builder hfileCompressionAlgorithm(String hfileCompressionAlgorithm) {
storageConfig.setValue(HFILE_COMPRESSION_ALGORITHM_NAME, hfileCompressionAlgorithm);
return this;
@@ -347,5 +369,4 @@ public class HoodieStorageConfig extends HoodieConfig {
return storageConfig;
}
}
}

View File

@@ -1439,6 +1439,14 @@ public class HoodieWriteConfig extends HoodieConfig {
return getBoolean(HoodieStorageConfig.PARQUET_DICTIONARY_ENABLED);
}
public String parquetWriteLegacyFormatEnabled() {
return getString(HoodieStorageConfig.PARQUET_WRITE_LEGACY_FORMAT_ENABLED);
}
public String parquetOutputTimestampType() {
return getString(HoodieStorageConfig.PARQUET_OUTPUT_TIMESTAMP_TYPE);
}
public long getLogFileMaxSize() {
return getLong(HoodieStorageConfig.LOGFILE_MAX_SIZE);
}