bryanck commented on code in PR #8701: URL: https://github.com/apache/iceberg/pull/8701#discussion_r1379406817
########## core/src/main/java/org/apache/iceberg/avro/TypeToSchema.java: ########## @@ -238,8 +218,68 @@ public Schema primitive(Type.PrimitiveType primitive) { throw new UnsupportedOperationException("Unsupported type ID: " + primitive.typeId()); } - results.put(primitive, primitiveSchema); - return primitiveSchema; } + + static class WithTypeToName extends TypeToSchema { + + private final Map<Type, Schema> results = Maps.newHashMap(); + + WithTypeToName(Map<Types.StructType, String> names) { + super((id, struct) -> names.get(struct)); + } + + Map<Type, Schema> getConversionMap() { + return results; + } + + @Override + public Schema struct(Types.StructType struct, List<Schema> fieldSchemas) { + Schema recordSchema = results.get(struct); + if (recordSchema != null) { + return recordSchema; + } + + recordSchema = super.struct(struct, fieldSchemas); + results.put(struct, recordSchema); + return recordSchema; + } + + @Override + public Schema list(Types.ListType list, Schema elementSchema) { + Schema listSchema = results.get(list); + if (listSchema != null) { + return listSchema; + } + + listSchema = super.list(list, elementSchema); + results.put(list, listSchema); + return listSchema; + } + + @Override + public Schema map(Types.MapType map, Schema keySchema, Schema valueSchema) { + Schema mapSchema = results.get(map); + if (mapSchema != null) { + return mapSchema; + } + + mapSchema = super.map(map, keySchema, valueSchema); + results.put(map, mapSchema); + return mapSchema; + } + + @Override + public Schema primitive(Type.PrimitiveType primitive) { + Schema primitiveSchema = super.primitive(primitive); + results.put(primitive, primitiveSchema); + return primitiveSchema; + } + } + + static class WithNamesFunction extends TypeToSchema { + WithNamesFunction(BiFunction<Integer, Types.StructType, String> namesFunction) { Review Comment: I went ahead and added the caching by record name also. -- 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: issues-unsubscr...@iceberg.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org For additional commands, e-mail: issues-h...@iceberg.apache.org