Re: Formal spec for Avro Schema

2024-05-15 Thread Elliot West
I note that the enum type appears to be missing the specification of the
default attribute.

On Wed, 15 May 2024 at 08:26, Martin Grigorov  wrote:

> Hi Clemens,
>
> What is the difference between your document and the specification [1] ?
> I haven't read it completely but it looks very similar to the
> specification to me.
>
> 1. https://avro.apache.org/docs/1.11.1/specification/
> 2.
> https://github.com/apache/avro/tree/main/doc/content/en/docs/%2B%2Bversion%2B%2B/Specification
> - sources of the specification
>
> On Wed, May 15, 2024 at 9:28 AM Clemens Vasters
>  wrote:
>
>> I wrote a formal spec for the Avro Schema format.
>>
>>
>>
>> https://gist.github.com/clemensv/498c481965c425b218ee156b38b49333
>>
>>
>>
>> Where would that go in the repo?
>>
>>
>>
>>
>>
>>
>> 
>>
>> *Clemens Vasters*
>>
>> Messaging Platform Architect
>>
>> Microsoft Azure
>>
>> È+49 151 44063557
>>
>> *  cleme...@microsoft.com
>> European Microsoft Innovation Center GmbH | Gewürzmühlstrasse 11 | 80539
>> Munich| Germany
>> 
>> Geschäftsführer/General Managers: Keith Dolliver, Benjamin O. Orndorff
>> Amtsgericht Aachen, HRB 12066
>>
>>
>>
>>
>>
>


Unions: default values

2020-07-14 Thread Elliot West
Forwards compatibility permits us to update the writer schema independently
of the consumer. In particular, with record types we are free to add fields
with no breaking changes with respect to the consumer - a record is an
extensible type.

Avro includes two additional extensible types: enum and union.

Until recently it was not possible to extend an enum with forwards
compatibility - adding a new symbol to a writer's enum would result in a
breaking change at the reader as it would not recognise the symbol. Avro
1.9.0 added the concept of a default value for enums - a symbol that the
reader should use if it encounters a symbol it does not recognise, and thus
it became possible to evolve enums in a manner that was forwards compatible.

However, this kind of functionality does not exist for unions and it is not
yet possible to add types to an enum in a similarly forward compatible
manner. Would it not make sense for unions to also support a default value?
Has there been any previous efforts to explore the possibility?

Thanks,

Elliot.


Re: SchemaValidator vs SchemaCompatibility

2020-07-07 Thread Elliot West
The error in question is: READER_FIELD_MISSING_DEFAULT_VALUE,
location:/fields/0/type/fields/0

READER_FIELD_MISSING_DEFAULT_VALUE indicates that the reader requires a
default value on a field
The field can be identified with the JSON pointer: /fields/0/type/fields/0

Applying the pointer to the reader schema suggests that you need to specify
a default value for field worksheet.worksheet1

At first glance, this appears correct as field worksheet1 is not present in
the type Worksheet in the writer schema, and so the reader would need a
substitute default value when writing. However, I notice that you do have
an alias in the worksheet_items field of the writer schema, mapping to the
name worksheet1. This will not work, as I understand it, because aliases
are a property of the schema, not the data, and so the reader will be
unaware of the alias declared on the writer schema. I expect what you need
to do is instead declare an alias in the reader schema on
the worksheet.worksheet1 field:

"aliases": [
  "worksheet_items"
]

Thanks,

Elliot.

On Tue, 7 Jul 2020 at 05:36, KV 59  wrote:

