1
0

[HUDI-2958] Automatically set spark.sql.parquet.writelegacyformat, when using bulkinsert to insert data which contains decimalType (#4253)

This commit is contained in:
xiarixiaoyao
2021-12-17 21:58:02 +08:00
committed by GitHub
parent e4cfb421c0
commit 9246b16492
6 changed files with 115 additions and 1 deletions

View File

@@ -33,6 +33,8 @@ import org.apache.spark.sql.util.CaseInsensitiveStringMap;
import java.util.Map;
import static org.apache.hudi.DataSourceUtils.mayBeOverwriteParquetWriteLegacyFormatProp;
/**
* DataSource V2 implementation for managing internal write logic. Only called internally.
* This class is only compatible with datasource V2 API in Spark 3.
@@ -53,6 +55,8 @@ public class DefaultSource extends BaseDefaultSource implements TableProvider {
HoodieTableConfig.POPULATE_META_FIELDS.defaultValue()));
boolean arePartitionRecordsSorted = Boolean.parseBoolean(properties.getOrDefault(HoodieInternalConfig.BULKINSERT_ARE_PARTITIONER_RECORDS_SORTED,
Boolean.toString(HoodieInternalConfig.DEFAULT_BULKINSERT_ARE_PARTITIONER_RECORDS_SORTED)));
// Auto set the value of "hoodie.parquet.writeLegacyFormat.enabled"
mayBeOverwriteParquetWriteLegacyFormatProp(properties, schema);
// 1st arg to createHoodieConfig is not really required to be set. but passing it anyways.
HoodieWriteConfig config = DataSourceUtils.createHoodieConfig(properties.get(HoodieWriteConfig.AVRO_SCHEMA_STRING.key()), path, tblName, properties);
return new HoodieDataSourceInternalTable(instantTime, config, schema, getSparkSession(),