[HUDI-1181] Fix decimal type display issue for record key field (#1953)
* [HUDI-1181] Fix decimal type display issue for record key field * Remove getNestedFieldVal method from DataSourceUtils * resolve comments Co-authored-by: Wenning Ding <wenningd@amazon.com>
This commit is contained in:
@@ -18,10 +18,6 @@
|
||||
|
||||
package org.apache.hudi.avro;
|
||||
|
||||
import org.apache.avro.JsonProperties;
|
||||
import java.time.LocalDate;
|
||||
import org.apache.avro.LogicalTypes;
|
||||
import org.apache.avro.generic.GenericData.Record;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.util.StringUtils;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
@@ -29,11 +25,17 @@ import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.exception.SchemaCompatabilityException;
|
||||
|
||||
import org.apache.avro.Conversions.DecimalConversion;
|
||||
import org.apache.avro.JsonProperties;
|
||||
import org.apache.avro.LogicalTypes;
|
||||
import org.apache.avro.LogicalTypes.Decimal;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.Schema.Field;
|
||||
import org.apache.avro.generic.GenericData;
|
||||
import org.apache.avro.generic.GenericData.Record;
|
||||
import org.apache.avro.generic.GenericDatumReader;
|
||||
import org.apache.avro.generic.GenericDatumWriter;
|
||||
import org.apache.avro.generic.GenericFixed;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.avro.io.BinaryDecoder;
|
||||
@@ -50,7 +52,9 @@ import java.io.ByteArrayOutputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.io.OutputStream;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.time.LocalDate;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
@@ -433,9 +437,6 @@ public class HoodieAvroUtils {
|
||||
/**
|
||||
* This method converts values for fields with certain Avro/Parquet data types that require special handling.
|
||||
*
|
||||
* Logical Date Type is converted to actual Date value instead of Epoch Integer which is how it is
|
||||
* represented/stored in parquet.
|
||||
*
|
||||
* @param fieldSchema avro field schema
|
||||
* @param fieldValue avro field value
|
||||
* @return field value either converted (for certain data types) or as it is.
|
||||
@@ -445,23 +446,44 @@ public class HoodieAvroUtils {
|
||||
return fieldValue;
|
||||
}
|
||||
|
||||
if (isLogicalTypeDate(fieldSchema)) {
|
||||
return LocalDate.ofEpochDay(Long.parseLong(fieldValue.toString()));
|
||||
if (fieldSchema.getType() == Schema.Type.UNION) {
|
||||
for (Schema schema : fieldSchema.getTypes()) {
|
||||
if (schema.getType() != Schema.Type.NULL) {
|
||||
return convertValueForAvroLogicalTypes(schema, fieldValue);
|
||||
}
|
||||
}
|
||||
}
|
||||
return fieldValue;
|
||||
return convertValueForAvroLogicalTypes(fieldSchema, fieldValue);
|
||||
}
|
||||
|
||||
/**
|
||||
* Given an Avro field schema checks whether the field is of Logical Date Type or not.
|
||||
* This method converts values for fields with certain Avro Logical data types that require special handling.
|
||||
*
|
||||
* Logical Date Type is converted to actual Date value instead of Epoch Integer which is how it is
|
||||
* represented/stored in parquet.
|
||||
*
|
||||
* Decimal Data Type is converted to actual decimal value instead of bytes/fixed which is how it is
|
||||
* represented/stored in parquet.
|
||||
*
|
||||
* @param fieldSchema avro field schema
|
||||
* @return boolean indicating whether fieldSchema is of Avro's Date Logical Type
|
||||
* @param fieldValue avro field value
|
||||
* @return field value either converted (for certain data types) or as it is.
|
||||
*/
|
||||
private static boolean isLogicalTypeDate(Schema fieldSchema) {
|
||||
if (fieldSchema.getType() == Schema.Type.UNION) {
|
||||
return fieldSchema.getTypes().stream().anyMatch(schema -> schema.getLogicalType() == LogicalTypes.date());
|
||||
private static Object convertValueForAvroLogicalTypes(Schema fieldSchema, Object fieldValue) {
|
||||
if (fieldSchema.getLogicalType() == LogicalTypes.date()) {
|
||||
return LocalDate.ofEpochDay(Long.parseLong(fieldValue.toString()));
|
||||
} else if (fieldSchema.getLogicalType() instanceof LogicalTypes.Decimal) {
|
||||
Decimal dc = (Decimal) fieldSchema.getLogicalType();
|
||||
DecimalConversion decimalConversion = new DecimalConversion();
|
||||
if (fieldSchema.getType() == Schema.Type.FIXED) {
|
||||
return decimalConversion.fromFixed((GenericFixed) fieldValue, fieldSchema,
|
||||
LogicalTypes.decimal(dc.getPrecision(), dc.getScale()));
|
||||
} else if (fieldSchema.getType() == Schema.Type.BYTES) {
|
||||
return decimalConversion.fromBytes((ByteBuffer) fieldValue, fieldSchema,
|
||||
LogicalTypes.decimal(dc.getPrecision(), dc.getScale()));
|
||||
}
|
||||
}
|
||||
return fieldSchema.getLogicalType() == LogicalTypes.date();
|
||||
return fieldValue;
|
||||
}
|
||||
|
||||
public static Schema getNullSchema() {
|
||||
|
||||
Reference in New Issue
Block a user