> Which is the right class to use to check compatibility?
>
> I'm using Avro 1.9.2 and I'm trying to check the compatibility for the
> fo schemas using the SchemaCompatibility and I can't figure what the issue
> is and why it says incompatible
>
> Reader Schemas
> -
>
>>   {
>> "type": "record",
>> "name": "WorksheetCompleted",
>> "namespace": "com.school.avro",
>> "doc": "Emitted After an Student completed working on worksheet",
>> "fields": [
>>   {
>> "name": "worksheet",
>> "type": {
>>   "type": "record",
>>   "name": "Worksheet",
>>   "doc": "Completed worksheet",
>>   "fields": [
>> {
>>   "name": "worksheet1",
>>   "type": {
>> "type": "array",
>> "items": {
>>   "type": "record",
>>   "name": "WorksheetItem",
>>   "doc": "One worksheet question with an answer",
>>   "fields": [
>> {
>>   "name": "question_id",
>>   "type": "long",
>>   "doc": "Question id"
>> },
>> {
>>   "name": "answer",
>>   "type": [
>> "null",
>> "string"
>>   ],
>>   "doc": "Answer",
>>   "default": null
>> }
>>   ]
>> }
>>   },
>>   "doc": "Collection of worksheet questions with answers"
>> }
>>   ]
>> }
>>   }
>> ]
>>   }
>>
>
> Writer Schema
> --
>   {
> "type": "record",
> "name": "WorksheetCompleted",
> "namespace": "com.school.avro",
> "doc": "Emitted After an Student completed working on worksheet",
> "fields": [
>   {
> "name": "worksheet",
> "type": {
>   "type": "record",
>   "name": "Worksheet",
>   "doc": "Completed worksheet",
>   "fields": [
> {
>   "name": "worksheet_items",
>   "type": {
> "type": "array",
> "items": {
>   "type": "record",
>   "name": "WorksheetItem",
>   "doc": "One worksheet question with an answer",
>   "fields": [
> {
>   "name": "question_id",
>   "type": "long",
>   "doc": "Question id"
> },
> {
>   "name": "answer",
>   "type": [
> "null",
> "string"
>   ],
>   "doc": "Answer",
>   "default": null
> }
>   ]
> }
>   },
>   "doc": "Collection of worksheet questions with answers",
>   "aliases": [
> "worksheet1"
>   ]
> },
> {
>   "name": "student",
>   "type": [
> "null",
> "string"
>   ],
>   "doc": "an Student who completed the worksheet",
>   "default": null
> },
> {
>   "name": "duration",
>   "type": [
> "null",
> "long"
>   ],
>   "doc": "Worksheet duration in milliseconds",
>   "default": null
> }
>   ]
> }
>   }
> ]
>
>>   }
>
>
> I get an error
>
> INCOMPATIBLE
> 

Re: Avro Schema Question

2018-05-26 Thread Elliot West
A word of caution on the union type. You may find support for unions very
patchy if you are hoping to process records using well known data
processing engines. We’ve been unable to usefully read union types in both
Apache Spark and Hive for example. The simple null union construct is the
exception: [null, typeA], as it is usually represented by a nullable
columns of typeA. We’ve resorted to prohibiting schemas with complex unions
so that our producers can’t create data that is not fully readable by our
consumers.

Elliot.

On Fri, 25 May 2018 at 22:30, Motoko Kusanagi <
major-motoko-kusan...@outlook.com> wrote:

> Hi Michael,
>
> Thanks!! Yes, it does.
>
> Scott
> --
> *From:* Michael Smith 
> *Sent:* Friday, May 25, 2018 2:21 PM
> *To:* user@avro.apache.org
> *Subject:* Re: Avro Schema Question
>
> {"type": "int"}, {"type": "string"} is not valid json, so you definitely
> can't do that. But
>
> [{"type": "int"}, {"type": "string"}] is a valid schema -- it can encode a
> single value that is either an int or a string. At the highest level, your
> schema can only be one type, but that type may be (and in fact probably
> will be) a complex type -- a union of records or a single record.
>
> Does that answer your question?
>
> On Fri, May 25, 2018 at 5:08 PM Motoko Kusanagi <
> major-motoko-kusan...@outlook.com> wrote:
>
> Hi,
>
>
> I read the specification multiple times. In the specification, it says "A
> Schema is represented in JSON  by one of:" in the
> Schema Declaration section. The "one" confuses me as I am interpreting it
> as exactly one of the 3 that it listed.
>
>
> In short, can I do this as a single schema?
>
> {type : int},
>
> {type : string},
>
> {type : int},
>
>
> Or do the following as a single schema?
>
> {type : int},
>
> {type : record },
>
> {type : record }, // Not the same as the previous.
>
> {type : string},
>
>
> Or do I have to "embed" the above under a complex type like a record if I
> want complex schema? Or does "one of" mean I have to choose one and exactly
> one for the high top-most level of the schema?
>
>
> Thanks!!
>
>
>
> --
>
> Michael A. Smith — Senior Systems Engineer
> --
>
> micha...@syapse.com
> syapse.com
> 100 Matsonford Road
> 
> Five Radnor Corporate Center
> Suite 444
> Radnor, PA 19087
> https://www.linkedin.com/in/michaelalexandersmith
>
>


