Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -278,7 +278,7 @@ public void rollback() {
* references, even if parsed from different files. Note: the context must be
* committed for this method to work.
*
* @return all parsed schemas, in the order they were parsed
* @return all parsed schemas
* @throws AvroTypeException if a schema reference cannot be resolved
*/
public List<Schema> resolveAllSchemas() {
Expand Down
6 changes: 3 additions & 3 deletions lang/java/avro/src/main/java/org/apache/avro/Schema.java
Original file line number Diff line number Diff line change
Expand Up @@ -1862,7 +1862,6 @@ private static Schema parseRecord(JsonNode schema, ParseContext context, String
Name name = parseName(schema, currentNameSpace);
String doc = parseDoc(schema);
Schema result = new RecordSchema(name, doc, isTypeError);
context.put(result);

JsonNode fieldsNode = schema.get("fields");
if (fieldsNode == null || !fieldsNode.isArray())
Expand All @@ -1879,6 +1878,7 @@ private static Schema parseRecord(JsonNode schema, ParseContext context, String
result.setFields(fields);
parsePropertiesAndLogicalType(schema, result, SCHEMA_RESERVED);
parseAliases(schema, result);
context.put(result);
return result;
}

Expand Down Expand Up @@ -1924,9 +1924,9 @@ private static Schema parseEnum(JsonNode schema, ParseContext context, String cu
}

Schema result = new EnumSchema(name, doc, symbols, defaultSymbol);
context.put(result);
parsePropertiesAndLogicalType(schema, result, ENUM_RESERVED);
parseAliases(schema, result);
context.put(result);
return result;
}

Expand Down Expand Up @@ -1959,9 +1959,9 @@ private static Schema parseFixed(JsonNode schema, ParseContext context, String c
throw new SchemaParseException("Invalid or no size: " + schema);

Schema result = new FixedSchema(name, doc, sizeNode.intValue());
context.put(result);
parsePropertiesAndLogicalType(schema, result, SCHEMA_RESERVED);
parseAliases(schema, result);
context.put(result);
return result;
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -724,7 +724,6 @@ protected Schema createSchema(Type type, Map<String, Schema> names) {
boolean error = Throwable.class.isAssignableFrom(c);
schema = Schema.createRecord(name, doc, space, error);
consumeAvroAliasAnnotation(c, schema);
names.put(c.getName(), schema);
for (Field field : getCachedFields(c))
if ((field.getModifiers() & (Modifier.TRANSIENT | Modifier.STATIC)) == 0
&& !field.isAnnotationPresent(AvroIgnore.class)) {
Expand Down Expand Up @@ -770,6 +769,7 @@ protected Schema createSchema(Type type, Map<String, Schema> names) {
}
names.put(fullName, schema);
}
names.put(c.getName(), schema);
return schema;
}
return super.createSchema(type, names);
Expand Down Expand Up @@ -908,11 +908,7 @@ public Protocol getProtocol(Class iface) {
}
}

// reverse types, since they were defined in reference order
List<Schema> types = new ArrayList<>(names.values());
Collections.reverse(types);
protocol.setTypes(types);

protocol.setTypes(new ArrayList<>(names.values()));
return protocol;
}

Expand Down
14 changes: 9 additions & 5 deletions lang/java/avro/src/test/java/org/apache/avro/TestSchema.java
Original file line number Diff line number Diff line change
Expand Up @@ -32,7 +32,10 @@
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.function.Function;
import java.util.stream.Collectors;

import com.fasterxml.jackson.core.JsonProcessingException;
import com.fasterxml.jackson.databind.JsonNode;
Expand Down Expand Up @@ -606,13 +609,14 @@ void testParseMultipleFile() throws IOException {
parser.parse(f1);
parser.parse(f2);
parser.parse(f3);
final List<Schema> schemas = parser.getParsedNamedSchemas();
final Map<String, Schema> schemas = parser.getParsedNamedSchemas().stream()
.collect(Collectors.toMap(Schema::getName, Function.identity()));
Assertions.assertEquals(3, schemas.size());
Schema schemaAppEvent = schemas.get(0);
Schema schemaDocInfo = schemas.get(1);
Schema schemaResponse = schemas.get(2);
Schema schemaAppEvent = schemas.get("ApplicationEvent");
Schema schemaDocInfo = schemas.get("DocumentInfo");
Schema schemaResponse = schemas.get("MyResponse");
Assertions.assertNotNull(schemaAppEvent);
Assertions.assertEquals(3, schemaAppEvent.getFields().size());
Assertions.assertEquals(4, schemaAppEvent.getFields().size());
Field documents = schemaAppEvent.getField("documents");
Schema docSchema = documents.schema().getTypes().get(1).getElementType();
Assertions.assertEquals(docSchema, schemaDocInfo);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,22 @@
}],
"doc": "",
"default": null
},
{
"name": "response",
"type": {
"namespace": "model",
"type": "record",
"doc": "",
"name": "MyResponse",
"fields": [
{
"name": "isSuccessful",
"type": "boolean",
"doc": "Indicator for successful or unsuccessful call"
}
]
}
}
]

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,22 @@
}],
"doc": "",
"default": null
},
{
"name": "response",
"type": {
"namespace": "model",
"type": "record",
"doc": "",
"name": "MyResponse",
"fields": [
{
"name": "isSuccessful",
"type": "boolean",
"doc": "Indicator for successful or unsuccessful call"
}
]
}
}
]

Expand Down
Loading