Skip to content

Conversation

@funcheetah
Copy link

No description provided.

cols.addAll(getPartitionCols(table));
}

return convertFieldSchemaToAvroSchema(recordName, recordNamespace, true, cols);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Instead of passing cols which is a List can we create a struct field schema here and call HiveTypeToAvroType directly? We can get rid of convertFieldSchemaToAvroSchema

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I see this is a tail reference/call, we can just move the code in convertFieldSchemaToAvroSchema inside convertHiveSchemaToAvro. But aside from that, what do you mean by "create a struct field schema"? I see the parseSchemaFromStruct method from HiveTypeToAvroType, but it is a private method, are you referring to this method?

for (int i = 0; i < fieldNames.size(); ++i) {
final TypeInfo fieldTypeInfo = fieldTypeInfos.get(i);
String fieldName = fieldNames.get(i);
fieldName = removePrefix(fieldName);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Do we need this? I think field names being passed here are relative, they come from StructTypeInfo.getAllStructFieldNames() so I don't think they are qualified from the root. A . in the field name is probably the actual name of the field here.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yeah, I think it can be removed? also since we are dealing with hive, I feel the field names won't contain . anyways.

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This should be easy to verify.

Comment on lines +70 to +72
// We don't cache the structType because otherwise it could be possible that a field
// "lastname" is of type "firstname", where firstname is a compiled class.
// This will lead to ambiguity.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I am not sure what this comment means. Which cache are we referring to?

if (schemaStr != null) {
schema = AvroSchemaUtil.toIceberg(new org.apache.avro.Schema.Parser().parse(schemaStr));
org.apache.avro.Schema avroSchema = new org.apache.avro.Schema.Parser().parse(schemaStr);
org.apache.avro.Schema hiveSchema = LegacyHiveSchemaUtils.convertHiveSchemaToAvro(table);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

At this step during the conversion we pass mkFieldsOptional as true to make fields nullable, but in the very next line we remove nullables from the schema. Can we just mark fields as non-nullable to being with and remove LegacyHiveSchemaUtils.extractActualTypeIfFieldIsNullableTypeRecord?

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Good catch, I think I can change the signature of convertHiveSchemaToAvro to add this boolean flag parameter to `convertHiveSchemaToAvro(Table table, boolean mkFieldsOptional), so that this function directly return a non-null version of the schema.

import org.slf4j.LoggerFactory;


public class LegacyHiveSchemaUtils {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The code in this class is way too verbose. This can be heavily simplified by using Visitors

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I feel the same, but I feel this will require non-trivial refactor of the code. I think we also want to publish this code soon, so there is this trade-off.

import org.codehaus.jackson.node.JsonNodeFactory;


public class HiveTypeToAvroType {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can we add test cases?

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

will do later, but the integration test I ran already passed all the tables.


org.apache.avro.Schema tableSchema = avroSchema;
boolean isHiveSchemaEvolved =
LegacyHiveSchemaUtils.isRecordSchemaEvolved(avroSchemaWithoutNullable, hiveSchemaWithoutNullable);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Seems like isRecordSchemaEvolved has to traverse the whole Schema tree. What do we gain by checking this first rather than just merging directly?

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think this is actually a good point. I think the 2 logic can be combined into just one pass.

Copy link

@wmoustafa wmoustafa left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Still going through the patch. Currently, there are many ways conversions take place and seems they could be simplified. For example, we can go from Hive type string to Hive TypeInfo (one type info to represent the whole schema) then to Avro schema.

for (int i = 0; i < fieldNames.size(); ++i) {
final TypeInfo fieldTypeInfo = fieldTypeInfos.get(i);
String fieldName = fieldNames.get(i);
fieldName = removePrefix(fieldName);

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This should be easy to verify.

schema = parseSchemaFromUnion((UnionTypeInfo) typeInfo, recordNamespace, recordName);
break;
default:
throw new UnsupportedOperationException("Conversion from " + category + " not supported");

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

is not supported

Comment on lines +120 to +121
// For example, in tracking.CommunicationRequestEvent.specificRequest,
// PropGenerated and PropExternalCommunication have the same structure. In case of duplicate typeinfos, we generate

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Best not to mention actual table and field names.


final List<FieldSchema> cols = new ArrayList<>();

cols.addAll(table.getSd().getCols());

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

There were concerns around using getSd().getCols(). Could you check if we should use HiveMetastoreClient.getSchema()?

for (TypeInfo ti : typeInfos) {
Schema candidate;
if (ti instanceof StructTypeInfo) {
StructTypeInfo sti = (StructTypeInfo) ti;

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

sti --> structTypeInfo

// a new record type for the duplicates.
List<Schema> schemas = new ArrayList<>();

for (TypeInfo ti : typeInfos) {

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

ti --> typeInfo

private static final String SHORT_TYPE_NAME = "short";
private static final String BYTE_TYPE_NAME = "byte";

public HiveTypeToAvroType(String namespace, boolean mkFieldsOptional) {

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Does it make sense to convert this to a utility class and move those parameters to convertFieldsTypeInfoToAvroSchema?

Comment on lines +59 to +60
Schema convertFieldsTypeInfoToAvroSchema(String recordNamespace, String recordName, List<String> fieldNames,
List<TypeInfo> fieldTypeInfos) {

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can we avoid using list of field names and list of field types throughout the PR? For example instead of of List<String> fieldNames and List<TypeInfo> fieldTypeInfos, we just pass StructTypeInfo. Usually the input is already a StructTypeInfo and then it is broken down to two lists then dealt with here. In cases where the original input comes as two lists, we can combine them using TypeInfoFactory. getStructTypeInfo() from Hive.

// We don't cache the structType because otherwise it could be possible that a field
// "lastname" is of type "firstname", where firstname is a compiled class.
// This will lead to ambiguity.
schema = parseSchemaFromStruct((StructTypeInfo) typeInfo, recordNamespace, recordName);

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can we rename this and other methods to something like convertStructTypeInfoToAvroSchema?

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

4 participants