Re: How to specify datum for avro union type without ambiguity?

2018-04-25 Thread Elliot West
A quick question: If the datum is valid in more than one schema, what is
the scenario where knowing the specific schema is necessary? Is it that the
equivalent schemas might evolve in a divergent manner over time or perhaps
that by targeting a specific schema you are wanting to convey some out of
band information that may have some meaning to a consumer, if not Avro?

Elliot.

On 25 April 2018 at 12:27, Marcelo Valle  wrote:

> I am writing a python program using the official avro library for python,
> version 1.8.2.
>
> This is a simple schema to show my problem:
>
> {
>   "type": "record",
>   "namespace": "com.example",
>   "name": "NameUnion",
>   "fields": [
> {
>   "name": "name",
>   "type": [
> {
>   "type": "record",
>   "namespace": "com.example",
>   "name": "FullName",
>   "fields": [
> {
>   "name": "first",
>   "type": "string"
> },
> {
>   "name": "last",
>   "type": "string"
> }
>   ]
> },
> {
>   "type": "record",
>   "namespace": "com.example",
>   "name": "ConcatenatedFullName",
>   "fields": [
> {
>   "name": "entireName",
>   "type": "string"
> }
>   ]
> }
>   ]
> }
>   ]
> }
>
> Possible datums for this schema would be `{"name": {"first": "Hakuna",
> "last": "Matata"}}` and `{"name": {"entireName": "Hakuna Matata"}}`.
>
> However, this gives margin to ambiguity, as not always avro will be able
> to detect the right schema specified in the union. In this case, either
> datum will correspond to 1 and only 1 valid schema, but there might be a
> case where more than 1 schema in the union would be valid.
>
> I wonder whether it would be possible to use a datum like `{"name":
> {"FullName": {"first": "Hakuna", "last": "Matata"}}}`, where the specific
> union schema name is specified in the datum.
>
> Is it possible? How to do it?
>
> --
> Marcelo Valle
> http://mvalle.com - @mvallebr
>


Schema composition; Circular dependencies across files

2018-01-09 Thread Elliot West
Hi,

I understand that Avro supports circular type dependencies, and also schema
composition. However, I cannot seem to be able to create a circular type
reference that spans multiple schema payloads. Consider this example:

*a.avsc*
{
  "type": "record", "name": "a", "fields": [{
"name": "X", "type": "b"
  }]
}

*b.avsc*
{
  "type": "record", "name": "b", "fields": [{
"name": "Y", "type": "c"
  }]
}

*c.avsc*
{
  "type": "record", "name": "c", "fields": [{
"name": "Z", "type": "a"
  }]
}


In this case, there seems to be no good entry point that will result in a
valid initial schema, and hence while these types are declared in different
files, the total schema cannot be correctly parsed.

Can you confirm if this is an issue? Is there an outstanding JIRA, and if
not should I raise one. Can you suggest work arounds?

Thanks,

Elliot.


Re: Avro error "org.apache.avro.AvroTypeException: Expected field name not found"

2017-08-09 Thread Elliot West
Try:

{
  "TMS_ID" : "asdf"
}

On Wed, 9 Aug 2017 at 19:22, Manish Mehndiratta 
wrote:

