voonhous commented on code in PR #17740:
URL: https://github.com/apache/hudi/pull/17740#discussion_r2656479003
##########
hudi-common/src/main/java/org/apache/hudi/avro/processors/DecimalLogicalTypeProcessor.java:
##########
@@ -34,15 +34,15 @@ public abstract class DecimalLogicalTypeProcessor extends
JsonFieldProcessor {
/**
* Check if the given schema is a valid decimal type configuration.
*/
- protected static boolean isValidDecimalTypeConfig(Schema schema) {
- LogicalTypes.Decimal decimalType = (LogicalTypes.Decimal)
schema.getLogicalType();
+ protected static boolean isValidDecimalTypeConfig(HoodieSchema schema) {
+ LogicalTypes.Decimal decimalType = (LogicalTypes.Decimal)
schema.toAvroSchema().getLogicalType();
Review Comment:
Since this is still under the Avro package. I think converting everything to
avro make sense.
During my test, IIRC, via the `HoodieSchema#createDecimal` helper, but
creating a decimal that is backed by fix with an improper fixed size will fail
silently.
A `HoodieSchema` will still be created, but it will not be an instance of
`HoodieSchema.Decimal`. This might be a validation that we need to add in that
helper method. Just something orthogonal that i happened to think of when
reading this part of the code.
##########
hudi-common/src/main/java/org/apache/hudi/avro/processors/DecimalLogicalTypeProcessor.java:
##########
@@ -53,25 +53,23 @@ protected static boolean isValidDecimalTypeConfig(Schema
schema) {
* @return Pair object, with left as boolean indicating if the parsing was
successful and right as the
* BigDecimal value.
*/
- protected static Pair<Boolean, BigDecimal> parseObjectToBigDecimal(Object
obj, Schema schema) {
+ protected static Pair<Boolean, BigDecimal> parseObjectToBigDecimal(Object
obj, HoodieSchema.Decimal schema) {
BigDecimal bigDecimal = null;
- LogicalTypes.Decimal logicalType = (LogicalTypes.Decimal)
schema.getLogicalType();
try {
if (obj instanceof BigDecimal) {
- bigDecimal = ((BigDecimal) obj).setScale(logicalType.getScale(),
RoundingMode.UNNECESSARY);
+ bigDecimal = ((BigDecimal) obj).setScale(schema.getScale(),
RoundingMode.UNNECESSARY);
} else if (obj instanceof String) {
// Case 2: Object is a number in String format.
try {
//encoded big decimal
- bigDecimal =
HoodieAvroUtils.convertBytesToBigDecimal(decodeStringToBigDecimalBytes(obj),
- (LogicalTypes.Decimal) schema.getLogicalType());
+ bigDecimal =
HoodieSchemaUtils.convertBytesToBigDecimal(decodeStringToBigDecimalBytes(obj),
schema);
} catch (IllegalArgumentException e) {
//no-op
}
}
// None fixed byte or fixed byte conversion failure would end up here.
if (bigDecimal == null) {
- bigDecimal = new BigDecimal(obj.toString(), new
MathContext(logicalType.getPrecision(),
RoundingMode.UNNECESSARY)).setScale(logicalType.getScale(),
RoundingMode.UNNECESSARY);
+ bigDecimal = new BigDecimal(obj.toString(), new
MathContext(schema.getPrecision(),
RoundingMode.UNNECESSARY)).setScale(schema.getScale(),
RoundingMode.UNNECESSARY);
}
} catch (java.lang.NumberFormatException | ArithmeticException ignored) {
Review Comment:
Nit: Possible to import this?
##########
hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/ProtoConversionUtil.java:
##########
@@ -474,26 +466,26 @@ private static Object convertObject(HoodieSchema schema,
Object value) {
Map<Object, Object> mapValue = (Map) value;
Map<Object, Object> mapCopy = new HashMap<>(mapValue.size());
for (Map.Entry<Object, Object> entry : mapValue.entrySet()) {
-
mapCopy.put(convertObject(HoodieSchema.fromAvroSchema(STRING_SCHEMA),
entry.getKey()), convertObject(schema.getValueType(), entry.getValue()));
+ mapCopy.put(convertObject(STRING_SCHEMA, entry.getKey()),
convertObject(schema.getValueType(), entry.getValue()));
}
return mapCopy;
case NULL:
return null;
case RECORD:
GenericData.Record newRecord = new
GenericData.Record(schema.toAvroSchema());
Message messageValue = (Message) value;
- Descriptors.FieldDescriptor[] orderedFields =
getOrderedFields(schema.toAvroSchema(), messageValue);
- for (Schema.Field field : schema.toAvroSchema().getFields()) {
+ Descriptors.FieldDescriptor[] orderedFields =
getOrderedFields(schema, messageValue);
+ for (HoodieSchemaField field : schema.getFields()) {
int position = field.pos();
Descriptors.FieldDescriptor fieldDescriptor =
orderedFields[position];
Object convertedValue;
- Schema fieldSchema = field.schema();
+ HoodieSchema fieldSchema = field.schema();
// if incoming message does not contain the field, fieldDescriptor
will be null
// if the field schema is a union, it is nullable
- if (fieldSchema.getType() == Schema.Type.UNION && (fieldDescriptor
== null || (!fieldDescriptor.isRepeated() &&
!messageValue.hasField(fieldDescriptor)))) {
+ if (fieldSchema.getType() == HoodieSchemaType.UNION &&
(fieldDescriptor == null || (!fieldDescriptor.isRepeated() &&
!messageValue.hasField(fieldDescriptor)))) {
Review Comment:
Nit: Check if `fieldSchema#isNullable`?
##########
hudi-common/src/main/java/org/apache/hudi/avro/MercifulJsonConverter.java:
##########
@@ -221,20 +210,20 @@ protected JsonFieldProcessor getProcessorForSchema(Schema
schema) {
/**
* Build type processor map for each avro type.
*/
- private Map<Schema.Type, JsonFieldProcessor> getFieldTypeProcessors() {
- Map<Schema.Type, JsonFieldProcessor> fieldTypeProcessors = new
EnumMap<>(Schema.Type.class);
- fieldTypeProcessors.put(Type.STRING, generateStringTypeHandler());
- fieldTypeProcessors.put(Type.BOOLEAN, generateBooleanTypeHandler());
- fieldTypeProcessors.put(Type.DOUBLE, generateDoubleTypeHandler());
- fieldTypeProcessors.put(Type.FLOAT, generateFloatTypeHandler());
- fieldTypeProcessors.put(Type.INT, generateIntTypeHandler());
- fieldTypeProcessors.put(Type.LONG, generateLongTypeHandler());
- fieldTypeProcessors.put(Type.ARRAY, generateArrayTypeHandler());
- fieldTypeProcessors.put(Type.RECORD, generateRecordTypeHandler());
- fieldTypeProcessors.put(Type.ENUM, generateEnumTypeHandler());
- fieldTypeProcessors.put(Type.MAP, generateMapTypeHandler());
- fieldTypeProcessors.put(Type.BYTES, generateBytesTypeHandler());
- fieldTypeProcessors.put(Type.FIXED, generateFixedTypeHandler());
+ private Map<HoodieSchemaType, JsonFieldProcessor> getFieldTypeProcessors() {
+ Map<HoodieSchemaType, JsonFieldProcessor> fieldTypeProcessors = new
EnumMap<>(HoodieSchemaType.class);
+ fieldTypeProcessors.put(HoodieSchemaType.STRING,
generateStringTypeHandler());
+ fieldTypeProcessors.put(HoodieSchemaType.BOOLEAN,
generateBooleanTypeHandler());
+ fieldTypeProcessors.put(HoodieSchemaType.DOUBLE,
generateDoubleTypeHandler());
+ fieldTypeProcessors.put(HoodieSchemaType.FLOAT,
generateFloatTypeHandler());
+ fieldTypeProcessors.put(HoodieSchemaType.INT, generateIntTypeHandler());
+ fieldTypeProcessors.put(HoodieSchemaType.LONG, generateLongTypeHandler());
+ fieldTypeProcessors.put(HoodieSchemaType.ARRAY,
generateArrayTypeHandler());
+ fieldTypeProcessors.put(HoodieSchemaType.RECORD,
generateRecordTypeHandler());
+ fieldTypeProcessors.put(HoodieSchemaType.ENUM, generateEnumTypeHandler());
+ fieldTypeProcessors.put(HoodieSchemaType.MAP, generateMapTypeHandler());
+ fieldTypeProcessors.put(HoodieSchemaType.BYTES,
generateBytesTypeHandler());
+ fieldTypeProcessors.put(HoodieSchemaType.FIXED,
generateFixedTypeHandler());
Review Comment:
Is it possible to shift some of the KV pairs in
`getLogicalFieldTypeProcessors` to this scope?
e.g. DECIMAL, TIME, DATE, etc.
--
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]