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 @@ -683,6 +683,14 @@ protected Schema createSchema(Type type, Map<String, Schema> names) {
setElement(result, component);
return result;
}
AvroEncode enc = ReflectionUtil.getAvroEncode(c);
if (enc != null) {
try {
return enc.using().getDeclaredConstructor().newInstance().getSchema();
} catch (Exception e) {
throw new AvroRuntimeException("Could not create schema from custom serializer for " + c.getName());
}
}
AvroSchema explicit = c.getAnnotation(AvroSchema.class);
if (explicit != null) // explicit schema
return new Schema.Parser().parse(explicit.value());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -37,23 +37,23 @@
@Test
void testWithinClass() throws IOException {

var wrapper = new Wrapper(new R1("test"));
var wrapper = new Wrapper(new R1("321"));

var read = readWrite(wrapper);

assertEquals("test", wrapper.getR1().getValue());
assertEquals("test used this", read.getR1().getValue());
assertEquals("321", wrapper.getR1().getValue());
assertEquals("321 used this", read.getR1().getValue());
}

@Test
void testDirect() throws IOException {

var r1 = new R1("test");
var r1 = new R1("123");

var read = readWrite(r1);

assertEquals("test", r1.getValue());
assertEquals("test used this", read.getValue());
assertEquals("123", r1.getValue());
assertEquals("123 used this", read.getValue());
}

@Test
Expand Down Expand Up @@ -127,14 +127,15 @@
public static class R1Encoding extends CustomEncoding<R1> {

{
schema = Schema.createRecord("R1", null, null, false,
Arrays.asList(new Schema.Field("value", Schema.create(Schema.Type.STRING), null, null)));
schema = Schema.createRecord("R1", null, "org.apache.avro.reflect.TestAvroEncode", false,
Arrays.asList(new Schema.Field("value", Schema.create(Schema.Type.INT), null, null)));
}

@Override
protected void write(Object datum, Encoder out) throws IOException {
if (datum instanceof R1) {
out.writeString(((R1) datum).getValue());
var value = ((R1) datum).getValue();
out.writeInt(Integer.parseInt(value));
} else {
throw new AvroTypeException("Expected R1, got " + datum.getClass());
}
Expand All @@ -143,7 +144,7 @@

@Override
protected R1 read(Object reuse, Decoder in) throws IOException {
return new R1(in.readString() + " used this");
return new R1(in.readInt() + " used this");
}
}

Expand Down