> Hi Team,
>
> I stripped out my avro schema file and json file to only one element and
> yet it continues to give me the same error.
>
> Exception in thread "main" *org.apache.avro.AvroTypeException: Expected
> field name not found*: TMS_ID
>
> *--- avsc file ---*
> {
>   "type" : "record",
>   "name" : "DataModel",
>   "fields" : [
> { "name" : "TMS_ID", "type" : "string", "default" : "NONE" }
> ]
> }
>
> *--- json file ---*
> {
> "DataModel" :{
> "TMS_ID" : "asdf"
> }
> }
>
> *--- avro validation query ---*
> java -jar avro-tools-1.8.1.jar fromjson --schema-file
> TMSDataModelFlat.avsc 90573445988483.json > validated.avro
>
> *--- error ---*
> Exception in thread "main" org.apache.avro.AvroTypeException: Expected
> field name not found: TMS_ID
> at org.apache.avro.io.JsonDecoder.doAction(JsonDecoder.java:477)
> at org.apache.avro.io.parsing.Parser.advance(Parser.java:88)
> at org.apache.avro.io.JsonDecoder.advance(JsonDecoder.java:139)
> at org.apache.avro.io.JsonDecoder.readString(JsonDecoder.java:219)
> at org.apache.avro.io.JsonDecoder.readString(JsonDecoder.java:214)
> at org.apache.avro.io
> .ResolvingDecoder.readString(ResolvingDecoder.java:201)
> at
> org.apache.avro.generic.GenericDatumReader.readString(GenericDatumReader.java:430)
> at
> org.apache.avro.generic.GenericDatumReader.readString(GenericDatumReader.java:422)
> at
> org.apache.avro.generic.GenericDatumReader.readWithoutConversion(GenericDatumReader.java:180)
> at
> org.apache.avro.generic.GenericDatumReader.read(GenericDatumReader.java:152)
> at
> org.apache.avro.generic.GenericDatumReader.readField(GenericDatumReader.java:240)
> at
> org.apache.avro.generic.GenericDatumReader.readRecord(GenericDatumReader.java:230)
> at
> org.apache.avro.generic.GenericDatumReader.readWithoutConversion(GenericDatumReader.java:174)
> at
> org.apache.avro.generic.GenericDatumReader.read(GenericDatumReader.java:152)
> at
> org.apache.avro.generic.GenericDatumReader.read(GenericDatumReader.java:144)
> at
> org.apache.avro.tool.DataFileWriteTool.run(DataFileWriteTool.java:99)
> at org.apache.avro.tool.Main.run(Main.java:87)
> at org.apache.avro.tool.Main.main(Main.java:76)
>
>
>


Schema compatibility library

2017-07-25 Thread Elliot West
Hello,

We at the Hotels.com Data Platform team have been using the features
provided by AVRO-1933 and AVRO-2003 in production for a while now. Given
that these issues have not yet been merged or released, we rolled the
functionality into a library for our use. We also developed a fluent API to
simplify the process of implementing compatibility checks in our internal
schema tooling and schema registries.

We've recently contributed our 'avro-compatibily' library as open source
and it is now located in GitHub and on Maven Central under an Apache 2.0
license. This sits alongside our earlier, related contribution of Jasvorno;
a library for strong, schema based conversion between 'natural' JSON
documents and Avro.

https://github.com/HotelsDotCom/avro-compatibility
https://github.com/HotelsDotCom/jasvorno


Please note that we are still committed to supporting and working on the
relevant open issues in the Avro project, and look forward to a point in
time where the functionality is a core part of Avro and our artifacts are
no longer needed.

Special thanks to Anders Sundelin for his foundational work on AVRO-1933.

All feedback appreciated, many thanks for your time,

Elliot.


Re: Decode Avro map with java

2017-03-04 Thread Elliot West
I think what you're seeing is the value type only, which may contain null
and is thus represented as an Avro union of null and string. The key type
of an Avro may is assumed to be string according to the specification.

On Sat, 4 Mar 2017 at 13:37, Telco Phone  wrote:

> I am trying to get to the key AND value data type for a java Map.
>
> Given the orginal java converted to avro
>
> private java.util.Map testmap;
>
>
> I am using the following in Java to decode.
>
>// Print out the schema
>System.err.println("Map class type " +f.schema().toString());
>
>// Print out datatype for key / value
>for (Schema st : f.schema().getValueType().getTypes())
>System.err.println("ST is " + st.getName());
>
> With a result of:
>
> Map class type
> {"type":"map","values":["null",{"type":"string","avro.java.string":"String"}],"avro.java.string":"String","default":null}
> ST is null
> ST is string
>
>
> Should I not get both the orginal  ?? (From the above
> testmap)
>
>
> Thanks in advance
>


Re: Implementation of compatibility rules

2017-02-22 Thread Elliot West
Update:

I had a go at modifying org.apache.avro.SchemaValidatorBuilder to use
SchemaCompatibility and have then run schema compatibility test suites
from both the Avro project and Confluent's Schema registry. Every case that
is tested appears to continue to function correctly with one exception;
SchemaCompatibility appears to favourably consider aliases when performing
name based compatibility checks whereas the implementation provided via
SchemaValidatorBuilder is more strict, and does not.

