[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:
@@ -43,9 +43,6 @@ import org.apache.hudi.keygen.KeyGenerator;
|
||||
import org.apache.hudi.keygen.parser.AbstractHoodieDateTimeParser;
|
||||
import org.apache.hudi.table.BulkInsertPartitioner;
|
||||
|
||||
import org.apache.avro.LogicalTypes;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.Schema.Field;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
@@ -55,12 +52,10 @@ import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.time.LocalDate;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
/**
|
||||
* Utilities used throughout the data source.
|
||||
@@ -69,42 +64,6 @@ public class DataSourceUtils {
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(DataSourceUtils.class);
|
||||
|
||||
/**
|
||||
* Obtain value of the provided field, denoted by dot notation. e.g: a.b.c
|
||||
*/
|
||||
public static Object getNestedFieldVal(GenericRecord record, String fieldName, boolean returnNullIfNotFound) {
|
||||
String[] parts = fieldName.split("\\.");
|
||||
GenericRecord valueNode = record;
|
||||
int i = 0;
|
||||
for (; i < parts.length; i++) {
|
||||
String part = parts[i];
|
||||
Object val = valueNode.get(part);
|
||||
if (val == null) {
|
||||
break;
|
||||
}
|
||||
|
||||
// return, if last part of name
|
||||
if (i == parts.length - 1) {
|
||||
Schema fieldSchema = valueNode.getSchema().getField(part).schema();
|
||||
return convertValueForSpecificDataTypes(fieldSchema, val);
|
||||
} else {
|
||||
// VC: Need a test here
|
||||
if (!(val instanceof GenericRecord)) {
|
||||
throw new HoodieException("Cannot find a record at part value :" + part);
|
||||
}
|
||||
valueNode = (GenericRecord) val;
|
||||
}
|
||||
}
|
||||
|
||||
if (returnNullIfNotFound) {
|
||||
return null;
|
||||
} else {
|
||||
throw new HoodieException(
|
||||
fieldName + "(Part -" + parts[i] + ") field not found in record. Acceptable fields were :"
|
||||
+ valueNode.getSchema().getFields().stream().map(Field::name).collect(Collectors.toList()));
|
||||
}
|
||||
}
|
||||
|
||||
public static String getTablePath(FileSystem fs, Path[] userProvidedPaths) throws IOException {
|
||||
LOG.info("Getting table path..");
|
||||
for (Path path : userProvidedPaths) {
|
||||
@@ -121,39 +80,6 @@ public class DataSourceUtils {
|
||||
throw new TableNotFoundException("Unable to find a hudi table for the user provided paths.");
|
||||
}
|
||||
|
||||
/**
|
||||
* 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.
|
||||
*/
|
||||
private static Object convertValueForSpecificDataTypes(Schema fieldSchema, Object fieldValue) {
|
||||
if (fieldSchema == null) {
|
||||
return fieldValue;
|
||||
}
|
||||
|
||||
if (isLogicalTypeDate(fieldSchema)) {
|
||||
return LocalDate.ofEpochDay(Long.parseLong(fieldValue.toString()));
|
||||
}
|
||||
return fieldValue;
|
||||
}
|
||||
|
||||
/**
|
||||
* Given an Avro field schema checks whether the field is of Logical Date Type or not.
|
||||
*
|
||||
* @param fieldSchema avro field schema
|
||||
* @return boolean indicating whether fieldSchema is of Avro's Date Logical Type
|
||||
*/
|
||||
private static boolean isLogicalTypeDate(Schema fieldSchema) {
|
||||
if (fieldSchema.getType() == Schema.Type.UNION) {
|
||||
return fieldSchema.getTypes().stream().anyMatch(schema -> schema.getLogicalType() == LogicalTypes.date());
|
||||
}
|
||||
return fieldSchema.getLogicalType() == LogicalTypes.date();
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a key generator class via reflection, passing in any configs needed.
|
||||
* <p>
|
||||
|
||||
Reference in New Issue
Block a user