Re: Converting arbitrary JSON to avro
Hi Russel, thanks for pointing out the python lib. I created a little converter script that reads in json using json.loads and writes the resulting object to avro using a specific schema. (Or does the lib already contain such a converter and I just missed it?) Thanks for the help, -markus Am 19.09.2012 um 01:18 schrieb Russell Jurney: Fwiw, I do this in web apps all the time via the python avro lib and json.dumps Russell Jurney twitter.com/rjurney russell.jur...@gmail.com datasyndrome.com On Sep 18, 2012, at 12:38 PM, Doug Cutting cutt...@apache.org wrote: On Tue, Sep 18, 2012 at 11:34 AM, Markus Strickler mar...@braindump.ms wrote: Json.Writer is indeed what I had in mind and I have successfully managed to convert my existing JSON to avro using it. However using GenericDatumReader on this feels pretty unnatural, as I seem to be unable to access fields directly. It seems I have to access the value field on each record which returns a Map which uses Utf8 Objects as keys for the actual fields. Or am I doing something wrong here? Hmm. We could re-factor Json.SCHEMA so the union is the top-level element. That would get rid of the wrapper around every value. It's a more redundant way to write the schema, but the binary encoding is identical (since a record wrapper adds no bytes). It would hence require no changes to Json.Reader or Json.Writer. [ long, double, string, boolean, null, {type : array, items : { type : record, name : org.apache.avro.data.Json, fields : [ { name : value, type : [ long, double, string, boolean, null, {type : array, items : Json}, {type : map, values : Json} ] } ] } }, {type : map, values : Json} ] You can try this by placing this schema in share/schemas/org/apache/avro/data/Json.avsc and re-building the avro jar. Would such a change be useful to you? If so, please file an issue in Jira. Or we could even refactor this schema so that a Json object is the top-level structure: {type : map, values : [ long, double, string, boolean, null, {type : array, items : { type : record, name : org.apache.avro.data.Json, fields : [ { name : value, type : [ long, double, string, boolean, null, {type : array, items : Json}, {type : map, values : Json} ] } ] } }, {type : map, values : Json} ] } This would change the binary format but would not change the representation that GenericDatumReader would hand you from my first example above (since the generic representation unwraps unions). Using this schema would require changes to Json.Writer and Json.Reader. It would better conform to the definition of Json, which only permits objects as the top-level type. Concerning the more specific schema, you are of course completely right. Unfortunately more or less all the fields in the JSON data format are optional and many have substructures, so, at least in my understanding, I have to use unions of null and the actual type throughout the schema. I tried using JsonDecoder first (or rather the fromjson option of the avro tool, which, I think, uses JsonDecoder) but given the current JSON structures, this didn't work. So I'll probably have to look into implementing my own converter. However given the rather complex structure of the original JSON I'm wondering if trying to represent the data in avro is such a good idea in the first place. It would be interesting to see whether, with the appropriate schema, whether the dataset is smaller and faster to process as Avro than as Json. If you have 1000 fields in your data but the typical record only has one or two non-null, then an Avro record is perhaps not a good representation. An Avro map might be better, but if the values are similarly variable then Json might be competitive. Cheers, Doug
Re: Converting arbitrary JSON to avro
Hi Doug, thank you for your detailed explanation. Json.Writer is indeed what I had in mind and I have successfully managed to convert my existing JSON to avro using it. However using GenericDatumReader on this feels pretty unnatural, as I seem to be unable to access fields directly. It seems I have to access the value field on each record which returns a Map which uses Utf8 Objects as keys for the actual fields. Or am I doing something wrong here? Concerning the more specific schema, you are of course completely right. Unfortunately more or less all the fields in the JSON data format are optional and many have substructures, so, at least in my understanding, I have to use unions of null and the actual type throughout the schema. I tried using JsonDecoder first (or rather the fromjson option of the avro tool, which, I think, uses JsonDecoder) but given the current JSON structures, this didn't work. So I'll probably have to look into implementing my own converter. However given the rather complex structure of the original JSON I'm wondering if trying to represent the data in avro is such a good idea in the first place. Again, thanks a lot for your help, -markus Am 17.09.2012 um 19:35 schrieb Doug Cutting: On Mon, Sep 17, 2012 at 9:40 AM, Markus Strickler mar...@braindump.ms wrote: I'm currently trying to convert already existing JSON (not generated by avro) to avro and am wondering if there is some generic way to do this (maybe an avro schema that matches arbitrary JSON)? Yes, there is support for reading and writing arbitrary Json data as Avro: http://avro.apache.org/docs/current/api/java/org/apache/avro/data/Json.html Json.Writer will take Json data that's been parsed into Jackson's JsonNode representation and write it as Avro data using the schema Json.SCHEMA, and Json.Reader will read Avro data written with this Schema into a JsonNode. Note that just because you wrote the data with Json.Writer doesn't mean you need to read it with Json.Reader. You could instead read it with GenericDatumReader, from MapReduce or Hive. However using a more-specific schema than Json.SCHEMA will result in a smaller and faster Avro encoding for your data. It's also likely to result in a schema that much better describes your data for use in Pig, Hive, etc. If all of your records are of the same schema, and that schema doesn't have unions (i.e., a given field always has values of the same type, all objects have the same set of fields, fully populated) then you may be able to use Avro's JsonDecoder. Note however that Avro's JsonEncoder/JsonDecoder are not generally appropriate for arbitrary Json, but rather are intended to represent Avro data as Json. (Unions are the biggest difference. Avro's Json encoding uses a Json object to tag each union value with the intended type. For example, an Avro union of a string and an int which has an int value of 1 would be encoded in Json as {int:1}.) For a given schema it is simple to write a short Java program that converts from Json to Avro. A general tool for such conversions doesn't yet exist but would make a great addition to Avro (if anyone's looking for a way to contribute). The core of this might be a method that walks a JsonNode and a Schema in parallel, returning an object in Avro's generic representation. Doug
Re: Converting arbitrary JSON to avro
On Tue, Sep 18, 2012 at 11:34 AM, Markus Strickler mar...@braindump.ms wrote: Json.Writer is indeed what I had in mind and I have successfully managed to convert my existing JSON to avro using it. However using GenericDatumReader on this feels pretty unnatural, as I seem to be unable to access fields directly. It seems I have to access the value field on each record which returns a Map which uses Utf8 Objects as keys for the actual fields. Or am I doing something wrong here? Hmm. We could re-factor Json.SCHEMA so the union is the top-level element. That would get rid of the wrapper around every value. It's a more redundant way to write the schema, but the binary encoding is identical (since a record wrapper adds no bytes). It would hence require no changes to Json.Reader or Json.Writer. [ long, double, string, boolean, null, {type : array, items : { type : record, name : org.apache.avro.data.Json, fields : [ { name : value, type : [ long, double, string, boolean, null, {type : array, items : Json}, {type : map, values : Json} ] } ] } }, {type : map, values : Json} ] You can try this by placing this schema in share/schemas/org/apache/avro/data/Json.avsc and re-building the avro jar. Would such a change be useful to you? If so, please file an issue in Jira. Or we could even refactor this schema so that a Json object is the top-level structure: {type : map, values : [ long, double, string, boolean, null, {type : array, items : { type : record, name : org.apache.avro.data.Json, fields : [ { name : value, type : [ long, double, string, boolean, null, {type : array, items : Json}, {type : map, values : Json} ] } ] } }, {type : map, values : Json} ] } This would change the binary format but would not change the representation that GenericDatumReader would hand you from my first example above (since the generic representation unwraps unions). Using this schema would require changes to Json.Writer and Json.Reader. It would better conform to the definition of Json, which only permits objects as the top-level type. Concerning the more specific schema, you are of course completely right. Unfortunately more or less all the fields in the JSON data format are optional and many have substructures, so, at least in my understanding, I have to use unions of null and the actual type throughout the schema. I tried using JsonDecoder first (or rather the fromjson option of the avro tool, which, I think, uses JsonDecoder) but given the current JSON structures, this didn't work. So I'll probably have to look into implementing my own converter. However given the rather complex structure of the original JSON I'm wondering if trying to represent the data in avro is such a good idea in the first place. It would be interesting to see whether, with the appropriate schema, whether the dataset is smaller and faster to process as Avro than as Json. If you have 1000 fields in your data but the typical record only has one or two non-null, then an Avro record is perhaps not a good representation. An Avro map might be better, but if the values are similarly variable then Json might be competitive. Cheers, Doug
Re: Converting arbitrary JSON to avro
Fwiw, I do this in web apps all the time via the python avro lib and json.dumps Russell Jurney twitter.com/rjurney russell.jur...@gmail.com datasyndrome.com On Sep 18, 2012, at 12:38 PM, Doug Cutting cutt...@apache.org wrote: On Tue, Sep 18, 2012 at 11:34 AM, Markus Strickler mar...@braindump.ms wrote: Json.Writer is indeed what I had in mind and I have successfully managed to convert my existing JSON to avro using it. However using GenericDatumReader on this feels pretty unnatural, as I seem to be unable to access fields directly. It seems I have to access the value field on each record which returns a Map which uses Utf8 Objects as keys for the actual fields. Or am I doing something wrong here? Hmm. We could re-factor Json.SCHEMA so the union is the top-level element. That would get rid of the wrapper around every value. It's a more redundant way to write the schema, but the binary encoding is identical (since a record wrapper adds no bytes). It would hence require no changes to Json.Reader or Json.Writer. [ long, double, string, boolean, null, {type : array, items : { type : record, name : org.apache.avro.data.Json, fields : [ { name : value, type : [ long, double, string, boolean, null, {type : array, items : Json}, {type : map, values : Json} ] } ] } }, {type : map, values : Json} ] You can try this by placing this schema in share/schemas/org/apache/avro/data/Json.avsc and re-building the avro jar. Would such a change be useful to you? If so, please file an issue in Jira. Or we could even refactor this schema so that a Json object is the top-level structure: {type : map, values : [ long, double, string, boolean, null, {type : array, items : { type : record, name : org.apache.avro.data.Json, fields : [ { name : value, type : [ long, double, string, boolean, null, {type : array, items : Json}, {type : map, values : Json} ] } ] } }, {type : map, values : Json} ] } This would change the binary format but would not change the representation that GenericDatumReader would hand you from my first example above (since the generic representation unwraps unions). Using this schema would require changes to Json.Writer and Json.Reader. It would better conform to the definition of Json, which only permits objects as the top-level type. Concerning the more specific schema, you are of course completely right. Unfortunately more or less all the fields in the JSON data format are optional and many have substructures, so, at least in my understanding, I have to use unions of null and the actual type throughout the schema. I tried using JsonDecoder first (or rather the fromjson option of the avro tool, which, I think, uses JsonDecoder) but given the current JSON structures, this didn't work. So I'll probably have to look into implementing my own converter. However given the rather complex structure of the original JSON I'm wondering if trying to represent the data in avro is such a good idea in the first place. It would be interesting to see whether, with the appropriate schema, whether the dataset is smaller and faster to process as Avro than as Json. If you have 1000 fields in your data but the typical record only has one or two non-null, then an Avro record is perhaps not a good representation. An Avro map might be better, but if the values are similarly variable then Json might be competitive. Cheers, Doug
Converting arbitrary JSON to avro
Hi- I'm currently trying to convert already existing JSON (not generated by avro) to avro and am wondering if there is some generic way to do this (maybe an avro schema that matches arbitrary JSON)? Or are there any helpers that would allow me to map a parsed JSON onto an enxisting avro schema, given I could create one that semantically maches the JSON data I have? Sorry if this sounds a bit vague, but I'm pretty new to avro. Thanks for any help, -markus