The specification <http://avro.apache.org/docs/1.8.1/spec.html#Aliases>
makes no definitive judgement on the matter, simply stating that 'an
implementation may optionally use aliases'. Should perhaps this be
configurable in the aforementioned implementations so that the user can
decide and also have a chance of obtaining consistent behaviour?

Elliot.

On 22 February 2017 at 13:48, Elliot West <tea...@gmail.com> wrote:

> Further to this, is there any reason why conceptually, the implementation
> of org.apache.avro.ValidateMutualRead.canRead(Schema, Schema) could not
> be changed from:
>
>   static void canRead(Schema writtenWith, Schema readUsing)
>   throws SchemaValidationException {
> boolean error;
> try {
>   error = Symbol.hasErrors(new ResolvingGrammarGenerator().generate(
>   writtenWith, readUsing));
> } catch (IOException e) {
>   throw new SchemaValidationException(readUsing, writtenWith, e);
> }
> if (error) {
>   throw new SchemaValidationException(readUsing, writtenWith);
> }
>   }
>
>
> to:
>
>   static void canRead(Schema writtenWith, Schema readUsing)
>   throws SchemaValidationException {
> SchemaCompatibilityType compatibilityType
>   = SchemaCompatibility.checkReaderWriterCompatibility(readUsing,
> writtenWith).getType();
> if (compatibilityType != SchemaCompatibilityType.COMPATIBLE) {
>   throw new SchemaValidationException(readUsing, writtenWith);
> }
>   }
>
>
> Or am I missing something fundamental?
>
> Thanks,
>
> Elliot.
>
> On 17 February 2017 at 12:27, Elliot West <tea...@gmail.com> wrote:
>
>> Hi,
>>
>> I've been attempting to understand the implementation of Avro schema
>> compatibility rules and am slightly confused by the structure of the code.
>> It seems that there are at least two possible entry points:
>>
>>- 
>> org.apache.avro.SchemaCompatibility.checkReaderWriterCompatibility(Schema,
>>Schema)
>>- org.apache.avro.SchemaValidatorBuilder
>>
>> The code paths of these do not seem to intersect, with one implementing a
>> static set of rule checks and the other seemingly delegating to grammar
>> based approach. Does this imply that there are in fact two implementations
>> of the compatibility rules?
>>
>> Apologies if this is a naïve question.
>>
>> Thanks,
>>
>> Elliot.
>>
>
>


Re: Implementation of compatibility rules

2017-02-22 Thread Elliot West
Further to this, is there any reason why conceptually, the implementation
of org.apache.avro.ValidateMutualRead.canRead(Schema, Schema) could not be
changed from:

  static void canRead(Schema writtenWith, Schema readUsing)
  throws SchemaValidationException {
boolean error;
try {
  error = Symbol.hasErrors(new ResolvingGrammarGenerator().generate(
  writtenWith, readUsing));
} catch (IOException e) {
  throw new SchemaValidationException(readUsing, writtenWith, e);
}
if (error) {
  throw new SchemaValidationException(readUsing, writtenWith);
}
  }


to:

  static void canRead(Schema writtenWith, Schema readUsing)
  throws SchemaValidationException {
SchemaCompatibilityType compatibilityType
  = SchemaCompatibility.checkReaderWriterCompatibility(readUsing,
writtenWith).getType();
if (compatibilityType != SchemaCompatibilityType.COMPATIBLE) {
  throw new SchemaValidationException(readUsing, writtenWith);
}
  }


Or am I missing something fundamental?

Thanks,

Elliot.

On 17 February 2017 at 12:27, Elliot West <tea...@gmail.com> wrote:

> Hi,
>
> I've been attempting to understand the implementation of Avro schema
> compatibility rules and am slightly confused by the structure of the code.
> It seems that there are at least two possible entry points:
>
>- 
> org.apache.avro.SchemaCompatibility.checkReaderWriterCompatibility(Schema,
>Schema)
>- org.apache.avro.SchemaValidatorBuilder
>
> The code paths of these do not seem to intersect, with one implementing a
> static set of rule checks and the other seemingly delegating to grammar
> based approach. Does this imply that there are in fact two implementations
> of the compatibility rules?
>
> Apologies if this is a naïve question.
>
> Thanks,
>
> Elliot.
>


Implementation of compatibility rules

