[
https://issues.apache.org/jira/browse/AVRO-1340?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16461286#comment-16461286
]
ASF GitHub Bot commented on AVRO-1340:
--------------------------------------
cutting closed pull request #298: AVRO-1340: Added Enum Defaults and unit tests.
URL: https://github.com/apache/avro/pull/298
This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:
As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):
diff --git a/doc/src/content/xdocs/idl.xml b/doc/src/content/xdocs/idl.xml
index c9a809f61..08c4585c2 100644
--- a/doc/src/content/xdocs/idl.xml
+++ b/doc/src/content/xdocs/idl.xml
@@ -152,15 +152,29 @@ protocol MyProtocol {
<section id="format_enums">
<title>Defining an Enumeration</title>
<p>
- Enums are defined in Avro IDL using a syntax similar to C or Java:
+ Enums are defined in Avro IDL using a syntax similar to C or Java. An
Avro Enum supports optional default values.
+ In the case that a reader schema is unable to recognize a symbol
written by the writer, the reader will fall back to using the defined default
value.
+ This default is only used when an incompatible symbol is read. It is
not used if the enum field is missing.
+ </p>
+ <p>
+ Example Writer Enum Definition
</p>
<source>
-enum Suit {
- SPADES, DIAMONDS, CLUBS, HEARTS
+enum Shapes {
+ SQUARE, TRIANGLE, CIRCLE, OVAL
}
</source>
<p>
- Note that, unlike the JSON format, anonymous enums cannot be defined.
+ Example Reader Enum Definition
+ </p>
+ <source>
+enum Shapes {
+ SQUARE, TRIANGLE, CIRCLE
+} = CIRCLE;
+ </source>
+ <p>
+ In the above example, the reader will use the default value of CIRCLE
whenever reading data written with the OVAL symbol of the writer.
+ Also note that, unlike the JSON format, anonymous enums cannot be
defined.
</p>
</section>
<section id="format_fixed">
diff --git a/lang/java/avro/src/main/java/org/apache/avro/Schema.java
b/lang/java/avro/src/main/java/org/apache/avro/Schema.java
index 9333b7984..9fcbfd3a7 100644
--- a/lang/java/avro/src/main/java/org/apache/avro/Schema.java
+++ b/lang/java/avro/src/main/java/org/apache/avro/Schema.java
@@ -102,7 +102,7 @@
private LogicalType logicalType = null;
Schema(Type type) {
- super(SCHEMA_RESERVED);
+ super(type == Type.ENUM ? ENUM_RESERVED : SCHEMA_RESERVED);
this.type = type;
}
@@ -127,6 +127,11 @@ public static Schema create(Type type) {
"doc", "fields", "items", "name", "namespace",
"size", "symbols", "values", "type", "aliases");
}
+ private static final Set<String> ENUM_RESERVED = new HashSet<>();
+ static {
+ ENUM_RESERVED.add("default");
+ ENUM_RESERVED.addAll(SCHEMA_RESERVED);
+ }
int hashCode = NO_HASHCODE;
@@ -171,7 +176,14 @@ public static Schema createRecord(String name, String doc,
String namespace,
public static Schema createEnum(String name, String doc, String namespace,
List<String> values) {
return new EnumSchema(new Name(name, namespace), doc,
- new LockableArrayList<>(values));
+ new LockableArrayList<>(values), null);
+ }
+
+ /** Create an enum schema. */
+ public static Schema createEnum(String name, String doc, String namespace,
+ List<String> values, String enumDefault) {
+ return new EnumSchema(new Name(name, namespace), doc,
+ new LockableArrayList<>(values), enumDefault);
}
/** Create an array schema. */
@@ -233,6 +245,11 @@ public void setFields(List<Field> fields) {
throw new AvroRuntimeException("Not an enum: "+this);
}
+ /** If this is an enum, return its default value. */
+ public String getEnumDefault() {
+ throw new AvroRuntimeException("Not an enum: "+this);
+ }
+
/** If this is an enum, return a symbol's ordinal value. */
public int getEnumOrdinal(String symbol) {
throw new AvroRuntimeException("Not an enum: "+this);
@@ -748,15 +765,19 @@ void fieldsToJson(Names names, JsonGenerator gen) throws
IOException {
private static class EnumSchema extends NamedSchema {
private final List<String> symbols;
private final Map<String,Integer> ordinals;
+ private final String enumDefault;
public EnumSchema(Name name, String doc,
- LockableArrayList<String> symbols) {
+ LockableArrayList<String> symbols, String enumDefault) {
super(Type.ENUM, name, doc);
this.symbols = symbols.lock();
this.ordinals = new HashMap<>();
+ this.enumDefault = enumDefault;
int i = 0;
for (String symbol : symbols)
if (ordinals.put(validateName(symbol), i++) != null)
throw new SchemaParseException("Duplicate enum symbol: "+symbol);
+ if (enumDefault != null && !symbols.contains(enumDefault))
+ throw new SchemaParseException("The Enum Default: " + enumDefault + "
is not in the enum symbol set: " + symbols);
}
public List<String> getEnumSymbols() { return symbols; }
public boolean hasEnumSymbol(String symbol) {
@@ -771,6 +792,8 @@ public boolean equals(Object o) {
&& symbols.equals(that.symbols)
&& props.equals(that.props);
}
+ @Override
+ public String getEnumDefault() { return enumDefault; }
@Override int computeHash() { return super.computeHash() +
symbols.hashCode(); }
void toJson(Names names, JsonGenerator gen) throws IOException {
if (writeNameRef(names, gen)) return;
@@ -783,6 +806,8 @@ void toJson(Names names, JsonGenerator gen) throws
IOException {
for (String symbol : symbols)
gen.writeString(symbol);
gen.writeEndArray();
+ if (getEnumDefault() != null)
+ gen.writeStringField("default", getEnumDefault());
writeProps(gen);
aliasesToJson(gen);
gen.writeEndObject();
@@ -1309,7 +1334,11 @@ static Schema parse(JsonNode schema, Names names) {
LockableArrayList<String> symbols = new
LockableArrayList<>(symbolsNode.size());
for (JsonNode n : symbolsNode)
symbols.add(n.getTextValue());
- result = new EnumSchema(name, doc, symbols);
+ JsonNode enumDefault = schema.get("default");
+ String defaultSymbol = null;
+ if (enumDefault != null)
+ defaultSymbol = enumDefault.getTextValue();
+ result = new EnumSchema(name, doc, symbols, defaultSymbol);
if (name != null) names.add(result);
} else if (type.equals("array")) { // array
JsonNode itemsNode = schema.get("items");
@@ -1330,9 +1359,14 @@ static Schema parse(JsonNode schema, Names names) {
} else
throw new SchemaParseException("Type not supported: "+type);
Iterator<String> i = schema.getFieldNames();
+
+ Set reserved = SCHEMA_RESERVED;
+ if (type.equals("enum")) {
+ reserved = ENUM_RESERVED;
+ }
while (i.hasNext()) { // add properties
String prop = i.next();
- if (!SCHEMA_RESERVED.contains(prop)) // ignore reserved
+ if (!reserved.contains(prop)) // ignore reserved
result.addProp(prop, schema.get(prop));
}
// parse logical type if present
@@ -1456,7 +1490,7 @@ private static Schema applyAliases(Schema s,
Map<Schema,Schema> seen,
case ENUM:
if (aliases.containsKey(name))
result = Schema.createEnum(aliases.get(name).full, s.getDoc(), null,
- s.getEnumSymbols());
+ s.getEnumSymbols(), s.getEnumDefault());
break;
case ARRAY:
Schema e = applyAliases(s.getElementType(), seen, aliases, fieldAliases);
diff --git a/lang/java/avro/src/main/java/org/apache/avro/SchemaBuilder.java
b/lang/java/avro/src/main/java/org/apache/avro/SchemaBuilder.java
index 9c768f279..579ecd0b5 100644
--- a/lang/java/avro/src/main/java/org/apache/avro/SchemaBuilder.java
+++ b/lang/java/avro/src/main/java/org/apache/avro/SchemaBuilder.java
@@ -767,6 +767,7 @@ public R size(int size) {
private EnumBuilder(Completion<R> context, NameContext names, String name)
{
super(context, names, name);
}
+ private String enumDefault = null;
private static <R> EnumBuilder<R> create(Completion<R> context,
NameContext names, String name) {
@@ -778,14 +779,19 @@ private EnumBuilder(Completion<R> context, NameContext
names, String name) {
return this;
}
- /** Configure this enum type's symbols, and end its configuration. **/
+ /** Configure this enum type's symbols, and end its configuration.
Populates the default if it was set.**/
public R symbols(String... symbols) {
Schema schema = Schema.createEnum(name(), doc(), space(),
- Arrays.asList(symbols));
+ Arrays.asList(symbols), this.enumDefault);
completeSchema(schema);
return context().complete(schema);
}
+ /** Set the default value of the enum. */
+ public EnumBuilder<R> defaultSymbol(String enumDefault) {
+ this.enumDefault = enumDefault;
+ return self();
+ }
}
/**
@@ -1200,14 +1206,14 @@ public final R nullType() {
/** Build an Avro enum type. Example usage:
* <pre>
- * enumeration("Suits").namespace("org.cards").doc("card suit names")
+ * enumeration("Suits").namespace("org.cards").doc("card suit
names").defaultSymbol("HEART")
* .symbols("HEART", "SPADE", "DIAMOND", "CLUB")
* </pre>
* Equivalent to Avro JSON Schema:
* <pre>
* {"type":"enum", "name":"Suits", "namespace":"org.cards",
* "doc":"card suit names", "symbols":[
- * "HEART", "SPADE", "DIAMOND", "CLUB"]}
+ * "HEART", "SPADE", "DIAMOND", "CLUB"], "default":"HEART"}
* </pre>
**/
public final EnumBuilder<R> enumeration(String name) {
diff --git
a/lang/java/avro/src/main/java/org/apache/avro/SchemaCompatibility.java
b/lang/java/avro/src/main/java/org/apache/avro/SchemaCompatibility.java
index 5ec07375a..b1a499e73 100644
--- a/lang/java/avro/src/main/java/org/apache/avro/SchemaCompatibility.java
+++ b/lang/java/avro/src/main/java/org/apache/avro/SchemaCompatibility.java
@@ -418,10 +418,15 @@ private SchemaCompatibilityResult
checkReaderWriterRecordFields(final Schema rea
// Reader field does not correspond to any field in the writer
record schema, so the
// reader field must have a default value.
if (readerField.defaultValue() == null) {
- // reader field has no default value
- result = result.mergedWith(SchemaCompatibilityResult.incompatible(
+ // reader field has no default value. Check for the enum default
value
+ if (readerField.schema().getType() == Type.ENUM &&
readerField.schema().getEnumDefault() != null) {
+ result = result.mergedWith(getCompatibility("type",
readerField.schema(),
+ writerField.schema(), location));
+ } else {
+ result =
result.mergedWith(SchemaCompatibilityResult.incompatible(
SchemaIncompatibilityType.READER_FIELD_MISSING_DEFAULT_VALUE,
reader, writer,
readerField.name(), asList(location)));
+ }
}
} else {
result = result.mergedWith(getCompatibility("type",
readerField.schema(),
@@ -443,9 +448,14 @@ private SchemaCompatibilityResult
checkReaderEnumContainsAllWriterEnumSymbols(
final Set<String> symbols = new TreeSet<>(writer.getEnumSymbols());
symbols.removeAll(reader.getEnumSymbols());
if (!symbols.isEmpty()) {
- result = SchemaCompatibilityResult.incompatible(
+ if(reader.getEnumDefault() != null &&
reader.getEnumSymbols().contains(reader.getEnumDefault())) {
+ symbols.clear();
+ result = SchemaCompatibilityResult.compatible();
+ } else {
+ result = SchemaCompatibilityResult.incompatible(
SchemaIncompatibilityType.MISSING_ENUM_SYMBOLS, reader, writer,
symbols.toString(), asList(location));
+ }
}
// POP "symbols" literal
location.removeFirst();
diff --git
a/lang/java/avro/src/main/java/org/apache/avro/io/parsing/ResolvingGrammarGenerator.java
b/lang/java/avro/src/main/java/org/apache/avro/io/parsing/ResolvingGrammarGenerator.java
index 7c5da1169..51d9bc992 100644
---
a/lang/java/avro/src/main/java/org/apache/avro/io/parsing/ResolvingGrammarGenerator.java
+++
b/lang/java/avro/src/main/java/org/apache/avro/io/parsing/ResolvingGrammarGenerator.java
@@ -99,7 +99,7 @@ public Symbol generate(Schema writer, Schema reader,
if (writer.getFullName() == null
|| writer.getFullName().equals(reader.getFullName())) {
return Symbol.seq(mkEnumAdjust(writer.getEnumSymbols(),
- reader.getEnumSymbols()), Symbol.ENUM);
+ reader.getEnumSymbols(), reader.getEnumDefault()),
Symbol.ENUM);
}
break;
@@ -416,11 +416,15 @@ public static void encode(Encoder e, Schema s, JsonNode n)
}
}
- private static Symbol mkEnumAdjust(List<String> wsymbols,
- List<String> rsymbols){
+ private static Symbol mkEnumAdjust(List<String> wsymbols, List<String>
rsymbols, Object rEnumDefault){
Object[] adjustments = new Object[wsymbols.size()];
for (int i = 0; i < adjustments.length; i++) {
int j = rsymbols.indexOf(wsymbols.get(i));
+ if (j == -1) {
+ if (rEnumDefault instanceof String) {
+ j = rsymbols.indexOf(rEnumDefault);
+ }
+ }
adjustments[i] = (j == -1 ? "No match for " + wsymbols.get(i)
: new Integer(j));
}
diff --git
a/lang/java/avro/src/test/java/org/apache/avro/TestSchemaBuilder.java
b/lang/java/avro/src/test/java/org/apache/avro/TestSchemaBuilder.java
index e250ffef1..af15c388d 100644
--- a/lang/java/avro/src/test/java/org/apache/avro/TestSchemaBuilder.java
+++ b/lang/java/avro/src/test/java/org/apache/avro/TestSchemaBuilder.java
@@ -561,6 +561,17 @@ public void testEnum() {
Assert.assertEquals(expected, schema);
}
+ @Test
+ public void testEnumWithDefault() {
+ List<String> symbols = Arrays.asList("a", "b");
+ String enumDefault = "a";
+ Schema expected = Schema.createEnum("myenum", null, null, symbols,
enumDefault);
+ expected.addProp("p", "v");
+ Schema schema = SchemaBuilder.enumeration("myenum")
+ .prop("p", "v").defaultSymbol(enumDefault).symbols("a", "b");
+ Assert.assertEquals(expected, schema);
+ }
+
@Test
public void testFixed() {
Schema expected = Schema.createFixed("myfixed", null, null, 16);
diff --git
a/lang/java/avro/src/test/java/org/apache/avro/TestSchemaCompatibility.java
b/lang/java/avro/src/test/java/org/apache/avro/TestSchemaCompatibility.java
index d46bfd322..05726e989 100644
--- a/lang/java/avro/src/test/java/org/apache/avro/TestSchemaCompatibility.java
+++ b/lang/java/avro/src/test/java/org/apache/avro/TestSchemaCompatibility.java
@@ -35,6 +35,12 @@
import static org.apache.avro.TestSchemas.ENUM1_ABC_SCHEMA;
import static org.apache.avro.TestSchemas.ENUM1_AB_SCHEMA;
import static org.apache.avro.TestSchemas.ENUM1_BC_SCHEMA;
+import static
org.apache.avro.TestSchemas.ENUM_AB_FIELD_DEFAULT_A_ENUM_DEFAULT_B_RECORD;
+import static
org.apache.avro.TestSchemas.ENUM_ABC_FIELD_DEFAULT_B_ENUM_DEFAULT_A_RECORD;
+import static org.apache.avro.TestSchemas.ENUM_AB_ENUM_DEFAULT_A_RECORD;
+import static org.apache.avro.TestSchemas.ENUM_ABC_ENUM_DEFAULT_A_RECORD;
+import static org.apache.avro.TestSchemas.ENUM_AB_ENUM_DEFAULT_A_SCHEMA;
+import static org.apache.avro.TestSchemas.ENUM_ABC_ENUM_DEFAULT_A_SCHEMA;
import static org.apache.avro.TestSchemas.FIXED_4_BYTES;
import static org.apache.avro.TestSchemas.FLOAT_SCHEMA;
import static org.apache.avro.TestSchemas.FLOAT_UNION_SCHEMA;
@@ -74,9 +80,11 @@
import org.apache.avro.SchemaCompatibility.SchemaIncompatibilityType;
import org.apache.avro.SchemaCompatibility.SchemaPairCompatibility;
import org.apache.avro.TestSchemas.ReaderWriter;
+import org.apache.avro.generic.GenericData;
import org.apache.avro.generic.GenericData.EnumSymbol;
import org.apache.avro.generic.GenericDatumReader;
import org.apache.avro.generic.GenericDatumWriter;
+import org.apache.avro.generic.GenericRecord;
import org.apache.avro.io.DatumReader;
import org.apache.avro.io.DatumWriter;
import org.apache.avro.io.Decoder;
@@ -97,6 +105,9 @@
new Schema.Field("oldfield1", INT_SCHEMA, null, null),
new Schema.Field("oldfield2", STRING_SCHEMA, null, null)));
+
+
+
@Test
public void testValidateSchemaPairMissingField() throws Exception {
final List<Schema.Field> readerFields = list(
@@ -358,7 +369,10 @@ public void testUnionReaderWriterSubsetIncompatibility() {
new ReaderWriter(LONG_LIST_RECORD, LONG_LIST_RECORD),
new ReaderWriter(LONG_LIST_RECORD, INT_LIST_RECORD),
- new ReaderWriter(NULL_SCHEMA, NULL_SCHEMA)
+ new ReaderWriter(NULL_SCHEMA, NULL_SCHEMA),
+ new ReaderWriter(ENUM_AB_ENUM_DEFAULT_A_RECORD,
ENUM_ABC_ENUM_DEFAULT_A_RECORD),
+ new ReaderWriter(ENUM_AB_FIELD_DEFAULT_A_ENUM_DEFAULT_B_RECORD,
ENUM_ABC_FIELD_DEFAULT_B_ENUM_DEFAULT_A_RECORD)
+
);
//
-----------------------------------------------------------------------------------------------
@@ -502,6 +516,10 @@ public Object getDecodedDatum() {
ENUM1_ABC_SCHEMA, new EnumSymbol(ENUM1_ABC_SCHEMA, "B"),
ENUM1_BC_SCHEMA, new EnumSymbol(ENUM1_BC_SCHEMA, "B")),
+ new DecodingTestCase(
+ ENUM_ABC_ENUM_DEFAULT_A_SCHEMA, new
EnumSymbol(ENUM_ABC_ENUM_DEFAULT_A_SCHEMA, "C"),
+ ENUM_AB_ENUM_DEFAULT_A_SCHEMA, new
EnumSymbol(ENUM_AB_ENUM_DEFAULT_A_SCHEMA, "A")),
+
new DecodingTestCase(
INT_STRING_UNION_SCHEMA, "the string",
STRING_SCHEMA, new Utf8("the string")),
diff --git
a/lang/java/avro/src/test/java/org/apache/avro/TestSchemaCompatibilityEnumDefaults.java
b/lang/java/avro/src/test/java/org/apache/avro/TestSchemaCompatibilityEnumDefaults.java
new file mode 100644
index 000000000..159733ac7
--- /dev/null
+++
b/lang/java/avro/src/test/java/org/apache/avro/TestSchemaCompatibilityEnumDefaults.java
@@ -0,0 +1,146 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.avro;
+
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericDatumReader;
+import org.apache.avro.generic.GenericDatumWriter;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.avro.io.DatumReader;
+import org.apache.avro.io.DatumWriter;
+import org.apache.avro.io.Decoder;
+import org.apache.avro.io.DecoderFactory;
+import org.apache.avro.io.Encoder;
+import org.apache.avro.io.EncoderFactory;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.io.ByteArrayOutputStream;
+
+import static
org.apache.avro.TestSchemaCompatibility.validateIncompatibleSchemas;
+import static org.apache.avro.TestSchemas.*;
+import static org.junit.Assert.assertEquals;
+
+public class TestSchemaCompatibilityEnumDefaults {
+ @Rule
+ public ExpectedException expectedException = ExpectedException.none();
+
+ @Test
+ public void testEnumDefaultNotAppliedWhenWriterFieldMissing() throws
Exception {
+ expectedException.expect(AvroTypeException.class);
+ expectedException.expectMessage("Found Record1, expecting Record1, missing
required field field1");
+
+ Schema writerSchema = SchemaBuilder.record("Record1").fields()
+ .name("field2").type(ENUM2_AB_SCHEMA).noDefault()
+ .endRecord();
+
+ Schema readerSchema = SchemaBuilder.record("Record1").fields()
+ .name("field1").type(ENUM_AB_ENUM_DEFAULT_A_SCHEMA).noDefault()
+ .endRecord();
+
+ GenericRecord datum = new GenericData.Record(writerSchema);
+ datum.put("field2", new GenericData.EnumSymbol(writerSchema, "B"));
+ serializeWithWriterThenDeserializeWithReader(writerSchema, datum,
readerSchema);
+ }
+
+ @Test
+ public void testEnumDefaultAppliedWhenNoFieldDefaultDefined() throws
Exception {
+ Schema writerSchema = SchemaBuilder.record("Record1").fields()
+ .name("field1").type(ENUM_ABC_ENUM_DEFAULT_A_SCHEMA).noDefault()
+ .endRecord();
+
+ Schema readerSchema = SchemaBuilder.record("Record1").fields()
+ .name("field1").type(ENUM_AB_ENUM_DEFAULT_A_SCHEMA).noDefault()
+ .endRecord();
+
+ GenericRecord datum = new GenericData.Record(writerSchema);
+ datum.put("field1", new GenericData.EnumSymbol(writerSchema, "C"));
+ GenericRecord decodedDatum =
serializeWithWriterThenDeserializeWithReader(writerSchema, datum, readerSchema);
+ //The A is the Enum fallback value.
+ assertEquals("A", decodedDatum.get("field1").toString());
+ }
+
+ @Test
+ public void testEnumDefaultNotAppliedWhenCompatibleSymbolIsFound() throws
Exception {
+ Schema writerSchema = SchemaBuilder.record("Record1").fields()
+ .name("field1").type(ENUM_ABC_ENUM_DEFAULT_A_SCHEMA).noDefault()
+ .endRecord();
+
+ Schema readerSchema = SchemaBuilder.record("Record1").fields()
+ .name("field1").type(ENUM_AB_ENUM_DEFAULT_A_SCHEMA).noDefault()
+ .endRecord();
+
+ GenericRecord datum = new GenericData.Record(writerSchema);
+ datum.put("field1", new GenericData.EnumSymbol(writerSchema, "B"));
+ GenericRecord decodedDatum =
serializeWithWriterThenDeserializeWithReader(writerSchema, datum, readerSchema);
+ assertEquals("B", decodedDatum.get("field1").toString());
+ }
+
+ @Test
+ public void testEnumDefaultAppliedWhenFieldDefaultDefined() throws Exception
{
+ Schema writerSchema = SchemaBuilder.record("Record1").fields()
+ .name("field1").type(ENUM_ABC_ENUM_DEFAULT_A_SCHEMA).noDefault()
+ .endRecord();
+
+ Schema readerSchema = SchemaBuilder.record("Record1").fields()
+ .name("field1").type(ENUM_AB_ENUM_DEFAULT_A_SCHEMA).withDefault("B")
+ .endRecord();
+
+ GenericRecord datum = new GenericData.Record(writerSchema);
+ datum.put("field1", new GenericData.EnumSymbol(writerSchema, "C"));
+ GenericRecord decodedDatum =
serializeWithWriterThenDeserializeWithReader(writerSchema, datum, readerSchema);
+ //The A is the Enum default, which is assigned since C is not in [A,B].
+ assertEquals("A", decodedDatum.get("field1").toString());
+ }
+
+ @Test
+ public void testFieldDefaultNotAppliedForUnknownSymbol() throws Exception {
+ expectedException.expect(AvroTypeException.class);
+ expectedException.expectMessage("No match for C");
+
+ Schema writerSchema = SchemaBuilder.record("Record1").fields()
+ .name("field1").type(ENUM1_ABC_SCHEMA).noDefault()
+ .endRecord();
+ Schema readerSchema = SchemaBuilder.record("Record1").fields()
+ .name("field1").type(ENUM1_AB_SCHEMA).withDefault("A")
+ .endRecord();
+
+ GenericRecord datum = new GenericData.Record(writerSchema);
+ datum.put("field1", new GenericData.EnumSymbol(writerSchema, "C"));
+ serializeWithWriterThenDeserializeWithReader(writerSchema, datum,
readerSchema);
+ }
+
+ private GenericRecord serializeWithWriterThenDeserializeWithReader(Schema
writerSchema, GenericRecord datum, Schema readerSchema) throws Exception {
+ ByteArrayOutputStream baos = new ByteArrayOutputStream();
+ Encoder encoder = EncoderFactory.get().binaryEncoder(baos, null);
+ DatumWriter<Object> datumWriter = new GenericDatumWriter<>(writerSchema);
+ datumWriter.write(datum, encoder);
+ encoder.flush();
+
+ byte[] bytes = baos.toByteArray();
+ Decoder decoder = DecoderFactory.get().resolvingDecoder(
+ writerSchema, readerSchema,
+ DecoderFactory.get().binaryDecoder(bytes, null));
+ DatumReader<Object> datumReader = new GenericDatumReader<>(readerSchema);
+ return (GenericRecord)datumReader.read(null, decoder);
+ }
+
+}
diff --git a/lang/java/avro/src/test/java/org/apache/avro/TestSchemas.java
b/lang/java/avro/src/test/java/org/apache/avro/TestSchemas.java
index 39dd1a241..466f1fd63 100644
--- a/lang/java/avro/src/test/java/org/apache/avro/TestSchemas.java
+++ b/lang/java/avro/src/test/java/org/apache/avro/TestSchemas.java
@@ -19,7 +19,10 @@
import static org.junit.Assert.assertTrue;
import java.util.ArrayList;
+import java.util.Arrays;
import java.util.Collections;
+import java.util.List;
+
import org.apache.avro.Schema.Field;
/** Schemas used by other tests in this package. Therefore package protected.
*/
@@ -47,6 +50,22 @@
list("A", "B", "C"));
static final Schema ENUM1_BC_SCHEMA = Schema.createEnum("Enum1", null, null,
list("B", "C"));
static final Schema ENUM2_AB_SCHEMA = Schema.createEnum("Enum2", null, null,
list("A", "B"));
+ static final Schema ENUM_ABC_ENUM_DEFAULT_A_SCHEMA =
Schema.createEnum("Enum", null, null, list("A", "B", "C"), "A");
+ static final Schema ENUM_AB_ENUM_DEFAULT_A_SCHEMA =
Schema.createEnum("Enum", null, null, list("A", "B"), "A");
+ static final Schema ENUM_ABC_ENUM_DEFAULT_A_RECORD =
Schema.createRecord("Record", null, null, false);
+ static final Schema ENUM_AB_ENUM_DEFAULT_A_RECORD =
Schema.createRecord("Record", null, null, false);
+ static final Schema ENUM_ABC_FIELD_DEFAULT_B_ENUM_DEFAULT_A_RECORD =
Schema.createRecord("Record", null, null, false);
+ static final Schema ENUM_AB_FIELD_DEFAULT_A_ENUM_DEFAULT_B_RECORD =
Schema.createRecord("Record", null, null, false);
+ static {
+ ENUM_ABC_ENUM_DEFAULT_A_RECORD.setFields(
+ list(new Schema.Field("Field", Schema.createEnum("Schema", null, null,
list("A","B","C"), "A"), null, null)));
+ ENUM_AB_ENUM_DEFAULT_A_RECORD.setFields(
+ list(new Schema.Field("Field", Schema.createEnum("Schema", null, null,
list("A","B"), "A"), null, null)));
+ ENUM_ABC_FIELD_DEFAULT_B_ENUM_DEFAULT_A_RECORD.setFields(
+ list(new Schema.Field("Field", Schema.createEnum("Schema", null, null,
list("A","B","C"), "A"), null, "B")));
+ ENUM_AB_FIELD_DEFAULT_A_ENUM_DEFAULT_B_RECORD.setFields(
+ list(new Schema.Field("Field", Schema.createEnum("Schema", null, null,
list("A","B"), "B"), null, "A")));
+ }
static final Schema EMPTY_UNION_SCHEMA = Schema.createUnion(new
ArrayList<>());
static final Schema NULL_UNION_SCHEMA =
Schema.createUnion(list(NULL_SCHEMA));
diff --git
a/lang/java/compiler/src/main/java/org/apache/avro/compiler/idl/ResolvingVisitor.java
b/lang/java/compiler/src/main/java/org/apache/avro/compiler/idl/ResolvingVisitor.java
index 88b1d6ff0..9ccd7e7a5 100644
---
a/lang/java/compiler/src/main/java/org/apache/avro/compiler/idl/ResolvingVisitor.java
+++
b/lang/java/compiler/src/main/java/org/apache/avro/compiler/idl/ResolvingVisitor.java
@@ -73,7 +73,7 @@ public SchemaVisitorAction visitTerminal(final Schema
terminal) {
break;
case ENUM:
newSchema = Schema.createEnum(terminal.getName(), terminal.getDoc(),
- terminal.getNamespace(), terminal.getEnumSymbols());
+ terminal.getNamespace(), terminal.getEnumSymbols(),
terminal.getEnumDefault());
break;
case FIXED:
newSchema = Schema.createFixed(terminal.getName(), terminal.getDoc(),
diff --git
a/lang/java/compiler/src/main/javacc/org/apache/avro/compiler/idl/idl.jj
b/lang/java/compiler/src/main/javacc/org/apache/avro/compiler/idl/idl.jj
index c0ec7f5d0..cae241ffc 100644
--- a/lang/java/compiler/src/main/javacc/org/apache/avro/compiler/idl/idl.jj
+++ b/lang/java/compiler/src/main/javacc/org/apache/avro/compiler/idl/idl.jj
@@ -1099,13 +1099,16 @@ Schema EnumDeclaration():
{
String name;
List<String> symbols;
+ String defaultSymbol = null;
}
{
- "enum"
+ "enum" { String doc = getDoc(); }
name = Identifier()
symbols = EnumBody()
+ [ <EQUALS> defaultSymbol=Identifier() <SEMICOLON>]
{
- Schema s = Schema.createEnum(name, getDoc(), this.namespace, symbols);
+ Schema s = Schema.createEnum(name, doc, this.namespace, symbols,
+ defaultSymbol);
names.put(s.getFullName(), s);
return s;
}
diff --git a/lang/java/compiler/src/test/idl/input/simple.avdl
b/lang/java/compiler/src/test/idl/input/simple.avdl
index 0ad7c7e37..29b013c9e 100644
--- a/lang/java/compiler/src/test/idl/input/simple.avdl
+++ b/lang/java/compiler/src/test/idl/input/simple.avdl
@@ -30,6 +30,12 @@ protocol Simple {
BAZ
}
+ enum Status {
+ A,
+ B,
+ C
+ } = C;
+
/** An MD5 hash. */
fixed MD5(16);
@@ -41,6 +47,9 @@ protocol Simple {
/** The kind of record. */
Kind @order("descending") kind;
+ /** The status of the record. */
+ Status status = "A";
+
@foo("bar") MD5 hash = "0000000000000000";
union {null, MD5} @aliases(["hash", "hsh"]) nullableHash = null;
diff --git a/lang/java/compiler/src/test/idl/output/simple.avpr
b/lang/java/compiler/src/test/idl/output/simple.avpr
index ae155f53b..ddcb626ad 100644
--- a/lang/java/compiler/src/test/idl/output/simple.avpr
+++ b/lang/java/compiler/src/test/idl/output/simple.avpr
@@ -9,6 +9,11 @@
"doc" : "A kind of record.",
"symbols" : [ "FOO", "BAR", "BAZ" ],
"aliases" : [ "org.foo.KindOf" ]
+ }, {
+ "type" : "enum",
+ "name" : "Status",
+ "symbols" : [ "A", "B", "C" ],
+ "default" : "C"
}, {
"type" : "fixed",
"name" : "MD5",
@@ -29,6 +34,11 @@
"type" : "Kind",
"doc" : "The kind of record.",
"order" : "descending"
+ }, {
+ "name" : "status",
+ "type" : "Status",
+ "doc" : "The status of the record.",
+ "default" : "A"
}, {
"name" : "hash",
"type" : "MD5",
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]
> use default to allow old readers to specify default enum value when
> encountering new enum symbols
> -------------------------------------------------------------------------------------------------
>
> Key: AVRO-1340
> URL: https://issues.apache.org/jira/browse/AVRO-1340
> Project: Avro
> Issue Type: Improvement
> Components: spec
> Environment: N/A
> Reporter: Jim Donofrio
> Priority: Minor
>
> The schema resolution page says:
> > if both are enums:
> > if the writer's symbol is not present in the reader's enum, then an
> error is signalled.
> This makes it difficult to use enum's because you can never add a enum value
> and keep old reader's compatible. Why not use the default option to refer to
> one of enum values so that when a old reader encounters a enum ordinal it
> does not recognize, it can default to the optional schema provided one. If
> the old schema does not provide a default then the older reader can continue
> to fail as it does today.
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)