[ https://issues.apache.org/jira/browse/AVRO-251?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12790283#action_12790283 ]
Doug Cutting commented on AVRO-251: ----------------------------------- Thiru> it won't match the standard JSON version of the schema [ ... ] Yes, but I don't expect folks to use this for anything but binary (which I should better document). This schema representation is really only useful when schemas are written out frequently, and should then be written more efficiently than they can be with JSON. It's not aesthetically ideal, but still useful, and I fear forcing the JSON encodings to correspond would be impractical for reasons you indicate. Thiru> Since the "order" field of the record is optional, it should be a union of null and Enum in the schema. It's not optional. It has a default value in the Java API, but it's always specified for every field. If it were such a union then its binary encoding would be larger. Philip> If we want to preserve the entire nuance of the writing schema, we should keep the defaults. Note that this currently does not preserve every nuance, e.g., user properties. So my vote is to remove default values as well. A container that stores lots of binary schemas would be wise to include one copy of the JSON schema-for-schemas, so that, if that meta-schema changes, the data can still be processed. This representation should contain only what's required to decode data, and its documentation should note that. It should only contain things that are required of "actual" schemas, and applications can interpret with an "expected" schema that contains default values, user properties, etc. Philip> I suspect this was a bug before (not popping back the previous space). I didn't see a test for it. Good point. Yes, it is a pre-existing bug triggered here by the inclusion of the Json schema from a different namespace in the Schema schema, so this patch does add a test, but not an explicit one. I've filed AVRO-255 for this. (This schema also identified AVRO-256.) Philip> Do you think it would be useful to have a hook at the schema type to have a way to evolve it? Should it be a union? It is in fact a union, wrapped in a record to give it a name. But, no, I think the best way to evolve it is for containers to keep a copy of the meta schema, as with any other and build on Avro's evolution mechanisms. Worst case, if an application neglects to store a copy and this schema changes in a subsequent release, the application can always go back to the prior release and retrieve it and use it as the "actual" schema while using the new release's version as "expected". Philip> I think you should call this "object" to follow the JSON terminology. It's not a JSON object but a "key value pair" within an object. So perhaps we should call it KeyValuePair? Note though that if we drop default values we don't need this schema here anyway. This code could however still be useful for a command-line tool that takes arbitrary Json input and encodes it in an Avro data file, and then for MapReduce programs that process Json-format generic data. It's nice to see how little code is required to incorporate full JSON data into Avro. Philip> From a maintainability perspective, it scares me that Type.values() has to be kept in sync with the order of the types in the union in Schema.m4 There are two things that protect us here: - We have unit tests that attempt to exhaustively test this encoding/decoding code. So it's in that sense its as safe as our DatumReader/DatumWriter implementations. But if this were our only protection, it would not be a programming style we'd recommend for applications. Except, - We have ValidatingEncoder and ValidatingDecoder. Folks who use this programming style who do not develop exhaustive test suites should be strongly advised to only use it with ValidatingEncoder and ValidatingDecoder. When I initially wrote this code there were a few errors (I forgot to read/write null values) that were caught by the validator. This "event based" programming style requires only a bit more coding than wrapper classes, but saves a level of redirection and/or copies. It also works well for applications whose serialized data does not correspond directly to in-memory structures (e.g., a streaming application with array values too large to fit in memory). The validators make it practical. Philip> It seems that instead of string, it could be a record (that contains a string) called "reference" That's a good idea. Using the event-style, the code need not actually change, nor would the serialized representation, only the schema. > add schema for schemas > ---------------------- > > Key: AVRO-251 > URL: https://issues.apache.org/jira/browse/AVRO-251 > Project: Avro > Issue Type: New Feature > Components: java > Reporter: Doug Cutting > Assignee: Doug Cutting > Fix For: 1.3.0 > > Attachments: AVRO-251.patch, AVRO-251.patch > > > A schema for schemas would permits schemas to be written in binary. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.