2017-02-17 Thread Elliot West
Hi,

I've been attempting to understand the implementation of Avro schema
compatibility rules and am slightly confused by the structure of the code.
It seems that there are at least two possible entry points:

   - org.apache.avro.SchemaCompatibility.checkReaderWriterCompatibility(Schema,
   Schema)
   - org.apache.avro.SchemaValidatorBuilder

The code paths of these do not seem to intersect, with one implementing a
static set of rule checks and the other seemingly delegating to grammar
based approach. Does this imply that there are in fact two implementations
of the compatibility rules?

Apologies if this is a naïve question.

Thanks,

Elliot.


Contribution: schema driven conversion of 'natural' JSON documents to and from Avro

2017-01-24 Thread Elliot West
Hello,

Today we (Hotels.com) open sourced a library for performing strong, schema
based conversions between arbitrarily structured 'natural' JSON documents
and Avro. 'Jasvorno' allows the conversion from any JSON document to Avro
and back, while ensuring conformity to a user defined schema. It removes
the requirement that documents adhere to Avro's specific JSON structure
(containing union index hints).

More information on the project can be found here:

https://github.com/HotelsDotCom/jasvorno


Thanks, Elliot.

Senior Developer
Big Data Platform team
Hotels.com


Re: Evolving schemas and namespaces

2016-12-02 Thread Elliot West
Hi Anders,

If the one schema is a compatible evolution of the other, what is the need
for multiple Java types? Schema compatibility implies that data written by
one schema can be safety marshalled to/from a Java class generated another
compatible version no?. Obviously the specific behaviours are dictated by
the selected compatibility level, the version of Avro data being
encoded/decoded, and the version of the schema from which that class was
generated. If the compatibility rules are followed I would not expect to
need multiple Java representations.

That said, I haven't tested my expectations concretely :-)

Elliot.

On 2 December 2016 at 15:43, Anders Sundelin <anders.sunde...@ericsson.com>
wrote:

> Hi Niels and Elliot,
>
> Thinking from the Java perspective now, the nice thing about namespaces
> (and, to a lesser degree, the name itself) is that they are mapped to
> packages (and classnames).
>
> If the v1 Avro spec was used as a base for generating java classes, then
> the corresponding Java class would then be "com.example.some.v1.MyType",
>
> and, correspondingly, the v2 Avro schema would generate into
> "com.example.some.v2.MyType"
>
> In other words, clients (or servers) could happily use both classes while
> talking to different other peers (supporting, for instance, protocol
> negotiation).
>
> This is the most important reason for why I think the rule you mention is
> kind of strange.
>
> What do you others think?
>
> BR
>
> /Anders
>
>
>
> On 2016-12-02 16:33, Elliot West wrote:
>
> There is perhaps a little ambiguity in the spec:
>
> From https://avro.apache.org/docs/current/spec.html#names
>   Record, enums and fixed are named types. Each has a fullname that is
> composed of two parts; a name and a namespace.* Equality of names is
> defined on the fullname*.
>
> From https://avro.apache.org/docs/current/spec.html#Schema+Resolution:
>   It is an error if the two schemas do not match.
>   To match, one of the following must hold:
>   ...
>   *both schemas are records with the same name*
>   ...
>
> I suspect that in this case 'name' means 'fullname' and therefore by
> choosing a different namespace you've declared to Avro that they should be
> considered different types.
>
> If you are trying to annotate different schemas with a version identifier,
> perhaps a 'doc' property might be more appropriate?
>
> On 2 December 2016 at 15:11, Niels Basjes <ni...@basjes.nl> wrote:
>
>> Hi,
>>
>> When I run the code below the output indicates that these two are 
>> incompatible in terms of schema evolution.
>>
>> The ONLY difference is the namespace (v1 and v2).
>>
>> If I remove the namespace line the are reported as 'compatible'.
>>
>> My question is why these two are considered to be incompatible?
>>
>> @Testpublic void evolveTest() throws IOException {
>>   Schema schemaV1 = new Schema.Parser().parse("{\n" +
>> "  \"type\" : \"record\",\n" +
>> "  \"name\" : \"Foo\",\n" +
>> "  \"namespace\" : \"nl.example.evoleschema.v1\",\n" +
>> "  \"fields\" : [ {\n" +
>> "\"name\" : \"count\",\n" +
>> "\"type\" : {\n" +
>> "  \"type\" : \"enum\",\n" +
>> "  \"name\" : \"Bar\",\n" +
>> "  \"symbols\" : [ \"ONE\", \"TWO\", \"THREE\" ]\n" +
>> "}\n" +
>> "  } ]\n" +
>> "}");
>>
>>   Schema schemaV2 = new Schema.Parser().parse("{\n" +
>> "  \"type\" : \"record\",\n" +
>> "  \"name\" : \"Foo\",\n" +
>> "  \"namespace\" : \"nl.example.evoleschema.v2\",\n" +
>> "  \"fields\" : [ {\n" +
>> "\"name\" : \"count\",\n" +
>> "\"type\" : {\n" +
>> "  \"type\" : \"enum\",\n" +
>> "  \"name\" : \"Bar\",\n" +
>> "  \"symbols\" : [ \"ONE\", \"TWO\", \"THREE\" ]\n" +
>> "}\n" +
>> "  } ]\n" +
>> "}");
>>
>>   LOG.info("{}", 
>> SchemaCompatibility.checkReaderWriterCompatibility(schemaV1, 
>> schemaV2).getType());
>>   LOG.info("{}", 
>> SchemaCompatibility.checkReaderWriterCompatibility(schemaV2, 
>> schemaV1).getType());
>> }
>>
>> --
>> Best regards / Met vriendelijke groeten, Niels Basjes
>>
>


Re: Evolving schemas and namespaces

2016-12-02 Thread Elliot West
There is perhaps a little ambiguity in the spec:

>From https://avro.apache.org/docs/current/spec.html#names
  Record, enums and fixed are named types. Each has a fullname that is
composed of two parts; a name and a namespace.* Equality of names is
defined on the fullname*.

>From https://avro.apache.org/docs/current/spec.html#Schema+Resolution:
  It is an error if the two schemas do not match.
  To match, one of the following must hold:
  ...
  *both schemas are records with the same name*
  ...

I suspect that in this case 'name' means 'fullname' and therefore by
choosing a different namespace you've declared to Avro that they should be
considered different types.

If you are trying to annotate different schemas with a version identifier,
perhaps a 'doc' property might be more appropriate?

On 2 December 2016 at 15:11, Niels Basjes  wrote:

> Hi,
>
> When I run the code below the output indicates that these two are 
> incompatible in terms of schema evolution.
>
> The ONLY difference is the namespace (v1 and v2).
>
> If I remove the namespace line the are reported as 'compatible'.
>
> My question is why these two are considered to be incompatible?
>
>
> @Test
> public void evolveTest() throws IOException {
>   Schema schemaV1 = new Schema.Parser().parse("{\n" +
> "  \"type\" : \"record\",\n" +
> "  \"name\" : \"Foo\",\n" +
> "  \"namespace\" : \"nl.example.evoleschema.v1\",\n" +
> "  \"fields\" : [ {\n" +
> "\"name\" : \"count\",\n" +
> "\"type\" : {\n" +
> "  \"type\" : \"enum\",\n" +
> "  \"name\" : \"Bar\",\n" +
> "  \"symbols\" : [ \"ONE\", \"TWO\", \"THREE\" ]\n" +
> "}\n" +
> "  } ]\n" +
> "}");
>
>   Schema schemaV2 = new Schema.Parser().parse("{\n" +
> "  \"type\" : \"record\",\n" +
> "  \"name\" : \"Foo\",\n" +
> "  \"namespace\" : \"nl.example.evoleschema.v2\",\n" +
> "  \"fields\" : [ {\n" +
> "\"name\" : \"count\",\n" +
> "\"type\" : {\n" +
> "  \"type\" : \"enum\",\n" +
> "  \"name\" : \"Bar\",\n" +
> "  \"symbols\" : [ \"ONE\", \"TWO\", \"THREE\" ]\n" +
> "}\n" +
> "  } ]\n" +
> "}");
>
>   LOG.info("{}", SchemaCompatibility.checkReaderWriterCompatibility(schemaV1, 
> schemaV2).getType());
>   LOG.info("{}", SchemaCompatibility.checkReaderWriterCompatibility(schemaV2, 
> schemaV1).getType());
> }
>
> --
> Best regards / Met vriendelijke groeten,
>
> Niels Basjes
>