Re: AVRO Best Practices for Sparse object storage

2020-06-26 Thread Doug Cutting
A map schema might be appropriate.  Another idea might be to define a
record for every field, then use an array whose values are a union of all
these records.  This is a bit more complicated but would probably use the
least space.

Doug

On Thu, Jun 25, 2020 at 4:14 PM Sohail Khan  wrote:

> Hello Team,
>
> I am trying to serialize data in AVRO format and store it in Database.
> This would bring down the disk requirement of the table. Currently we are
> storing it in JSON format.
> I Have a very huge POJO with string type fields (Approximately 100), but
> for a given POJO hardly 20 or 30 have values, rest are null. I call it a
> sparse object. I am currently achieving approximately 20 percent
> improvement. Any suggestions, How to take it further, what are the best
> practices w.r.t to handling null values
>
> Thanks and Regards
> Sohail Khan
>


Re: schema resolution vs logical types

2020-04-09 Thread Doug Cutting
On Wed, Apr 8, 2020 at 5:03 AM roger peppe  wrote:

> On Tue, 7 Apr 2020 at 17:57, Doug Cutting  wrote:
>
>> On Tue, Apr 7, 2020 at 4:03 AM roger peppe  wrote:
>>
>>> On the one hand the specification says
>>> <https://avro.apache.org/docs/1.9.1/spec.html#Parsing+Canonical+Form+for+Schemas>
>>> :
>>>
>>> If the Parsing Canonical Forms of two different schemas are textually
>>>> equal, then those schemas are "the same" as far as any reader is concerned
>>>
>>>
>> This statement in the specification could perhaps be improved.  What it
>> means is that low-level parsing errors will not be encountered when using
>> two such schemas.  It does not mean they're equivalent for all purposes.
>>
>
> OK, that could definitely be improved then!
>

If you can suggest improved wording for this, please feel free to make a
pull request.


> For the purposes of schema resolution, two schemas that are decimal logical
>>>> types *match* if their scales and precisions match.
>>>
>>>
>>>
>> Schema resolution involves a different kind of equivalence for schemas.
>> Two compatible schemas here may have quite different binary formats, fields
>> might be reordered, removed, or added.  Scalar types may be promoted, etc.
>>
>
> Perhaps it might be worth mentioning in the schema resolution section that
> other fields can play a role here.
> That section mentions field reordering, scalar type promotion, etc, but
> doesn't talk about logical types at all. It reads like it's supposed to be
> definitive.
>

This is historic.  Logical types were added as a new, incremental feature.
Initially no languages implemented them.  Now a couple do, but some still
do not.  So they are described as a separate, optional feature.  Similarly,
implementation of aliases and schema resolution are optional.  Such is the
reality of adding features to something implemented in multiple programming
languages via volunteer efforts.

Perhaps this optionality and layering of features could be more explicit in
the specification.


> The logical type attributes are definitely not irrelevant to readers
> trying to parse incoming data. We could wrangle "low level" vs "high level"
> parsing errors but in the end, knowing the scale of a number is critically
> important to the reader of the data. If messages are encoded along with a
> schema (e.g. using Single Object Encoding) that strips out important
> attributes like that, then we have no decent assurance that our data won't
> be corrupted when we read it.
>
> We're using a schema registry to store schemas for our messages in Kafka,
> and we've got full compatibility enabled, but it ignores logical types for
> the purposes of schema comparison. I suspect that it's comparing
> canonicalised schemas. This means that if someone accidentally changes a
> decimal scale and pushes a new schema, we're at risk of not being able to
> read messages (if the reader takes logical types into account) or silent
> data corruption (if it doesn't).
>

This compatibility checking was was implemented before logical types were
added to Avro and has not yet been updated to support logical types like
decimal.  So your decimal data is treated simply as bytes.  These are not
corrupted, FWIW.


> There is a proposal to add an alternate canonical form that incorporates
>> logical types:
>>
>> https://github.com/apache/avro/pull/805
>> https://issues.apache.org/jira/browse/AVRO-2299
>>
>> Does this look like what you'd like?  It seems that patch has been
>> ignored, but perhaps we can pick it up again and get it committed.
>>
>
> Something along those kinds of lines seems like it would be useful, yes.
> I'm not entirely convinced about the exact content of that particular PR,
> however:
>
>- the naming could be considered confusing. Why "Standard" Canonical
>Form? The PR doesn't make it very clear exactly why this kind of canonical
>form is required, which might inform the naming better. Maybe "Resolving
>Canonical Form" might work better.
>
>
This adds a collection of features into the new canonicalized form,
including aliases, defaults, and logical types.  I don't see an obvious,
precise, name for that set of features.  It's more than just resolution, I
think.  I'd welcome a better suggestion than Standard, though!

>
>- it doesn't seem to suggest an ordering for custom properties. I
>wonder if it might be better just to say that all properties should be in
>lexical order.
>
> I think the order for custom properties is meant to be
application-specified in the current patch.  I agree that lexicograp

Re: schema resolution vs logical types

2020-04-07 Thread Doug Cutting
On Tue, Apr 7, 2020 at 4:03 AM roger peppe  wrote:

> On the one hand the specification says
> 
> :
>
> If the Parsing Canonical Forms of two different schemas are textually
>> equal, then those schemas are "the same" as far as any reader is concerned
>
>
This statement in the specification could perhaps be improved.  What it
means is that low-level parsing errors will not be encountered when using
two such schemas.  It does not mean they're equivalent for all purposes.


> but on the other, when discussing the decimal logical type, it says:
>
> For the purposes of schema resolution, two schemas that are decimal logical
>> types *match* if their scales and precisions match.
>
>
>
Schema resolution involves a different kind of equivalence for schemas.
Two compatible schemas here may have quite different binary formats, fields
might be reordered, removed, or added.  Scalar types may be promoted, etc.


> Given that the spec recommends using the canonical form for schema
> fingerprints, ISTM there might be some possibility for attack (or at least
> data corruption) there - if we unwittingly read a decimal value that was
> written with a different scale, we could read numbers thinking they're a
> different order of magnitude than they actually are.
>

Identical Parsing Canonical form only tells you whether you can parse the
data, not whether you can resolve it.  Indeed, if you use a different
logical type definition but only check parsing-level compatibility then you
can get incorrect data.

There is a proposal to add an alternate canonical form that incorporates
logical types:

https://github.com/apache/avro/pull/805
https://issues.apache.org/jira/browse/AVRO-2299

Does this look like what you'd like?  It seems that patch has been ignored,
but perhaps we can pick it up again and get it committed.

Thanks,

Doug

Does this


Re: Is there a way to skip field decoding without materializing the data?

2020-04-03 Thread Doug Cutting
Yes, as Roger suggests, in Avro this is done by specifying a reader schema
that contains a subset of the fields written.

E.g.,  https://icircuit.net/avro-schema-projection/1446

Doug


On Fri, Apr 3, 2020 at 8:09 AM roger peppe  wrote:

> If you're using a custom codec, this is potentially possible if you're
> reading only a single record, not a sequence of records. I've been
> considering implementing this as an optimisation. However, I don't think
> it's possible to skip entirely when reading a sequence of records because
> there's no record header that encodes the record length.
>
> However if you read into a schema that only has F1, none of the other
> fields will be materialised, only read and discarded, AIUI (although I'm
> not familiar with the Java codec so it might work differently).
>
>   cheers, rog.
>
>
> On Tue, 31 Mar 2020, 15:59 JohnGa,  wrote:
>
>> Hello,
>> I am using BinaryDecoder with a compiled schema.  Let's say there are 10
>> fields: F1, F2,F10.  After reading F1, I want to skip to the end of
>> the
>> record without decoding the fields from F2 to F10.  Is there a way to
>> accomplish this in Avro?
>>
>> Thanks.
>>
>>
>>
>> --
>> Sent from:
>> http://apache-avro.679487.n3.nabble.com/Avro-Users-f679479.html
>>
>


Re: defaults for complex types (was Re: recursive types)

2019-12-06 Thread Doug Cutting
On Fri, Dec 6, 2019 at 2:38 AM Ryan Skraba  wrote:

> Naively, I would expect any JSON encoded data to be a valid default
> value (which is not what the spec says).  Does anyone know why the
> "first schema only" rule was added to the spec?
>

I think we felt this would make things simpler.  That specifying a
mechanism for resolving the ambiguities in the JSON representations of ints
and longs, floats and doubles, strings and bytes, records and maps, etc.
would make implementation and comprehension more difficult.

In retrospect, it might have been better to use the type-tagged format
specified of the "JSON Encoding" section for default values.  This may be a
historical artifact.  If default values were added to Avro before the JSON
encoding, then the concept of the type tagging would not have been in the
spec when default values were defined.  However changing the format of
default values after they were defined would create a breaking
incompatibility.  That said, I don't recall anyone ever suggesting this
improvement before.

Doug


Re: Should a Schema be serializable in Java?

2019-07-15 Thread Doug Cutting
I can't think of a reason Schema should not implement Serializable.

There's actually already an issue & patch for this:

https://issues.apache.org/jira/browse/AVRO-1852

Doug

On Mon, Jul 15, 2019 at 6:49 AM Ismaël Mejía  wrote:

> +d...@avro.apache.org
>
> On Mon, Jul 15, 2019 at 3:30 PM Ryan Skraba  wrote:
> >
> > Hello!
> >
> > I'm looking for any discussion or reference why the Schema object isn't
> serializable -- I'm pretty sure this must have already been discussed (but
> the keywords +avro +serializable +schema have MANY results in all the
> searches I did: JIRA, stack overflow, mailing list, web)
> >
> > In particular, I was at a demo today where we were asked why Schemas
> needed to be passed as strings to run in distributed tasks.  I remember
> running into this problem years ago with MapReduce, and again in Spark, and
> again in Beam...
> >
> > Is there any downside to making a Schema implement
> java.lang.Serializable?  The only thing I can think of is that the schema
> _should not_ be serialized with the data, and making it non-serializable
> loosely enforces this (at the cost of continually writing different
> flavours of "Avro holders" for when you really do want to serialize it).
> >
> > Willing to create a JIRA and work on the implementation, of course!
> >
> > All my best, Ryan
>


Re: Alias with Backward Compatibility

2018-10-22 Thread Doug Cutting
I think the rule is that if you want bi-directional compatibility, give
every field a default value.

Doug

On Mon, Oct 22, 2018 at 2:13 PM Jesse Anderson 
wrote:

> In the schema resolution of the Avro spec, it doesn't really talk about
> the effect of aliases on schema resolution. So, aliases should be avoided
> for forward compatibility, but are fine for backward compatibility? If you
> want full compatibility, you should think long and hard about your field
> names.
>
> On Mon, Oct 22, 2018 at 2:05 PM Doug Cutting  wrote:
>
>> Despite the change in field name, they have the same binary format, so
>> you can read one and write the other in either order.  If you're actually
>> adding or removing fields or re-ordering fields then it probably would not
>> work.
>>
>> Doug
>>
>> On Mon, Oct 22, 2018 at 1:57 PM Jesse Anderson 
>> wrote:
>>
>>> I was trying to get V2 writer with V1 reader.
>>>
>>> This V2 schema will not work because of suit alias:
>>> {
>>>  "namespace": "model",
>>>  "type": "record",
>>>  "name": "SimpleCardV2",
>>>  "aliases": ["SimpleCard"],
>>>  "fields": [
>>>  {
>>>"name": "cardsuit",
>>>"type": "string",
>>>"doc" : "The suit of the card",
>>> *   "aliases": ["suit"]*
>>>  }
>>>  ]
>>> }
>>>
>>> This V2 schema will work because suit is no longer aliased:
>>> {
>>>  "namespace": "model",
>>>  "type": "record",
>>>  "name": "SimpleCardV2",
>>>  "aliases": ["SimpleCard"],
>>>  "fields": [
>>>  {
>>>"name": "suit",
>>>"type": "string",
>>>"doc" : "The suit of the card"
>>>  }
>>>  ]
>>> }
>>>
>>> Even with suit no longer aliased, the schema change doesn't pass the
>>> compatibility checks, but does pass serializing and deserializing them back
>>> into SpecificRecord objects.
>>>
>>> Thanks,
>>>
>>> Jesse
>>>
>>> On Mon, Oct 22, 2018 at 1:46 PM Doug Cutting  wrote:
>>>
>>>> Aliases in the reader's schema are used to adapt the writer's schema.
>>>> Aliases in the writer's schema are not considered.  So your V2 can read V1
>>>> data, but not vice versa.  Is that what you're asking?
>>>>
>>>> Thanks,
>>>>
>>>> Doug
>>>>
>>>> On Mon, Oct 22, 2018 at 1:01 PM Jesse Anderson <
>>>> je...@bigdatainstitute.io> wrote:
>>>>
>>>>> The email subject should have been "Alias with Forward Compatibility."
>>>>> Are using aliases forward incompatible? I'm using Avro 1.8.2.
>>>>>
>>>>> Using Hotels.com's Avro compatibility checker, I get:
>>>>> com.hotels.avro.compatibility.SchemaCompatibilityException:
>>>>> Compatibility type 'CAN_BE_READ_BY' does not hold between 1 schema(s) in
>>>>> the chronology because: Schema[0] has incompatibilities: ['NAME_MISMATCH:
>>>>> expected: model.SimpleCardV2' at
>>>>> '/name','READER_FIELD_MISSING_DEFAULT_VALUE: suit' at '/fields/0'].
>>>>>
>>>>> Thanks,
>>>>>
>>>>> Jesse
>>>>>
>>>>> On Fri, Oct 19, 2018 at 4:17 PM Jesse Anderson <
>>>>> je...@bigdatainstitute.io> wrote:
>>>>>
>>>>>> Is adding an alias a backwards incompatible change? I have two
>>>>>> schemas. I'm going from V2 to V1 and getting an error that the field is
>>>>>> missing. Am I missing something again?
>>>>>>
>>>>>> Here is V1:
>>>>>> {
>>>>>>  "namespace": "model",
>>>>>>  "type": "record",
>>>>>>  "name": "SimpleCard",
>>>>>>  "fields": [
>>>>>>  {
>>>>>>"name": "suit",
>>>>>>"type": "string",
>>>>>>"doc" : "The suit of the card"
>>>>>>  }
>>>>>>  ]
>>>>>> }
>>>>>>
>>>>>> Here is V2:
>>>>>> {
>>>>>>  "namespace": "model",
>>>>>>  "type": "record",
>>>>>>  "name": "SimpleCardV2",
>>>>>>  "aliases": ["SimpleCard"],
>>>>>>  "fields": [
>>>>>>  {
>>>>>>"name": "cardsuit",
>>>>>>"type": "string",
>>>>>>"doc" : "The suit of the card",
>>>>>>"aliases": ["suit"]
>>>>>>  }
>>>>>>  ]
>>>>>> }
>>>>>>
>>>>>> Here is the stack trace:
>>>>>> org.apache.avro.AvroTypeException: Found model.SimpleCardV2,
>>>>>> expecting model.SimpleCard, missing required field suit
>>>>>> at
>>>>>> org.apache.avro.io.ResolvingDecoder.doAction(ResolvingDecoder.java:292)
>>>>>> at org.apache.avro.io.parsing.Parser.advance(Parser.java:88)
>>>>>> at
>>>>>> org.apache.avro.io.ResolvingDecoder.readFieldOrder(ResolvingDecoder.java:130)
>>>>>> at
>>>>>> org.apache.avro.generic.GenericDatumReader.readRecord(GenericDatumReader.java:215)
>>>>>> at
>>>>>> org.apache.avro.generic.GenericDatumReader.readWithoutConversion(GenericDatumReader.java:175)
>>>>>> at
>>>>>> org.apache.avro.generic.GenericDatumReader.read(GenericDatumReader.java:153)
>>>>>> at
>>>>>> org.apache.avro.generic.GenericDatumReader.read(GenericDatumReader.java:145)
>>>>>>
>>>>>> Thanks,
>>>>>>
>>>>>> Jesse
>>>>>>
>>>>>


Re: Alias Issue

2018-10-15 Thread Doug Cutting
You can easily test two versions of a record schema with the same name if
you use GenericData.  You need different names or packages if you're
generating Java code (SpecificData) or using reflection (ReflectData).

One exception is if you want to have both versions in a single file or
schema.  In this case, different names within a union are an appropriate
solution.

Thanks,

Doug

On Fri, Oct 12, 2018 at 4:20 PM Jesse Anderson 
wrote:

> Doug,
>
> Thanks. Those were the two changes.
>
> A related question to your point about leaving them alone. What is the
> best practice around having two versions around for testing purposes?
> Should you have different namespaces or different record names?
>
> Thanks,
>
> Jesse
>
> On Fri, Oct 12, 2018 at 3:16 PM Doug Cutting  wrote:
>
>> Jesse,
>>
>> The record names should match (although Java has been loose about
>> enforcement of that).
>>
>> Also, that should be "aliases", not "alias".
>>
>> What happens if you add:
>>   "aliases": ["SimpleCard"]
>> to the second schema, and change the field alias to:
>>  "aliases": ["suit"]
>> ?
>>
>> Alternately, you could just leave the name of the record alone, as
>> "SimpleCard" in both cases.
>>
>> Does that help?
>>
>> Thanks,
>>
>> Doug
>>
>>
>>
>>
>> On Fri, Oct 12, 2018 at 11:16 AM Jesse Anderson <
>> je...@bigdatainstitute.io> wrote:
>>
>>> I'm hitting an issue with using an alias. When I do the schema
>>> evolution, I get an exception with "missing required field." As far I know,
>>> this should be a viable schema. Is there something I'm missing?
>>>
>>> Here is the version 1 of the schema:
>>>
>>> {
>>>  "namespace": "model",
>>>  "type": "record",
>>>  "name": "SimpleCard",
>>>  "fields": [
>>>  {
>>>"name": "suit",
>>>"type": "string",
>>>"doc" : "The suit of the card"
>>>  }
>>>  ]
>>> }
>>>
>>> Here is version 2 of the schema. The changes are a different record name
>>> and I've aliased the suit/cardsuit field.
>>>
>>> {
>>>  "namespace": "model",
>>>  "type": "record",
>>>  "name": "SimpleCardV2",
>>>  "fields": [
>>>  {
>>>"name": "cardsuit",
>>>"type": "string",
>>>"doc" : "The suit of the card",
>>>"alias": ["suit"]
>>>  }
>>>  ]
>>> }
>>>
>>> Here is the stack trace of the error:
>>>
>>> org.apache.avro.AvroTypeException: Found model.SimpleCard, expecting
>>> model.SimpleCardV2, missing required field cardsuit
>>> at
>>> org.apache.avro.io.ResolvingDecoder.doAction(ResolvingDecoder.java:292)
>>> at org.apache.avro.io.parsing.Parser.advance(Parser.java:88)
>>> at
>>> org.apache.avro.io.ResolvingDecoder.readFieldOrder(ResolvingDecoder.java:130)
>>> at
>>> org.apache.avro.generic.GenericDatumReader.readRecord(GenericDatumReader.java:215)
>>> at
>>> org.apache.avro.generic.GenericDatumReader.readWithoutConversion(GenericDatumReader.java:175)
>>> at
>>> org.apache.avro.generic.GenericDatumReader.read(GenericDatumReader.java:153)
>>> at
>>> org.apache.avro.generic.GenericDatumReader.read(GenericDatumReader.java:145)
>>> at helpers.AvroReaderHelper.getObject(AvroReaderHelper.java:56)
>>>
>>> Thanks,
>>>
>>> Jesse
>>>
>>


Re: Alias Issue

2018-10-12 Thread Doug Cutting
Jesse,

The record names should match (although Java has been loose about
enforcement of that).

Also, that should be "aliases", not "alias".

What happens if you add:
  "aliases": ["SimpleCard"]
to the second schema, and change the field alias to:
 "aliases": ["suit"]
?

Alternately, you could just leave the name of the record alone, as
"SimpleCard" in both cases.

Does that help?

Thanks,

Doug




On Fri, Oct 12, 2018 at 11:16 AM Jesse Anderson 
wrote:

> I'm hitting an issue with using an alias. When I do the schema evolution,
> I get an exception with "missing required field." As far I know, this
> should be a viable schema. Is there something I'm missing?
>
> Here is the version 1 of the schema:
>
> {
>  "namespace": "model",
>  "type": "record",
>  "name": "SimpleCard",
>  "fields": [
>  {
>"name": "suit",
>"type": "string",
>"doc" : "The suit of the card"
>  }
>  ]
> }
>
> Here is version 2 of the schema. The changes are a different record name
> and I've aliased the suit/cardsuit field.
>
> {
>  "namespace": "model",
>  "type": "record",
>  "name": "SimpleCardV2",
>  "fields": [
>  {
>"name": "cardsuit",
>"type": "string",
>"doc" : "The suit of the card",
>"alias": ["suit"]
>  }
>  ]
> }
>
> Here is the stack trace of the error:
>
> org.apache.avro.AvroTypeException: Found model.SimpleCard, expecting
> model.SimpleCardV2, missing required field cardsuit
> at org.apache.avro.io.ResolvingDecoder.doAction(ResolvingDecoder.java:292)
> at org.apache.avro.io.parsing.Parser.advance(Parser.java:88)
> at
> org.apache.avro.io.ResolvingDecoder.readFieldOrder(ResolvingDecoder.java:130)
> at
> org.apache.avro.generic.GenericDatumReader.readRecord(GenericDatumReader.java:215)
> at
> org.apache.avro.generic.GenericDatumReader.readWithoutConversion(GenericDatumReader.java:175)
> at
> org.apache.avro.generic.GenericDatumReader.read(GenericDatumReader.java:153)
> at
> org.apache.avro.generic.GenericDatumReader.read(GenericDatumReader.java:145)
> at helpers.AvroReaderHelper.getObject(AvroReaderHelper.java:56)
>
> Thanks,
>
> Jesse
>


Re: What is the default value I can specify for bytes when it is in union?

2018-06-06 Thread Doug Cutting
The default for a union is interpreted as the type of its first branch.

https://avro.apache.org/docs/current/spec.html#Unions

Thus, in your example, if you want a nullable byte array, place "null"
first in the union, e.g.:

{"type":"record","name":"hello","fields":[{"name":"id","type":["null",
"bytes"],"default":null}]}

Doug

On Mon, Jun 4, 2018 at 2:02 AM kant kodali  wrote:

> Hi All,
>
> What is the default value I can specify for bytes when it is in union?
>
> Tried the following and all fail.
>
> {"type":"record","name":"hello","fields":[{"name":"id","type":["bytes",
> "null"],"default":null}]}
> {"type":"record","name":"hello","fields":[{"name":"id","type":["bytes",
> "null"],"default":"null"}]}
> {"type":"record","name":"hello","fields":[{"name":"id","type":["bytes",
> "null"],"default":"\u00FF"}]}
>
> Thanks!
>
>


Re: Read schema from avro file without reading entire file

2018-05-03 Thread Doug Cutting
You might instead try using the blob's reader method?

Something like:

InputStream input = Channels.newInputStream(blob.reader());
try {
  return new DataFileStream(input, new GenericDatumReader()).getSchema();
} finally {
  input.close();
}

Doug

On Wed, May 2, 2018 at 4:30 PM Rodrigo Ipince  wrote:

> Hi,
>
>
> (Disclaimer: I'm new to Avro and Beam)
>
>
> Question: *is there a way to read the schema from an Avro file in GCS
> without having to read the entire file?*
>
>
> Context:
>
> I have a bunch of large files in GCS
>
> I want to process them by doing
> AvroIO.readGenericRecords(theSchema).from(filePattern) (this is from the
> Apache Beam SDK). However, I don’t know the schema up front.
>
>
> Now, I can read one of the files and extract the schema from it up front,
> sort of like this:
>
> ```
> Blob avroFile = … // get Blob from GCS
>
> SeekableInput seekableInput = new
> SeekableByteArrayInput(avroFile.getContent());
>
> DatumReader datumReader = new GenericDatumReader<>();
>
> try (DataFileReader dataFileReader = new
> DataFileReader<>(seekableInput, datumReader)) {
>
>   String schema = dataFileReader.getSchema().toString();
>
> }
>
> ```
>
>
> but.. the file is really large, and my nodes are really tiny, so they run
> out of memory. Is there a way to not have to read the entire file in order
> to extract the schema?
>
> Thanks!
>
>


Re: JSON format change while converting from Avro Pojo

2018-04-25 Thread Doug Cutting
To preserve type information, Avro's json encoding tags union values with
their type.

https://avro.apache.org/docs/current/spec.html#json_encoding

If you wish to avoid this tagging, then you may use toString() on Avro
data.  This will generate valid Json, although some type information may be
lost in the case of unions.

For example, a union of a string and an enum cannot always be losslessly
converted to and from Json without tagging.  Nor can unions of ints and
longs or unions of records and maps.  Avro's type model is more complex
than Json's.  However if your schemas do not contain any unions that are
ambiguous in Json then you can safely use toString().

Doug

On Wed, Apr 25, 2018 at 6:35 AM, amrishsintu  wrote:

> Hi,
>
> I am trying to convert a POJO which is set with data to JSON in java.
> While converting I get data as
> "lastName": {
> "string": "XYZ"
> }
> This happens when we use the following snippet of the schema.
> "name": "lastName",
> "type": ["string", "null"],
>
> But it works fine when we use below schema
> "name": "lastName",
> "type": ["string", "null"],
> And produces result as :- lastName":  "XYZ"
>
> But we need to make the schema such that it should work properly even if
> lastName comes as null.
>
> Please help us so that we can resolve the union issue of [“string”:”null”]
>
> Thanks,
> Amrish
>
>
>
>
> --
> Sent from: http://apache-avro.679487.n3.nabble.com/Avro-Users-f679479.html
>


Re: GenericData.deepCopy() HotSpot

2018-02-05 Thread Doug Cutting
Your code builds a new builder and instance each time through the loop:

  for (int i=0;i<100;i++) {
  user = User.newBuilder().build();
  ...

How does it perform if you move that second line outside the loop?

Thanks,

Doug


On Fri, Feb 2, 2018 at 3:50 PM, Nishanth S <nishanth.2...@gmail.com> wrote:

> Thanks Doug .  Here  is a  comparison .
>
> Load Avro  Record Size : Roughly15 Kb
>
> I have used the same payload  with a schema  that has  around 2k fields
> and  also  withanother schema   that has  5 fileds . I re used the
> avro object in both cases   using a builder once . Test was run for 1 M
> records writing the  same amount of data  (1 Gb ) to  a  local drive . Ran
> this few times as  single threaded . Average TPS in case of smaller schema
> is  40 K where  as with a bigger schema it drops down to 10 K  even though
> both are  writing the same amount of data. Since I am   only creating the
> avro object once  in both  cases   it looks   like  there is an overhead in
> the  datafilewriter too in case of bigger schemas .
>
>
>
> public static void main(String[] args){
> try{
> new LoadGenerator().load();
> }catch(IOException e){
> e.printStackTrace();
> }
> }
>
> DataFileWriter dataFileWriter;
> DatumWriter datumWriter;
> FileSystem hdfsFileSystem;
> Configuration conf;
> Path path;
> OutputStream outStream;
> User user;
> com.google.common.base.Stopwatch stopwatch = new
> com.google.common.base.Stopwatch().start();
> public  void load() throws IOException{
> conf = new Configuration();
> hdfsFileSystem = FileSystem.get(conf);
> datumWriter = new SpecificDatumWriter(User.class);
> dataFileWriter = new DataFileWriter(datumWriter);
> dataFileWriter.setCodec(CodecFactory.snappyCodec());
> path = new Path("/projects/tmp/load.avro");
> outStream=hdfsFileSystem.create(path, true);
> dataFileWriter.create(User.getClassSchema(), outStream);
> dataFileWriter.setFlushOnEveryBlock(false);
> // Create and Load User
> int numRecords =100;
> for (int i=0;i<100;i++){
> user = User.newBuilder().build();
> user.setFirstName("testName"+new Random().nextLong());
> user.setFavoriteNumber(Integer.valueOf(new Random().nextInt()));
> user.setFavoriteColor("blue" +new Random().nextFloat());
> user.setData(ByteBuffer.wrap(new byte[15000]));
> dataFileWriter.append(user);
> }
> dataFileWriter.close();
> stopwatch.stop();
> long elapsedTime = stopwatch.elapsedTime(TimeUnit.SECONDS);
> System.out.println("Time elapsed for myCall() is "+ elapsedTime);
>
> On Mon, Jan 29, 2018 at 11:01 AM, Doug Cutting <cutt...@gmail.com> wrote:
>
>> Builders have some inherent overheads.  Things could be optimized to
>> better minimize this, but it will likely always be faster to reuse a single
>> instance when writing.
>>
>> The deepCopy's are probably of the default values of each field you're
>> not setting.  If you're only setting a few fields then you might use a
>> builder to create a single instance so its defaults are set, then reuse
>> that instance as you write, setting only those few fields you need to
>> differ from the default.  (This only works if you're setting the same
>> fields every time.  Otherwise you'd need to restore the default value.)
>>
>> An optimization for Avro here might be to inline default values for
>> immutable types when generating the build() method.
>>
>> Doug
>>
>> On Fri, Jan 26, 2018 at 9:04 AM, Nishanth S <nishanth.2...@gmail.com>
>> wrote:
>>
>>> Hello Every One,
>>>
>>> We have a process that reads data from a  local file share  ,serailizes
>>> and writes to HDFS in avro format. .I am just wondering if I am building
>>> the avro objects correctly. For every record that  is read from the binary
>>> file we create an equivalent avro object in the below format.
>>>
>>> Parent p = new Parent();
>>> LOGHDR hdr = LOGHDR.newBuilder().build()
>>> MSGHDR msg = MSGHDR.newBuilder().build()
>>> p.setHdr(hdr);
>>> p.setMsg(msg);
>>> p..
>>> p..set
>>> datumFileWriter.write(p);
>>>
>>> This avro schema has  around 1800 fileds including 26 nested types
>>> within it .I did some load testing and figured that if I serialize the same
>>> object to disk  performance is  6 x times faster  than a constructing a new
>>> object (p.build). When a new  avro object is constructed everytime using
>>> RecordBuilder.build()  much of the time is spend in
>>> GenericData.deepCopy().Has any one run into a similar problem ? We are
>>> using Avro 1.8.2.
>>>
>>> Thanks,
>>> Nishanth
>>>
>>>
>>>
>>>
>>>
>>
>


Re: GenericData.deepCopy() HotSpot

2018-01-29 Thread Doug Cutting
Builders have some inherent overheads.  Things could be optimized to better
minimize this, but it will likely always be faster to reuse a single
instance when writing.

The deepCopy's are probably of the default values of each field you're not
setting.  If you're only setting a few fields then you might use a builder
to create a single instance so its defaults are set, then reuse that
instance as you write, setting only those few fields you need to differ
from the default.  (This only works if you're setting the same fields every
time.  Otherwise you'd need to restore the default value.)

An optimization for Avro here might be to inline default values for
immutable types when generating the build() method.

Doug

On Fri, Jan 26, 2018 at 9:04 AM, Nishanth S  wrote:

> Hello Every One,
>
> We have a process that reads data from a  local file share  ,serailizes
> and writes to HDFS in avro format. .I am just wondering if I am building
> the avro objects correctly. For every record that  is read from the binary
> file we create an equivalent avro object in the below format.
>
> Parent p = new Parent();
> LOGHDR hdr = LOGHDR.newBuilder().build()
> MSGHDR msg = MSGHDR.newBuilder().build()
> p.setHdr(hdr);
> p.setMsg(msg);
> p..
> p..set
> datumFileWriter.write(p);
>
> This avro schema has  around 1800 fileds including 26 nested types within
> it .I did some load testing and figured that if I serialize the same object
> to disk  performance is  6 x times faster  than a constructing a new object
> (p.build). When a new  avro object is constructed everytime using
> RecordBuilder.build()  much of the time is spend in
> GenericData.deepCopy().Has any one run into a similar problem ? We are
> using Avro 1.8.2.
>
> Thanks,
> Nishanth
>
>
>
>
>


Re: Concurrent Building of Avro Objects

2018-01-23 Thread Doug Cutting
This sounds like AVRO-1760, fixed since Avro 1.8.0.

https://issues.apache.org/jira/browse/AVRO-1760

What version of Avro are you using?

Doug

On Mon, Jan 22, 2018 at 9:45 AM, Nishanth S  wrote:

> Hi All,
>
> We have a process that reads data from a  local file share  ,serailizes
> and writes to HDFS in avro format. Currently it is running as a single
> threaded process. When converted  to a parallel process we did  get  some
> performance improvement  but  not the desired .Thread dumps are pasted
> below .I am just wondering if I am building the avro objects correctly.
> For every record that  that is read from the binary file we create an
> equivalent avro object in the below format. Our avro schema is  pretty
> big,  around 1800 fields and all of those have default values . After doing
> some profiling  I  could see that the most  time consuming method
> is  org.apache.avro.generic.GenericData.getDefaultValue() . This is in
> fact taking  more time than doing the actual reads/writes. Thanks for
> taking a look.
>
> Parent p = new Parent();
> LOGHDR hdr = LOGHDR.newBuilder().build()
> MSGHDR msg = MSGHDR.newBuilder().build()
> p.setHdr(hdr);
> p.setMsg(msg);
>
> Then  all fields in p and all the nested types that p holds together like
> LOGHDR and MSGHDR are set  .
>
>
>
>
> "pool-6-thread-5" #53 prio=5 os_prio=0 tid=0x7fad896c7800 nid=0x4328
> waiting for monitor entry [0x7fad52833000]
>   java.lang.Thread.State: BLOCKED (on object monitor)
>at java.util.Collections$Synchron
> izedMap.get(Collections.java:2584)
>- waiting to lock <0x00066a5e3460> (a
> java.util.Collections$SynchronizedMap)
>at org.apache.avro.generic.Generi
> cData.getDefaultValue(GenericData.java:981)
>at org.apache.avro.data.RecordBui
> lderBase.defaultValue(RecordBuilderBase.java:135)
>
>
> "pool-6-thread-4" #52 prio=5 os_prio=0 tid=0x7fad896c6000 nid=0x4327
> waiting for monitor entry [0x7fad52934000]
>   java.lang.Thread.State: BLOCKED (on object monitor)
>at java.util.Collections$Synchron
> izedMap.get(Collections.java:2584)
>- waiting to lock <0x00066a5e3460> (a
> java.util.Collections$SynchronizedMap)
>at org.apache.avro.generic.Generi
> cData.getDefaultValue(GenericData.java:981)
>at org.apache.avro.data.RecordBui
> lderBase.defaultValue(RecordBuilderBase.java:135)
>at com.model.avro.SEGMENT1B$Build
> er.build(SEGMENT1B.java:4362)
>
> "pool-6-thread-2" #50 prio=5 os_prio=0 tid=0x7fad8953a800 nid=0x4325
> runnable [0x7fad52b36000]
>java.lang.Thread.State: RUNNABLE
> at java.util.Collections$SynchronizedMap.get(Collections.java:
> 2584)
> - locked <0x00066a5e3460> (a java.util.Collections$Synchron
> izedMap)
> at org.apache.avro.generic.GenericData.getDefaultValue(GenericD
> ata.java:981)
>
>
> On Fri, Jan 19, 2018 at 6:04 PM, Nishanth S 
> wrote:
>
>> Hi All,
>>
>> We have a process that reads data from a  local file share  ,serailizes
>> and writes to HDFS in avro format. Currently it is running as a single
>> threaded process. When converted t to a parallel process we did  get  some
>> performance improvement  but  not the desired .Thread dumps show  that at
>> any time only on thread  has access to  this method and others are  blocked
>> .I am just wondering if I am building the avro objects correctly.
>>
>> "pool-6-thread-5" #53 prio=5 os_prio=0 tid=0x7fad896c7800 nid=0x4328
>> waiting for monitor entry [0x7fad52833000]
>>   java.lang.Thread.State: BLOCKED (on object monitor)
>>at java.util.Collections$Synchron
>> izedMap.get(Collections.java:2584)
>>- waiting to lock <0x00066a5e3460> (a
>> java.util.Collections$SynchronizedMap)
>>at org.apache.avro.generic.Generi
>> cData.getDefaultValue(GenericData.java:981)
>>at org.apache.avro.data.RecordBui
>> lderBase.defaultValue(RecordBuilderBase.java:135)
>>
>>
>> "pool-6-thread-4" #52 prio=5 os_prio=0 tid=0x7fad896c6000 nid=0x4327
>> waiting for monitor entry [0x7fad52934000]
>>   java.lang.Thread.State: BLOCKED (on object monitor)
>>at java.util.Collections$Synchron
>> izedMap.get(Collections.java:2584)
>>- waiting to lock <0x00066a5e3460> (a
>> java.util.Collections$SynchronizedMap)
>>at org.apache.avro.generic.Generi
>> cData.getDefaultValue(GenericData.java:981)
>>at org.apache.avro.data.RecordBui
>> lderBase.defaultValue(RecordBuilderBase.java:135)
>>at com.model.avro.SEGMENT1B$Build
>> er.build(SEGMENT1B.java:4362)
>>
>> "pool-6-thread-2" #50 prio=5 os_prio=0 tid=0x7fad8953a800 nid=0x4325
>> runnable [0x7fad52b36000]
>>java.lang.Thread.State: RUNNABLE
>> at 

Re: How to override hashCode/equals on java classes generated by avro.

2017-11-29 Thread Doug Cutting
If you specify "order" : "ignore" in a field then it won't be considered by
hashCode or equals. Processing also won't descend into the value of that
field, speeding things up.

Doug

On Nov 29, 2017 1:15 AM, "Tushar Gosavi"  wrote:

> Hi All,
>
> I am creating java objects using Avro Maven plugin, and I have a
> requirement
> to generate hashCode and equals based on specific fields. Is that possible?
>
> For example, I have a schema, Question
> {
>   "type" : "record",
>   "name" : "Question",
>   "namespace" : "com.tugo.questions.schema",
>   "fields" : [ {
> "name" : "id",
> "type" : "string",
>   }, {
> "name" : "text",
> "type" : "string"
>   }, {
> "name" : "type",
> "type" : "string"
>   },
>    // few more fields
> }
>
> I want to generate hashCode and equals based on just the id string in the
> Question
> schema. By default, the generated class inherit hashCode and equals from
> SpecificRecordBase
> which calls GenericData.hashCode, which iterates over all fields and
> compute the hashCode.
> This takes a considerable amount of time with the large schema. Any advice
> on how to achieve
> this?
>
> Thanks & Regards,
> -Tushar.
>


Re: Java Class Generation Drops Underscore in field Name

2017-08-28 Thread Doug Cutting
This sounds like a bug.  There is logic in
SpecificCompiler#generateMethodName to try to avoid name conflicts, but
it's clearly faulty in this case.  One could add a flag that inhibited
underscore removal, which would help here.  What might be better would be
have a table of method names that have been generated, have compile(Schema)
method clear the table, then have generateMethodName store each generated
name in the table, and, when a name conflicts, suffix it with a $ or number
until it doesn't conflict.

Please file an issue in Jira.  https://issues.apache.org/jira/projects/AVRO

Thanks,

Doug

On Mon, Aug 28, 2017 at 9:30 AM, McCullough, Alex <
alex.mccullo...@capitalone.com> wrote:

> Hey All,
>
>
>
> I have an avro scheme that defines two fields with very close names such
> as:
>
> · field_random_name
>
> · fieldrandom_name
>
>
>
> Now from a  schema perspective these are unique names, but when the Java
> classes are generated I have issues because the underscores are removed and
> the fields end up the same and ultimately cause the create of duplicate
> methods with the same signature.
>
>
>
> Is there a solution to this so the java class methods retain the
> underscores? I realize your first response might be, naming fields like
> that is crazy, just change the field names…. Unfortunately, these are
> client defined schemas and messages and so changing them is not so straight
> forward, I wanted to explore other options before saying that’s the only
> way to reasonably manage this.
>
>
>
> Let me know if you need any other info.
>
>
>
> Thanks,
>
> Alex
>
>
>
> --
>
> The information contained in this e-mail is confidential and/or
> proprietary to Capital One and/or its affiliates and may only be used
> solely in performance of work or services for Capital One. The information
> transmitted herewith is intended only for use by the individual or entity
> to which it is addressed. If the reader of this message is not the intended
> recipient, you are hereby notified that any review, retransmission,
> dissemination, distribution, copying or other use of, or taking of any
> action in reliance upon this information is strictly prohibited. If you
> have received this communication in error, please contact the sender and
> delete the material from your computer.
>


Re: appending to Object Container Files

2017-08-25 Thread Doug Cutting
The Avro file format supports appends.  However some filesystems
(e.g., HDFS and S3) may not support append, so applications generally
avoid depending on it.  Also, it can complicate application semantics
when the contents of files change after they are first created.

The Java API supports append:

https://avro.apache.org/docs/1.8.2/api/java/org/apache/avro/file/DataFileWriter.html#appendTo(java.io.File)

There is also an open feature request to extend the concat tool to
support append:

https://issues.apache.org/jira/browse/AVRO-1856

Doug

On Fri, Aug 25, 2017 at 12:10 PM, Matthew Stowe
 wrote:
> Hello Avro User Community,
>
>
>
> I am reviewing the specification for Object Container Files
> (http://avro.apache.org/docs/1.8.2/spec.html#Object+Container+Files) and the
> available Avro libraries from Apache and Microsoft.
>
>
>
> I am hoping the community can help clarify a question I have with respect to
> Object Container files.
>
>
>
> My question is…
>
> Is it compliant with the specification to append objects to an Object
> Container File after the header and some number of initial objects have been
> written and the file closed?
>
>
>
> The reason I ask this question is that while the specification does not
> explicitly state that appending to an Object Container File is not allowed,
> none of the libraries I have evaluated support doing so.  The libraries I
> have looked at support creating a new Object Container File, writing the
> header, some number of objects, and then closing it… but they do not support
> coming back to that file at a later time and appending more objects.
>
>
>
> Is this simply a gap in the libraries or am I missing something in the
> specification that states appending to Object Container Files is prohibited?
> Or is this rather against some best practice that I am not aware of, such as
> in HDFS?
>
>
>
> Cheers,
>
> Matt


Re: Typo in the docs?

2017-07-17 Thread Doug Cutting
Aliases are only applicable to named types and fields.

https://avro.apache.org/docs/current/spec.html#Aliases

Named types are records, enums and fixed:

https://avro.apache.org/docs/current/spec.html#names

The fixed documentation that says, "supports two attributes" is indeed
confusing.  We should perhaps change it to something like "supports the
following attributes".   Also, the "namespace" attribute should be noted as
optional.  Please file a Jira issue.

Thanks,

Doug


On Wed, Jul 12, 2017 at 12:44 PM, Jeremy Fine 
wrote:

> I think they mean that fixed REQUIRES two attributes. I'm pretty sure
> 'alias' is valid for all types. But you're right, the docs there could use
> a bit of cleaning up. But what docs don't?
>
>
> On Wed, Jul 12, 2017 at 2:52 PM, Andy Chambers 
> wrote:
>
>> Hey Folks,
>>
>> The docs about the "fixed" type seem a little confused.
>>
>> It says "Fixed uses the type name "fixed" and supports two attributes:"
>> and then goes on to list 3 attributes, including "aliases" whose
>> description says "a JSON array of strings, providing alternate names for
>> this enum (optional)." which seems to be copy/pasted from the enum type.
>>
>> I'm guessing that aliases is not really a valid attribute for the fixed
>> type right?
>>
>> Cheers,
>> Andy
>>
>
>
>
> --
>
> [image: Intent Media] 
>
> *Jeremy Fine*
> Software Engineer
> 315 Hudson Street, 9th Floor, New York, NY 10013
> We've been named a Crain's "Best Place to Work" for three years running!
> 
>


Re: Parsing canonical forms with schemas having default values.

2017-06-07 Thread Doug Cutting
When reading data, two schemas are used: a schema with the same
fingerprint as used to write the data, typically the actual schema
used to write, and the schema you'd like to project to.  Default
values are only used from the latter schema.

Matching fingerprints indicate binary compatibility.  Schema
resolution allows evolution to a schema with a different binary
format, i.e., with additional fields that specify a default value.

Schema compatibility through resolution cannot be represented in a
single number like a fingerprint.

Doug

On Tue, Jun 6, 2017 at 11:41 AM, Satish Duggana
 wrote:
> https://avro.apache.org/docs/1.8.1/spec.html#Parsing+Canonical+Form+for+Schemas
>> Parsing Canonical Form is a transformation of a writer's schema that let's
>> us define what it means for two schemas to be "the same" for the purpose of
>> reading data written agains the schema. It is called Parsing Canonical Form
>> because the transformations strip away parts of the schema, like "doc"
>> attributes, that are irrelevant to readers trying to parse incoming data. It
>> is called Canonical Form because the transformations normalize the JSON text
>> (such as the order of attributes) in a way that eliminates unimportant
>> differences between schemas. If the Parsing Canonical Forms of two different
>> schemas are textually equal, then those schemas are "the same" as far as any
>> reader is concerned, i.e., there is no serialized data that would allow a
>> reader to distinguish data generated by a writer using one of the original
>> schemas from data generated by a writing using the other original schema.
>> (We sketch a proof of this property in a companion document.)
>
>
> Currently, it keeps only attributes of type, name, fields, symbols, items,
> values, size and strips all others including default attribute.
> Should not default attribute also be kept? Because schema with default value
> and without default value are not canonically same with respect to schema
> evolution.
>
> Thanks,
> Satish.
>


Re: avro-tools not serialising multibyte chars today

2017-03-17 Thread Doug Cutting
Maybe your JVM's default charset has changed?  Try -Dfile.encoding="UTF-8"
when you start Java.

Even if that fixes things, it's perhaps still a bug.  The tool should
probably not depend on the default charset, but should explicitly set its
expected input encoding.  So, if that's the problem, please file an issue.

Doug

On Mar 17, 2017 2:06 AM, "nick lawson"  wrote:

> In the past I have used the avro-tools jar "fromjson" to convert a json
> file
> containing utf-8 multibyte chars to avro as expected.  This data is type
> "bytes" in the schema.
>
> Today this isn't working for me  - instead the multibyte characters are
> each
> represented in my avro output as a single ? (questionmark).
>
> No doubt this is due to me changing something in myenvironment. Does anyone
> know what I need to set/download to get back to normal running?
>
> Thanks,
>
> Nick
>
>
>
>
>
> --
> View this message in context: http://apache-avro.679487.n3.
> nabble.com/avro-tools-not-serialising-multibyte-chars-today-tp4037037.html
> Sent from the Avro - Users mailing list archive at Nabble.com.
>


Re: Implementation of compatibility rules

2017-02-22 Thread Doug Cutting
Support for aliases should be easy to add by calling
Schema#applyAliases before the compatibility check.

Whether aliases should be applied depends on whether the compatibility
check is meant to be valid only for implementations that support
aliases or also ones that do not.

Note that support for aliases might be implemented through a service.
A schema registry service could be extended to also apply aliases.  A
command to retrieve a writer's schema with a given ID could also be
provided the reader's schema, and its result would be the writer's
schema with the reader's aliases applied.

Doug

On Wed, Feb 22, 2017 at 8:47 AM, Joseph P.  wrote:
> This change (considering alias in schema compatibility) is really welcomed
> and needed in our usage of it. So thanks a lot for this much needed change
> (IMHO).
>
> best,
> joseph
>
> On Wed, Feb 22, 2017 at 4:55 PM, Elliot West  wrote:
>>
>> 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 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  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  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: java.math.BigDecimal to Avro .avdl file help please

2017-02-16 Thread Doug Cutting
I believe this has already been implemented but not yet released.

It was implemented in:

https://issues.apache.org/jira/browse/AVRO-1847

This is slated to be included in the 1.8.2 release, which should soon be out.

Doug

On Wed, Feb 15, 2017 at 6:50 PM, Steve Sun  wrote:
> Thanks Nathan for your info and effort.
>
> I really appreciate it.
>
> Also, I posted the same question on Stackoverflow:
> http://stackoverflow.com/questions/41969300/java-math-bigdecimal-to-avro-avdl-file,
> looks like we're not alone.
>
> Feel free to share your updates below that question as well to benefit more
> developers.
>
> Steve
>
>
>
> On Wed, Feb 15, 2017 at 8:14 AM, Nathan Fisher 
> wrote:
>>
>> Hi Steve,
>>
>> I'm seeing the same issue. I think you'll need to drop down to JSON. The
>> IDL appears to provide a subset of the Avro specification.
>>
>> I've started mapping the gaps with the following record type:
>>
>> enum YesNo { YES, NO }
>> fixed TWO(2);
>>
>> record IdlHarness {
>> // primitives
>> boolean bool = true;
>> int i = 0;
>> int iN = null;
>> long l = 0;
>> long lN = null;
>> float f = 0.1;
>> float fN = 0.1;
>> double d = 0.1;
>> double dN = null;
>> bytes b = "banana";
>> bytes bN = null;
>> string s = "s";
>> string sN = null;
>>
>> // complex
>> YesNo en = "YES";
>> YesNo enN = null;
>> array a = [1];
>> array aN = null;
>> map m = { "hello": 1 };
>> map mN = null;
>> union {null, string} un = "hello";
>> union {null, string} unN = null;
>> TWO fix = "12";
>> TWO fixN = null;
>>
>> // logical
>> // decimal dec; // error
>> date dt = "2017-01-01";
>> date dtN = null;
>> // time-millis ms; // fails
>> // time-micros us; // fails
>> // timestamp-millis tsms; // fails
>> // timestamp-micros tsus; // fails
>> // duration du; // fails
>> }
>>
>> I've put my results into the Google sheet below. It's still a work in
>> progress and I'll try to turn it into a JUnit test scenario.
>>
>> https://docs.google.com/spreadsheets/d/1dQAyWbrD5PmHzMJ7sVovHBoZug6jlt_x_Htc7ixdbP8/edit?usp=sharing
>>
>> Kind regards,
>> Nathan
>>
>> On Wed, 1 Feb 2017 at 05:12 Steve Sun  wrote:
>>>
>>> Hi Avro users mailing list,
>>>
>>> I'm having trouble writing an Avro schema for java.math.BigDecimal type,
>>> I tried the following:
>>>
>>> 1. Based on Avro official doc:
>>> https://avro.apache.org/docs/1.8.1/spec.html#Decimal, I know I need to
>>> define Logical Types myself to support BigDecimal, but that link gives
>>> example only in avsc, I'm trying to figure it out in avdl.
>>> 2. Based on Avro
>>> dochttps://avro.apache.org/docs/1.7.6/api/java/org/apache/avro/reflect/package-summary.html
>>> and this
>>> example:https://github.com/apache/avro/blob/master/share/test/schemas/stringables.avdl,
>>> I wrote below avdl:
>>>
>>>
>>>
>>> @namespace("test")
>>> protocol My_Protocol_v1 {
>>>
>>>   record BigDecimal {
>>> @java-class("java.math.BigDecimal") string value;
>>>   }
>>>
>>>
>>>
>>> But it's not working:
>>> This IDL schema compiles fine and can generate a Java class called
>>> BigDecimal, but I cannot really use the generated BigDecimal as
>>> java.math.BigDecimal, what's wrong? or How should I do it?
>>>
>>> Thanks a lot
>>> Steve
>>
>> --
>> - from my thumbs to your eyes
>
>


Re: avro maven generated code doesn't implement serialize

2016-06-06 Thread Doug Cutting
SpecificRecordBase implements Serializable.

Doug

On Mon, Jun 6, 2016 at 11:27 AM, Giri P  wrote:

> Hi,
>
> When I generate the code using avro maven plugin it doesn't implement
> serilalizable.
>
> public class Attributed extends
> org.apache.avro.specific.SpecificRecordBase implements
> org.apache.avro.specific.SpecificRecord {
>
> I see jira https://issues.apache.org/jira/browse/AVRO-1502 which was
> resolved to implement serializable.
>
> Am I missing something when I generate the code.
>
> Thanks
> Giri
>


Re: Not able to resolve union for array type

2016-06-03 Thread Doug Cutting
Your schema permits null or an array of records. I suspect you want an
array containing nulls or records, e.g.,

{"type":"array","items":["null",{"type":"record"...
On Jun 3, 2016 5:54 PM, "Giri P"  wrote:

> Hi,
>
> I'm getting below error when I try to insert null into union of array
>
> Caused by: org.apache.avro.UnresolvedUnionException: Not in union
> ["null",{"type":"array","items":{"type":"record","name":"DeMatchedAttr","namespace":"cp","fields":[{"name":"id","type":"long","doc":"Attribute
> id"},{"name":"updateDate","type":"long","doc":"Update date of the
> attribute"},{"name":"value","type":"string","doc":"Value of the
> attribute"}]}}]: null
>
> Is there any issue with the schema ?
>
> Thanks for the help
>
> -Giri
>


Re: OutOfMemoryError while writing a large map to a file

2016-01-28 Thread Doug Cutting
On Thu, Jan 28, 2016 at 7:51 AM, David Kincaid  wrote:
> Does anyone have a suggestion for making the write not try to copy the data
> in memory as it's writing?

BlockingBinaryEncoder is meant to do this, but I don't know how much
it's been used.

https://avro.apache.org/docs/current/api/java/org/apache/avro/io/BlockingBinaryEncoder.html

Doug


Re: Adding new field with default value to an Avro schema

2015-02-03 Thread Doug Cutting
On Tue, Feb 3, 2015 at 9:34 AM, Lukas Steiblys lu...@doubledutch.me wrote:
 On a related note, is there a tool that can check the backwards
 compatibility of schemas?

https://avro.apache.org/docs/current/api/java/org/apache/avro/SchemaCompatibility.html

Doug


Re: Feature: Clear all fields / Reset all fields to default value on Record template

2015-01-07 Thread Doug Cutting
On Tue, Jan 6, 2015 at 1:33 PM, Maulik Gandhi mmg...@gmail.com wrote:
 I was wondering if adding a functionality of clearing all fields on Record,
 makes sense or not?
 I was wondering if adding a functionality of reseting all fields to default
 value (the default value would be what has been defined in AVDL) on Record,
 makes sense or not?

These both seem reasonable to me.

The simplest thing to do is to clear the fieldSetFlags.  This could be
done with a method on RecordBuilderBase, something like (untested):

public void reset() {
  Arrays.fill(fieldSetFlags(), false);
}

The builder uses default values for fields that have not been set.

Doug


Re: Exception No protocol name specified for Record

2015-01-07 Thread Doug Cutting
You're calling GoraCompiler#main, which inherits
SpecificCompiler#main, which unconditionally calls compileSchema.

You could wrap your record schema in {protocol:X, ..., types: [
schema ] } so this works.

Or you might use SpecificCompilerTool, which supports compiling both
schemas and protocols from the command line.  This is in the
avro-tools jar with dependencies included or in avro-tools-nodeps
without.

Doug



On Wed, Jan 7, 2015 at 10:49 AM, Lewis John Mcgibbney
lewis.mcgibb...@gmail.com wrote:
 Hey Joey or anyone else.
 Any ideas what is up here? I read through the Avro spec to see if I was
 missing something (which I most likely am) but couldn't find a sausage.
 Thanks again in advance for any help.
 The schema is valid JSON, the stack trace is looking for a Protocol when it
 is clearly marked as a Record.

 On Tue, Jan 6, 2015 at 2:42 PM, Joey Echeverria j...@cloudera.com wrote:

 Can you send the definition of the ant task that you're using to parse it?

 -Joey

 On Tue, Jan 6, 2015 at 1:50 PM, Lewis John Mcgibbney
 lewis.mcgibb...@gmail.com wrote:
  Hi Folks,
  I have the following small schema
 
  {
name: Host,
type: record,
namespace: org.apache.nutch.storage,
doc: Host represents a store of webpages or other data which
  resides on
  a server or other computer so that it can be accessed over the
  Internet,
fields: [
  {
name: metadata,
type: {
  type: map,
  values: [null,bytes]
},
doc: A multivalued metadata container used for storing a wide
  variety of host metadata such as structured web server characterists
  etc,
default: {
 
}
  },
  {
name: outlinks,
type: {
  type: map,
  values: [null,string]
},
doc: Hyperlinks which direct outside of the current host domain
  these can used in a histogram style manner to generate host statistics,
default: {
 
}
  },
  {
name: inlinks,
type: {
  type: map,
  values: [null,string]
},
doc: Hyperlinks which link to pages within the current host
  domain
  these can used in a histogram style manner to generate host statistics,
default: {
 
}
  }
]
  }
 
 
  When I use Avro 1.7.6 to parse it via an Ant task, I am getting the
  following parse exception
 
   [java] Caused by: org.apache.avro.SchemaParseException: No protocol
  name specified:
 
  {name:Host,type:record,namespace:org.apache.nutch.storage,doc:Host
  represents a store of webpages or other data which resides on a server
  or
  other computer so that it can be accessed over the
 
  Internet,fields:[{name:metadata,type:{type:map,values:[null,bytes]},doc:A
  multivalued metadata container used for storing a wide variety of host
  metadata such as structured web server characterists
 
  etc,default:{}},{name:outlinks,type:{type:map,values:[null,string]},doc:Hyperlinks
  which direct outside of the current host domain these can used in a
  histogram style manner to generate host
 
  statistics,default:{}},{name:inlinks,type:{type:map,values:[null,string]},doc:Hyperlinks
  which link to pages within the current host domain these can used in a
  histogram style manner to generate host statistics,default:{}}]}
   [java] at org.apache.avro.Protocol.parseName(Protocol.java:426)
   [java] at org.apache.avro.Protocol.parse(Protocol.java:399)
   [java] at org.apache.avro.Protocol.parse(Protocol.java:390)
   [java] at org.apache.avro.Protocol.parse(Protocol.java:361)
   [java] at
 
  org.apache.avro.compiler.specific.SpecificCompiler.compileProtocol(SpecificCompiler.java:248)
   [java] at
 
  org.apache.avro.compiler.specific.SpecificCompiler.compileProtocol(SpecificCompiler.java:238)
   [java] at
 
  org.apache.avro.compiler.specific.SpecificCompiler.main(SpecificCompiler.java:719)
   [java] at sun.reflect.NativeMethodAccessorImpl.invoke0(Native
  Method)
   [java] at
 
  sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)
   [java] at
 
  sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
   [java] at java.lang.reflect.Method.invoke(Method.java:606)
   [java] at
  org.apache.tools.ant.taskdefs.ExecuteJava.run(ExecuteJava.java:217)
   [java] at
  org.apache.tools.ant.taskdefs.ExecuteJava.execute(ExecuteJava.java:152)
   [java] ... 20 more
   [java] Java Result: -1
 
  The thing is that I do not wish this to be a Protocol as such, it is
  merely
  a simple Schema. Can anyone help me out to debug this?
  Thank you
  Lewis
 
  --
  Lewis



 --
 Joey Echeverria




 --
 Lewis


Re: Avro schema and data read with it.

2014-12-17 Thread Doug Cutting
Avro skips over fields that were present in the writer's schema but
are no longer present in the reader's schema.  Skipping is
substantially faster than reading for most types.  For known-size
types like string, bytes, fixed, double and float the file pointer can
be incremented past skipped values.  For skipped structures like
records, maps and arrays, no memory is allocated and no stores are
made.  Avro data files are not in a columnar format however, so the
i/o and decompression of skipped fields is not generally avoided.

Doug

On Wed, Dec 17, 2014 at 7:53 AM, ÐΞ€ρ@Ҝ (๏̯͡๏) deepuj...@gmail.com wrote:
 I have a data that is persisted in Avro format. Each record has a certain
 schema and it contains 10 fields while it is persisted.

 When I read the same record(s) from other process, i also specify a schema
 with a subset of fields (5).

 Will only 5 columns be read from disk?
 or
 Will all the columns be read but 5 are later discarded?
 or
 Are all the columns read but only five are accessible since the schema used
 to read contain only five columns?

 Please suggest.

 Regards,
 Deepak



Re: Reading huge records

2014-12-16 Thread Doug Cutting
Avro does permit partial reading of arrays.

Arrays are written as a series of length-prefixed blocks:

http://avro.apache.org/docs/current/spec.html#binary_encode_complex

The standard encoders do not write arrays as multiple blocks, but
BlockingBinaryEncoder does.  It can be used with any DatumWriter
implementation.  If you, for example, have an array whose
implementation is backed by a database and contains billions of
elements, it can be written as a single Avro value with a
BlockingBinaryEncoder.

http://avro.apache.org/docs/current/api/java/org/apache/avro/io/BlockingBinaryEncoder.html

All Decoder implementations read array blocks correctly, but none of
the standard DatumReader implementations support reading of partial
arrays.  So you could use the Decoder API directly to read your data,
or you might extend an existing DatumReader to read partial arrays.
For example, you might override GenericDatumReader#readArray() to only
read the first N elements, then skip the rest.  Or all the array
elements might be stored externally as they are read.

Doug


On Mon, Dec 15, 2014 at 2:54 PM, yael aharon yael.aharo...@gmail.com wrote:
 Hello,
 I need to read very large avro records, where each record is an array which
 can have hundreds of millions of members. I am concerned about reading a
 whole record into memory at once.
 Is there a way to read only a part of the record instead?
 thanks, Yael


Re: GenericData union validation - IndexOutOfBoundsException

2014-12-16 Thread Doug Cutting
Yes, this looks like a bug in GenericData#validate().  It should use
GenericData#resolveUnion().

Please file an issue in Jira.  If you are able, attach a patch that
includes a test.

Thanks,

Doug

On Tue, Dec 9, 2014 at 12:30 PM, Jeffrey Mullins (BLOOMBERG/ BOSTON)
jmullin...@bloomberg.net wrote:
 Hi,

 When invoking GenericData.get().validate() with a union schema containing 
 multiple record schemas an IndexOutOfBoundsException is possible.

 The exception results under the following conditions:
   1) Union contains multiple record schemas
   2) The first record schema in the union contains more fields than 
 subsequent record schemas
   3) validate() is invoked with the union schema and a record of one of the 
 subsequent record schemas.

 Below is a simple unit test which reproduces the exception.

   @Test
   public void testValidateUnion() {
   Schema type1Schema = SchemaBuilder.record(Type1)
   .fields()
   .requiredString(myString)
   .requiredInt(myInt)
   .endRecord();

   Schema type2Schema = SchemaBuilder.record(Type2)
   .fields()
   .requiredString(myString)
   .endRecord();

   Schema unionSchema = SchemaBuilder.unionOf()
   .type(type1Schema).and().type(type2Schema)
   .endUnion();

 GenericRecord record = new GenericData.Record(type2Schema);
 record.put(myString, myValue);
 GenericData.get().validate(unionSchema, record);
   }

 I'm happy to submit a JIRA for this issue, but thought I'd check in with the 
 mailing list before doing so.

 Thanks,
 Jeff


Re: How to fix Expected start-union. Got VALUE_NUMBER_INT when converting JSON to Avro on the command line?

2014-12-15 Thread Doug Cutting
Avro's JSON encoding requires that non-null union values be tagged
with their intended type.  This is because unions like
[bytes,string] and [int,long] are ambiguous in JSON, the first
are both encoded as JSON strings, while the second are both encoded as
JSON numbers.

http://avro.apache.org/docs/current/spec.html#json_encoding

Thus your record must be encoded as:

{name: Alyssa, favorite_number: {int: 7}, favorite_color: null}

Doug

On Mon, Dec 15, 2014 at 7:27 AM, Emre Sevinc emre.sev...@gmail.com wrote:
 Hello,

 I'm trying to validate a JSON file using an Avro schema and write the
 corresponding Avro file. First, I've defined the following Avro schema named
 user.avsc:

 {namespace: example.avro,
  type: record,
  name: user,
  fields: [
  {name: name, type: string},
  {name: favorite_number,  type: [int, null]},
  {name: favorite_color, type: [string, null]}
  ]
 }

 Then created a user.json file:

 {name: Alyssa, favorite_number: 7, favorite_color: null}

 And then tried to run:

 java -jar ~/bin/avro-tools-1.7.7.jar fromjson --schema-file user.avsc
 user.json  user.avro

 But I get the following exception:

 Exception in thread main org.apache.avro.AvroTypeException: Expected
 start-union. Got VALUE_NUMBER_INT
 at org.apache.avro.io.JsonDecoder.error(JsonDecoder.java:697)
 at org.apache.avro.io.JsonDecoder.readIndex(JsonDecoder.java:441)
 at
 org.apache.avro.io.ResolvingDecoder.doAction(ResolvingDecoder.java:290)
 at org.apache.avro.io.parsing.Parser.advance(Parser.java:88)
 at
 org.apache.avro.io.ResolvingDecoder.readIndex(ResolvingDecoder.java:267)
 at
 org.apache.avro.generic.GenericDatumReader.read(GenericDatumReader.java:155)
 at
 org.apache.avro.generic.GenericDatumReader.readField(GenericDatumReader.java:193)
 at
 org.apache.avro.generic.GenericDatumReader.readRecord(GenericDatumReader.java:183)
 at
 org.apache.avro.generic.GenericDatumReader.read(GenericDatumReader.java:151)
 at
 org.apache.avro.generic.GenericDatumReader.read(GenericDatumReader.java:142)
 at org.apache.avro.tool.DataFileWriteTool.run(DataFileWriteTool.java:99)
 at org.apache.avro.tool.Main.run(Main.java:84)
 at org.apache.avro.tool.Main.main(Main.java:73)


 Am I missing something? Why do I get Expected start-union. Got
 VALUE_NUMBER_INT.


 --
 Emre Sevinç


Re: How to bind several Responder in single port

2014-12-15 Thread Doug Cutting
On Mon, Dec 15, 2014 at 12:21 AM, 聂琨琳 nie...@126.com wrote:
 Is there any way to bind several Responder in single port?

That's not currently supported.

You could programatically create a protocol with the union of the
types and messages of several protocols and serve that with a single
responder.  That responder could then delegate requests to responders
specific to each protocol.

Doug


Re: Building skip table for Avro data

2014-12-08 Thread Doug Cutting
On Thu, Dec 4, 2014 at 8:05 PM, Ken Krugler kkrugler_li...@transpac.com wrote:
 a. how is it sorted lexicographically (as per the SortedKeyValueFile
 JavaDocs)?

The key/value pairs are sorted by their key schema, as per Avro's
order specification:

http://avro.apache.org/docs/current/spec.html#order

 b. How would a reader who's expecting a regular Avro file read the records?
 Would they get records that were the union of fields in the key + value
 schemas?

Looking at the source, it looks like the key/value pairs are stored as
a record schema with fields named key and value.  The name of the
record is org.apache.avro.mapreduce.KeyValuePair.

Doug


Re: avdl schema compatibility

2014-10-27 Thread Doug Cutting
The type of the objfsptr parameter looks correct to me.  Rather than a
reference to the name of the record it has the record's definition.
What looks incorrect about this to you?

That said, I have no idea what's causing that error.

Doug

On Sun, Oct 26, 2014 at 12:20 PM, Camp, Jonathan
jonathan.c...@bluecoat.com wrote:
 Hi All, I am trying to use avro's protocol definition to provide the basis 
 for an RPC system. In this case, I'll be handling the handshaking and 
 transport layer rather the avro's IPC package. Are the schemas generated from 
 the protocol files compatible with the standard DataFileWriter/Reader schemas 
 (I am using python3)? If I try to load a file generated from an avdl message 
 (using DataFileWriter(bytes, DatumWriter(), 
 protocol.message_map['hash'].request)) then when I try to load it, I receive 
 error messages stating that the writer_schema is essentially invalid (see 
 stacktrace below).

 Here is a code snippet that seems to show an invalid schema being generated. 
 I think its because the value of the 'type' field (for objfsptr) is a 
 python/json dictionary rather that the 'record' type i think its supposed to 
 be.

 import avro.protocol
 p = avro.protocol.Parse(open(service.avpr).read())
 print(p.message_map['hash'].request.to_json())

 [{'name': 'objfsptr',
   'type': {'fields': [{'name': 'uri', 'type': 'string'}],
'name': 'ObjFSPtr',
'namespace': 'FooServiceType',
'type': 'record'}},
  {'default': 10, 'name': 'x', 'type': 'int'}]


 Stacktrace from a deserialization attempt:

 Traceback (most recent call last):
   File ./schema_test.py, line 74, in module
 requests = deserializer.deserialize('hash', srequest)
   File /data/bluecoat/services.py, line 123, in deserialize
 reader = DataFileReader(io.BytesIO(raw_bytes), DatumReader())
   File /data/avro/datafile.py, line 363, in __init__
 schema.Parse(self.GetMeta(SCHEMA_KEY).decode('utf-8')))
   File /data/avro/schema.py, line 1283, in Parse
 return SchemaFromJSONData(json_data, names)
   File /data/avro/schema.py, line 1254, in SchemaFromJSONData
 return parser(json_data, names=names)
   File /data/avro/schema.py, line 1142, in _SchemaFromJSONArray
 return UnionSchema(map(MakeSchema, json_array))
   File /data/avro/schema.py, line 866, in __init__
 self._schemas = tuple(schemas)
   File /data/avro/schema.py, line 1141, in MakeSchema
 return SchemaFromJSONData(json_data=desc, names=names)
   File /data/avro/schema.py, line 1254, in SchemaFromJSONData
 return parser(json_data, names=names)
   File /data/avro/schema.py, line 1154, in _SchemaFromJSONObject
 if type in PRIMITIVE_TYPES:
 TypeError: unhashable type: 'dict'




 service.avdl

 @namespace(FooService)
 protocol MyProtocol {

 @namespace(FooServiceType)
 record ObjFSPtr {
 string uri;
 }

 string hash(FooServiceType.ObjFSPtr objfsptr, int x=10);
 }


 generated service.avpr:

 {
   protocol : MyProtocol,
   namespace : FooService,
   types : [ {
 type : record,
 name : ObjFSPtr,
 namespace : FooServiceType,
 fields : [ {
   name : uri,
   type : string
 } ]
   } ],
   messages : {
 hash : {
   request : [ {
 name : objfsptr,
 type : FooServiceType.ObjFSPtr
   }, {
 name : x,
 type : int,
 default : 10
   } ],
   response : string
 }
   }
 }


 Thanks!

 Jonathan Camp


Re: Converting POJO to Generic or Indexed Records

2014-10-24 Thread Doug Cutting
On Tue, Oct 21, 2014 at 10:11 AM, umahida urvish.mah...@gmail.com wrote:
 I have complicated nested POJOs. I am able to generate the schema using
 ReflectData tool.
 I want to convert them to Generic Record preferably or Indexed Records.


To convert from a reflect representation to generic you can serialize
instances with ReflectDatumWriter then read them with
GenericDatumReader.  For example, something like:

public Object reflectToGeneric(Object object, Schema s) {
ReflectDatumWriterObject writer = new ReflectDatumWriterObject(s);
ByteArrayOutputStream out = new ByteArrayOutputStream();
writer.write(object, EncoderFactory.get().directBinaryEncoder(out, null));
GenericDatumReaderObject reader = new GenericDatumReaderObject(s);
return reader.read(null,
DecoderFactory.get().binaryDecoder(out.toByteArray(), null));
}

If you're converting multiple objects it would be faster to reuse the
writer, reader, encoder, decoder, etc.

Doug


Re: invalid file with avro tools random and tojson

2014-10-24 Thread Doug Cutting
This is a bug.  It works if you change the command to not use standard
output, e.g.:

  random --schema-file schema.avsc --count 20 test.avro

The problem is that TestUtil.java prints something to standard output
that should go to standard error.

I filed an issue in Jira for this and will post a fix there soon.

  https://issues.apache.org/jira/browse/AVRO-1597

Doug

On Tue, Oct 14, 2014 at 5:33 PM, Pritchard, Charles X. -ND
charles.x.pritchard@disney.com wrote:
 Thought I’d ask for a second set of eyes.
 I’m working with a large schema, a basic translation of some tab separated 
 data with many fields (over 300).

 It appears that the output of Random can not be read back as JSON, with this 
 schema.
 Is this a bug, or me simply mis-using the method calls in Avro tools?



 $ java -jar avro-tools-1.7.7.jar random --schema-file schema.avsc --count 20 
 -  test.avro
 $ java -jar avro-tools-1.7.7.jar tojson test.avro
 Exception in thread main org.apache.avro.AvroRuntimeException: 
 java.io.IOException: Block size invalid or too large for this implementation: 
 -51
 at 
 org.apache.avro.file.DataFileStream.hasNextBlock(DataFileStream.java:275)
 at 
 org.apache.avro.file.DataFileStream.hasNext(DataFileStream.java:197)
 at org.apache.avro.tool.DataFileReadTool.run(DataFileReadTool.java:77)
 at org.apache.avro.tool.Main.run(Main.java:84)
 at org.apache.avro.tool.Main.main(Main.java:73)
 Caused by: java.io.IOException: Block size invalid or too large for this 
 implementation: -51
 at 
 org.apache.avro.file.DataFileStream.hasNextBlock(DataFileStream.java:266)
 ... 4 more



Re: Extracting records as bytes from avro file

2014-10-15 Thread Doug Cutting
There is no method to read individual records as binary since they're
not delimited nor length-prefixed.  Instead you can get a block of
records as binary and the count of records in the block and pass that
to a deserializer that parses individual records.

Doug

On Wed, Oct 15, 2014 at 4:47 AM, Ivan Balashov ibalas...@gmail.com wrote:
 Hi,

 Imagine a scenario where one reads avro file and distributes
 individual records (aka datums). Suppose the schema is known, and we
 just need to pass records as bytes to another system for further
 processing.

 Given the above, is there a way to read individual records as bytes
 without deserialising them first?

 As far as the documentation goes it can be easily done with blocks,
 but it doesn't mention records.

 Just to be more clear, there is a method which allows to write encoded
 datum to stream
 org.apache.avro.file.DataFileWriter#appendEncoded

 Is there a similar method for reading records as bytes?

 Thanks,


Re: Adding Conclusion to Avro Spec

2014-09-29 Thread Doug Cutting
On Sat, Sep 27, 2014 at 9:20 PM, Lewis John Mcgibbney
lewis.mcgibb...@gmail.com wrote:
 Is there a possibility that someone can add material on how static the
 specification documentation is

Minor versions are meant to have back-compatible APIs and both back-
and forward-compatible data formats.  Since the specification
primarily concerns data formats, it's pretty stable.  In particular,
every schema since 1.0 can be parsed by every 1.x release and every
datum serialized since 1.0 should be readable by every 1.x release.
We broke this policy once, in 1.3, which incompatibly altered the Avro
file format.  I doubt we'll break it again.  Any further changes to
forward or backward data compatibility would require a 2.0 release.

 where people should go if they find
 something wrong/require more clarification about, etc.

You're there!

This information could be included in the specification.  My instinct
is to add it at the start, rather than the end, but I don't feel too
strongly about that.

Doug


Re: Avro and serializable interface

2014-08-28 Thread Doug Cutting
On Thu, Aug 28, 2014 at 4:22 AM, Casadio Phaedra
phaedra.casa...@datamanagementpa.it wrote:
 Now the problem is that avro beans does not extend serializable interface.

 How can i solve the problem? if it’s possible…

There is a request to change this.  I will upload a patch there soon.

https://issues.apache.org/jira/browse/AVRO-1502

 Also, trough avro tools in command line version, how can i pass
 fieldvisibility param ?

Please file a request to add this in Jira.

Thanks,

Doug


Re: Passively Converting Null Map to be valid

2014-08-27 Thread Doug Cutting
On Wed, Aug 27, 2014 at 11:08 AM, Micah Whitacre mkwhita...@gmail.com wrote:
 We are reading with a BufferedBinaryDecoder and using the new schema as both
 the written and reader schema because the written schema is not preserved
 with the payload so it is not easy to retrieve.

 My questions are:
 1. Is the change we made to add a new defaulted union truly non-passive?

Adding the new union field changes the format for how instances are
written, even those that have a null value for this new field.  (A
union is written as an int indicating the selected schema, then an
instance of that schema.)

 2. Is there a workaround so I can continue to evolve my schema?

To permit schema evolution, you must pass the schema used to write
when reading.  Avro's data file format and RPC take care of this.  If
you're writing Avro data to some other container then you need to do
this yourself.

Doug


Re: Avro multiplexing, availability of authentication info to services

2014-08-22 Thread Doug Cutting
On Fri, Aug 22, 2014 at 6:34 AM, Sam Lawrance s...@illumynite.com wrote:

 is there any way for the service implementation to obtain information
 about the calling client, such as the hostname or some form of context from
 SASL?


Transceiver#getRemoteName() returns the remote host name, but this is not
currently easily accessed from a service implementation.

If access from message implementation is desired, then we could add a
static Responder#getTransceiver() method that reads a ThreadLocal set
around calls to Responder#respond(Message,Object).

If access for every call to a given server is desired, then we might add an
RPCContext#getTransceiver() method that accesses a field set by Responder
that would then be visible to RPCPlugin implementations.

Doug


Re: State of the C++ vs Java implementations

2014-08-15 Thread Doug Cutting
On Thu, Aug 14, 2014 at 1:03 PM, John Lilley john.lil...@redpoint.net
wrote:

 Do you know where I can find a list of codecs supported in Java vs C++?


Grepping the Avro C++ headers, it seems to support just the null codec and
deflate.  These are the two codecs that every implementation is meant to
support.

http://avro.apache.org/docs/current/spec.html#Required+Codecs

It would wonderful if someone contributed Snappy support to C++, but that's
not yet happened.

Java additionally supports snappy, bzip2 and xz.

http://avro.apache.org/docs/current/api/java/org/apache/avro/file/CodecFactory.html

Doug


Re: State of the C++ vs Java implementations

2014-08-14 Thread Doug Cutting
On Thu, Aug 14, 2014 at 11:56 AM, John Lilley john.lil...@redpoint.net
wrote:

 I’m seeing discussion of a new Decimal encoding in the mailing list, and
 it would be bad for us to commit to the C++ Avro, and then find that our
 customers have created Avro files (using Java, MapReduce, etc) that we
 can’t read.  We don’t have control over what files we encounter, and it is
 desirable for our product to read whatever a customer throws at it, within
 reason.


Except for compression codecs, all implementations should be able to read
all data files written by other implementations.  The Avro schema language
has not changed incompatibly since 1.0.  Additions such as Decimal are
back-compatible.  Implementations that have no knowledge of Decimal schemas
can still process data written that contains Decimals but will see them as
byte arrays with schema attributes.

Doug


Re: Avro and Jackson dependency

2014-08-08 Thread Doug Cutting
Avro currently requires Jackson 1.x.  Jackson 2.x has an incompatible
API in a different package.  The two versions of Jackson do not
conflict so both may be used within a single application.

Whether  when to eventually upgrade Avro to use Jackson 2.x is discussed in:

  https://issues.apache.org/jira/browse/AVRO-1126

Doug

On Thu, Aug 7, 2014 at 9:41 AM, Casadio Phaedra
phaedra.casa...@datamanagementpa.it wrote:
 After compiling my schema to java, i’ve this error in class files:



 The type org.codehaus.jackson.JsonParser cannot be resolved. It is
 indirectly referenced from required .class files



 In my project i’ve last Jackson version 2.4.1, seems that avro is using a
 real old one?



 Also in generated bean i’ve found some deprecations, it’s normal?



 Thanks Phaedra.


Re: Problem with cross-references (avro-maven-plugin)

2014-08-04 Thread Doug Cutting
I think it works on Windows because the directory there is listed in
an order that happens to work, whereas in Linux the directory ordering
doesn't happen to work.  We could perhaps always sort directory
listings alphabetically so this is deterministic, but requiring folks
to create schema names so that their import order corresponds to
alphabetical order doesn't seem ideal.

Order matters because Avro doesn't support forward declarations.  When
includes are interdependent, specifying includes as directories is
error prone.  For independent includes, an import directory is
reasonable.

We should better document this.  Where should we add such documentation?

Adding support for forward declarations is a topic that has been
discussed a bit previously:

  http://s.apache.org/OKV

Forward declarations might be implemented for the schema parser used
by the compiler, but the parser used when parsing schemas from data
files, RPC handshakes, etc. should probably not permit forward
declarations.

Doug

On Mon, Aug 4, 2014 at 12:28 PM, Peterson, Michael
michael.d.peter...@truvenhealth.com wrote:
 Hello,



 We recently refactored our large avro schema into separate schemas to make
 them more composable, since many of them get incorporated into other
 objects.  We were glad to see that the avro-maven-plugin allows this
 composable model rather than having to do one large schema with everything
 embedded.



 However, we have hit a problem – the Parser cannot always resolve the
 cross-references.  I upgraded to avro-1.7.7 (both the core avro and the
 avro-maven-plugin project), but that does not solve the issue.



 The problem is that names/schemas are not always resolved.  Worse, we see
 different behavior on Windows vs. Linux.  Below I show a set up with a dummy
 schema that works on Windows 7, but fails on Linux (tested on Centos 6 and
 Ubuntu 14.04).  We also have a more complicated schema with many objects
 which have a number of cross references (but not circular ones) that fails
 on both Windows and Linux.



 Is this behavior a defect?  Should the Schema.Parser be able to resolve
 these cross-dependencies?



 A larger question is why we need to put things in two directories (top level
 and “imports”).  Couldn’t the Parser be made to resolve references of things
 when they are all in the same directory?







 Here is a detailed example – again this one works on Windows but fails on
 Linux:



 u070072@TST imports$ tree

 imports

 ├── pom.xml   (1.4 KiB)

 ├── src

 │   ├── main

 │  ├── java

 │ ├── quux00

 │├── App.java   (182 B)

 │  ├── resources

 │ ├── avro

 │├── Top.avsc   (473 B)

 │├── import

 │   ├── Bottom.avsc   (239 B)

 │   ├── Foo.avsc   (307 B)

 │   ├── Middle.avsc   (393 B)





 $ mvn -X generate-sources

 [INFO] BUILD FAILURE

 [INFO]
 

 [INFO] Total time: 0.786 s

 [INFO] Finished at: 2014-08-04T15:06:02-05:00

 [INFO] Final Memory: 19M/843M

 [INFO]
 

 [ERROR] Failed to execute goal
 org.apache.avro:avro-maven-plugin:1.7.7:schema (default) on project imports:
 Execution default of goal org.apache.avro:avro-maven-plugin:1.7.7:schema
 failed: Undefined name: Bottom - [Help 1]

 org.apache.maven.lifecycle.LifecycleExecutionException: Failed to execute
 goal org.apache.avro:avro-maven-plugin:1.7.7:schema (default) on project
 imports: Execution default of goal
 org.apache.avro:avro-maven-plugin:1.7.7:schema failed: Undefined name:
 Bottom

 at
 org.apache.maven.lifecycle.internal.MojoExecutor.execute(MojoExecutor.java:224)

 at
 org.apache.maven.lifecycle.internal.MojoExecutor.execute(MojoExecutor.java:153)

 at
 org.apache.maven.lifecycle.internal.MojoExecutor.execute(MojoExecutor.java:145)

 at
 org.apache.maven.lifecycle.internal.LifecycleModuleBuilder.buildProject(LifecycleModuleBuilder.java:108)

 at
 org.apache.maven.lifecycle.internal.LifecycleModuleBuilder.buildProject(LifecycleModuleBuilder.java:76)

 at
 org.apache.maven.lifecycle.internal.builder.singlethreaded.SingleThreadedBuilder.build(SingleThreadedBuilder.java:51)

 at
 org.apache.maven.lifecycle.internal.LifecycleStarter.execute(LifecycleStarter.java:116)

 at org.apache.maven.DefaultMaven.doExecute(DefaultMaven.java:361)

 at org.apache.maven.DefaultMaven.execute(DefaultMaven.java:155)

 at org.apache.maven.cli.MavenCli.execute(MavenCli.java:584)

 at org.apache.maven.cli.MavenCli.doMain(MavenCli.java:213)

 at org.apache.maven.cli.MavenCli.main(MavenCli.java:157)

 at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)

 at
 sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)

 at
 

Re: How to deserialize avro file with union/many schemas?

2014-07-24 Thread Doug Cutting
On Thu, Jul 24, 2014 at 7:23 AM, Echo echo...@gmail.com wrote:
 The avro library can't read the file with that 'union' schema, so I wonder:

With which Avro library can't you read a file with a union schema?
Unions are a standard feature and every implementation should be able
to read a file with such a schema.

You can view the file contents with the 'tojson' command line tool:

java -jar /path/to/avro-tools.jar tojson --pretty myfile.avro

(You can also use the 'getschema' command line tool to view the schema
of the file.)

Doug


Re: How to use java-class with JSON schema?

2014-07-03 Thread Doug Cutting
The java-class attribute is supported by the reflect implementation,
not by the code-generating specific implementation.  So you could
define Foo in Java with something like:

public class Foo {
  private long batchId;
  @Stringable private Timestamp timestamp;
  public Foo() {}
  public Foo(long batchId, Timestamp timestamp) { ... }
}

then use ReflectData to read/write instances.  Note that
java.sql.Timestamp doesn't have a string constructor.  Are you using a
different timestamp class?  If you're defining your own then you could
instead add the @Stringable annotation to your Timestamp class rather
than to each field where it is used.

Reflect-defined schemas can refer to specific-defined classes, but not
vice-versa, since the compiler doesn't use reflection to discover
schemas, but rather always generates from the schema alone.

Doug

On Wed, Jul 2, 2014 at 8:05 AM, Ian Hummel i...@themodernlife.net wrote:
 Hi gang,

 I'm trying to build a JSON schema with a custom type as the field instead of
 just a String.  Is java-class supposed to work in that use case?  I can't
 seem to make any progress.

 Example schema (Foo.avsc):

 {
 namespace : com.example,
 type : record,
 name : Foo,
 fields : [
 { name : batchId, type : long },
 { name : timestamp, type : string, java-class :
 com.example.Timestamp }
 ]
 }

 The Timestamp class has a public constructor which takes a single String
 argument.  I even tried annotating it with @Stringable.  However, the
 generated java class always uses String, not my custom type.

 $ java -jar ~/Downloads/avro-tools-1.7.6.jar compile -string schema
 src/main/avro/Foo.avsc /tmp/foo

 From the generated .java file

 ...

   /**

* All-args constructor.

*/

   public Foo(java.lang.Long batchId, java.lang.String timestamp) {

 this.batchId = batchId;

 this.timestamp = timestamp;

   }

 ...


 Any help appreciated,

 - Ian.


Re: aliasing items in the default namespace

2014-06-30 Thread Doug Cutting
On Fri, Jun 27, 2014 at 5:01 PM, Josh Buffum jbuf...@gmail.com wrote:
 Is there a way for me to create a new record (with a namespace) using an old
 record name (with no namespace) as an alias?

No, but this is easily remedied.

I filed the following issue  attached a fix:

https://issues.apache.org/jira/browse/AVRO-1535

Please let me know if this works for you.

Cheers,

Doug


Re: Schema Coersion

2014-06-27 Thread Doug Cutting
Aliases may help here.  If you have a schema in one namespace and wish
to read data written with a schema in another namespace, potentially
with different field names even, then you can add aliases to your
schema naming the other schema (and its fields, if needed).

For example, if you want to read data written with:

{type:record, name:org.foo.Frob ...}

into a class org.bar.Fritz, then you'd generate code from a schema with:

{type:record, name:org.bar.Fritz, alias:org.foo.Frob ...}

Does that make sense?  Does it apply to your situation?

Doug


On Fri, Jun 27, 2014 at 9:22 AM, Hans Uhlig huh...@uhlisys.com wrote:
 Is it possible to coerce the SpecificDatumReader to read a file from a
 different Schema? We use Avro code generation for speed and type enforcement
 in our environment normally but one of the data sources out of our control.
 This other source doesn't use a namespace which means our use of codegen
 isnt possible in java. Is there a way for us to read the data into our
 codegenerated classes (only difference is the namespace) from the
 namespaceless files we receive? Without doing double class allocation?


Re: Schema Coersion

2014-06-27 Thread Doug Cutting
On Fri, Jun 27, 2014 at 9:40 AM, Pritchard, Charles X. -ND
charles.x.pritchard@disney.com wrote:
 is there a means to coerce data written as byte[] into String (and the other 
 way around) ?

Not easily today.  Binary string values are a subset of bytes (those
that are valid UTF-8 sequences), so string to bytes would be a
lossless conversion.  Bytes to string however isn't guaranteed to
produce valid data.

Perhaps we should permit one or both of these as a standard type promotion in:

http://avro.apache.org/docs/current/spec.html#Schema+Resolution

(Int-to-float and long-to-float are both lossy promotions that we
already permit.)

In the meantime, the standard Java APIs won't permit this conversion.
I posted a patch that adds support for promotion from string to bytes.
Would this be useful?  If so, please add comments to the issue.

https://issues.apache.org/jira/browse/AVRO-1533

Doug


Re: 1.7.6 Slow Deserialization

2014-06-23 Thread Doug Cutting
On Mon, Jun 16, 2014 at 7:59 AM, Han, Xiaodan xiaodan@baml.com wrote:
 Is GenericDatumReader thread safe? What about the writer.

Yes, a DatumReader instance may be used in multiple threads.  Encoder
and Decoder are not thread-safe, but DatumReader and DatumWriter are.

 If we change the behavior of the Reader, would you suggest we also make the 
 same change for the Writer?

Yes, re-using a single GenericDatumWriter to write multiple objects
should improve performance.

Doug


Re: 1.7.6 Slow Deserialization

2014-06-13 Thread Doug Cutting
On Wed, Jun 11, 2014 at 1:05 PM, Han, Xiaodan xiaodan@baml.com wrote:
 org.apache.avro.specific.SpecificDatumReader.findStringClass(SpecificDatumReader.java:80)
 org.apache.avro.generic.GenericDatumReader.getStringClass(GenericDatumReader.java:394)

The result of findStringClass are cached by getStringClass, so there
should only be one call per schema used with a GenericDatumReader
instance.  So, with 22k calls to this method in the samples, perhaps
you're either creating a new schema or a new GenericDatumReader per
instance read.  Could that be possible?

Doug


Re: Custom hashCode function

2014-05-28 Thread Doug Cutting
On Mon, May 26, 2014 at 3:43 AM, Han JU ju.han.fe...@gmail.com wrote:
 My question for the moment: is it possible to custom the hashCode function
 of a avro record? Say a record a a field `uid` and I'd like to return this
 value as the hashCode.

If you're using Avro's reflect model then you can simply override your
hashCode method.

If you're using generic or specific then you can specify a schema
where all other fields are ignored, e.g.:

{type:record, name:org.random.Foo, fields:[
  {name:uid, type:string},
  {name:x, type:long, order:ignore},
  {name:y, type:float, order:ignore},
  {name:z, type:double, order:ignore}
]}

Avro's hashes and comparisons only consider fields that are not ignored.

Doug


Re: Schema import dependencies

2014-05-28 Thread Doug Cutting
Your userInfo.avsc is not a standalone schema since it depends on
mailing_address already being defined.  A schema included in a data file is
always standalone, and would include the mailing_address schema definition
within the userInfo schema's address field.

Some tools will process such non-standalone schemas in separate files.  For
example, the Java schema compiler will accept multiple schema files on the
command line, and those later on the command line may reference types
defined earlier.  Java's maven tasks also permit references to other files,
but these are probably not of interest to a Python developer.

The IDL tool uses the JVM as its runtime but is not Java-specific.

Doug


On Wed, May 28, 2014 at 11:53 AM, Wai Yip Tung w...@tungwaiyip.info wrote:

 I want to extend this question somewhat. I begin to realized avro has
 accommodation to compose schema from user defined type. I want to check if
 I understand it correctly and also the proper way to use it.

 I take a single, two level nested schema from the web (see using an
 embedded record).

 http://docs.oracle.com/cd/E26161_02/html/GettingStartedGuide/avroschemas.html

 I break it down to two separate records. The main `userInfo` record and
 the embedded `mailing_address` record as two separate JSON object.


 
 userInfo.avsc

 {
 type : record,
 name : userInfo,
 namespace : my.example,
 fields : [{name : username,
  type : string,
  default : NONE},

 {name : age,
  type : int,
  default : -1},

  {name : phone,
   type : string,
   default : NONE},

  {name : housenum,
   type : string,
   default : NONE},

  {name : address,
   type : mailing_address,   --- user defined type
   default : NONE},
 ]
 }

 
 mailing_address.avsc

 {
  type : record,
  name : mailing_address, --- defined here
  fields : [
 {name : street,
  type : string,
  default : NONE},

 {name : city,
  type : string,
  default : NONE},

 {name : state_prov,
  type : string,
  default : NONE},

 {name : country,
  type : string,
  default : NONE},

 {name : zip,
  type : string,
  default : NONE}
 ]}
 }
 

 Is this a valid composite avro schema definition?

 The second question is how can we actually use this in practice. If we
 have two separate file, is there a standard API that load them both.
 Hrishikesh P mentions avro maven plugin. I mainly use the Python API so I
 am unfamiliar with this. Is a comparable API exist?

 I understand the IDL form has explicit linking of schema files. I will
 look into it next.

 Wai Yip


   Doug Cutting cutt...@apache.org
  Thursday, May 22, 2014 2:57 PM
 You might instead use Avro IDL to define your schemas. It permits you
 define multiple schemas in a single file, so that you can determine
 the order they're defined in. It also permits ordered inclusion of
 types from other files, both IDL files and schema files.

 Doug

 On Thu, May 22, 2014 at 10:46 AM, Hrishikesh P

   Hrishikesh P hrishi.engin...@gmail.com
  Thursday, May 22, 2014 10:46 AM
 I have a few avro schemas that I am generating the code from using the
 avro maven plugin. I have dependencies in the schemas which I was able to
 resolve by putting the schemas in separate folders and/or renaming the
 schema file names with 01-, 02-, ...etc so that the dependencies get
 compiled first. However, this only works on mac but not on RHEL (probably
 because of the different ways the directories are read on them?). Anybody
 knows the best way to handle schema dependencies? If I specify individual
 schema names in the POM in the imports section, the schemas get compiled
 but I have listed the folders and I would like to avoid listing individual
 files if possible.

 Here's a related issue: https://issues.apache.org/jira/browse/AVRO-1367

 Thanks in advance.




Re: Schema import dependencies

2014-05-28 Thread Doug Cutting
IDL is a language-independent way let you merge two schema files into one
standalone schema file.

Doug


On Wed, May 28, 2014 at 3:40 PM, Wai Yip Tung w...@tungwaiyip.info wrote:

 Let's say we are interested to keep 2 schema file because they come from 2
 separate organization. When we generate a data file they need to be merged
 into one standalone schema. The maven plugin does this. Otherwise we have
 to merge it ourselves. This is not too hard to merge. I just want make sure
 I'm not missing some exiting tool or API available.

 Wai Yip

   Doug Cutting cutt...@apache.org
  Wednesday, May 28, 2014 12:09 PM
 Your userInfo.avsc is not a standalone schema since it depends on
 mailing_address already being defined.  A schema included in a data file is
 always standalone, and would include the mailing_address schema definition
 within the userInfo schema's address field.

 Some tools will process such non-standalone schemas in separate files.
  For example, the Java schema compiler will accept multiple schema files on
 the command line, and those later on the command line may reference types
 defined earlier.  Java's maven tasks also permit references to other files,
 but these are probably not of interest to a Python developer.

 The IDL tool uses the JVM as its runtime but is not Java-specific.

 Doug



   Wai Yip Tung w...@tungwaiyip.info
  Wednesday, May 28, 2014 11:53 AM
  I want to extend this question somewhat. I begin to realized avro has
 accommodation to compose schema from user defined type. I want to check if
 I understand it correctly and also the proper way to use it.

 I take a single, two level nested schema from the web (see using an
 embedded record).

 http://docs.oracle.com/cd/E26161_02/html/GettingStartedGuide/avroschemas.html

 I break it down to two separate records. The main `userInfo` record and
 the embedded `mailing_address` record as two separate JSON object.


 
 userInfo.avsc

 {
 type : record,
 name : userInfo,
 namespace : my.example,
 fields : [{name : username,
  type : string,
  default : NONE},

 {name : age,
  type : int,
  default : -1},

  {name : phone,
   type : string,
   default : NONE},

  {name : housenum,
   type : string,
   default : NONE},

  {name : address,
   type : mailing_address,   --- user defined type
   default : NONE},
 ]
 }

 
 mailing_address.avsc

 {
  type : record,
  name : mailing_address, --- defined here
  fields : [
 {name : street,
  type : string,
  default : NONE},

 {name : city,
  type : string,
  default : NONE},

 {name : state_prov,
  type : string,
  default : NONE},

 {name : country,
  type : string,
  default : NONE},

 {name : zip,
  type : string,
  default : NONE}
 ]}
 }
 

 Is this a valid composite avro schema definition?

 The second question is how can we actually use this in practice. If we
 have two separate file, is there a standard API that load them both.
 Hrishikesh P mentions avro maven plugin. I mainly use the Python API so I
 am unfamiliar with this. Is a comparable API exist?

 I understand the IDL form has explicit linking of schema files. I will
 look into it next.

 Wai Yip


   Doug Cutting cutt...@apache.org
  Thursday, May 22, 2014 2:57 PM
 You might instead use Avro IDL to define your schemas. It permits you
 define multiple schemas in a single file, so that you can determine
 the order they're defined in. It also permits ordered inclusion of
 types from other files, both IDL files and schema files.

 Doug

 On Thu, May 22, 2014 at 10:46 AM, Hrishikesh P

   Hrishikesh P hrishi.engin...@gmail.com
  Thursday, May 22, 2014 10:46 AM
 I have a few avro schemas that I am generating the code from using the
 avro maven plugin. I have dependencies in the schemas which I was able to
 resolve by putting the schemas in separate folders and/or renaming the
 schema file names with 01-, 02-, ...etc so that the dependencies get
 compiled first. However, this only works on mac but not on RHEL (probably
 because of the different ways the directories are read on them?). Anybody
 knows the best way to handle schema dependencies? If I specify individual
 schema names in the POM in the imports section, the schemas get compiled
 but I have listed the folders and I would like to avoid listing individual
 files if possible.

 Here's a related issue: https://issues.apache.org/jira/browse/AVRO-1367

 Thanks in advance.




Re: Is it possible to write a magic byte in Avro file head?

2014-05-23 Thread Doug Cutting
org.apache.avro.mapred.AvroOutputFormat automatically copies metadata
from the job configuration to the output files.  In particular, it
copies values for keys starting with AvroJob.TEXT_PREFIX as strings,
and for those starting with AvroJob.BINARY_PREFIX it decodes a binary
metadata value from the configuration value.

It does not appear that the output formats in
org.apache.avro.mapreduce have yet implemented this feature.  Please
file an issue in Jira if you think it would be useful, ideally
accompanied by a patch that implements it.

Thanks,

Doug

On Fri, May 23, 2014 at 10:16 AM, James Campbell
ja...@breachintelligence.com wrote:
 Is it possible to configure a AvroKeyOutputFormat to include specific
 metadata in each of the output files generated by the job?  I’m thinking of
 metadata about the time/version of the job that produced the entire file,
 which would not necessarily need to be stored in each record.



 2014-05-17 7:20 GMT+08:00 Doug Cutting cutt...@apache.org:



 This incompatibly alters the Avro file format.  Could you perhaps

 instead add this into the Avro file's metadata?



 Doug



 On Thu, May 15, 2014 at 5:44 AM, Fengyun RAO raofeng...@gmail.com wrote:

  I have a cache file using Avro serialization, and I want to add a magic

 byte

  indicating cache version at the beginning of the file.

  I find it's easy to serialize, but difficult to deserialize in C#.

  First I open a filestream, read my magic byte, and then pass the stream

 to

  the DataFileReader:

 

  var reader = DataFileReaderDictionarystring,

 MyType.OpenReader(stream,

  CACHE_SCHEMA)

 

  but it throws an AvroRuntimeException(Not an Avro data file)

 

  I look into the OpenReader() method:

 

// verify magic header

byte[] magic = new byte[DataFileConstants.Magic.Length];

inStream.Seek(0, SeekOrigin.Begin);

 

  It will always seek back to the beginning of the FileStream (which

 includes

  my own byte), and thus throws an Exception.

 

  However, in java version, I could use DataFileStream which wouldn't seek

  back and it works.

 

  Is there a way to make it work in C# version? I also wonder why there

 isn't

  an equivalent DataFileStream class in C#.






Re: Schema import dependencies

2014-05-22 Thread Doug Cutting
You might instead use Avro IDL to define your schemas.  It permits you
define multiple schemas in a single file, so that you can determine
the order they're defined in.  It also permits ordered inclusion of
types from other files, both IDL files and schema files.

Doug

On Thu, May 22, 2014 at 10:46 AM, Hrishikesh P
hrishi.engin...@gmail.com wrote:
 I have a few avro schemas that I am generating the code from using the avro
 maven plugin. I have dependencies in the schemas which I was able to resolve
 by putting the schemas in separate folders and/or renaming the schema file
 names with 01-, 02-, ...etc so that the dependencies get compiled first.
 However, this only works on mac but not on RHEL (probably because of the
 different ways the directories are read on them?). Anybody knows the best
 way to handle schema dependencies? If I specify individual schema names in
 the POM in the imports section, the schemas get compiled but I have listed
 the folders and I would like to avoid listing individual files if possible.

 Here's a related issue: https://issues.apache.org/jira/browse/AVRO-1367

 Thanks in advance.


Re: Is it possible to write a magic byte in Avro file head?

2014-05-16 Thread Doug Cutting
This incompatibly alters the Avro file format.  Could you perhaps
instead add this into the Avro file's metadata?

Doug

On Thu, May 15, 2014 at 5:44 AM, Fengyun RAO raofeng...@gmail.com wrote:
 I have a cache file using Avro serialization, and I want to add a magic byte
 indicating cache version at the beginning of the file.
 I find it's easy to serialize, but difficult to deserialize in C#.
 First I open a filestream, read my magic byte, and then pass the stream to
 the DataFileReader:

 var reader = DataFileReaderDictionarystring, MyType.OpenReader(stream,
 CACHE_SCHEMA)

 but it throws an AvroRuntimeException(Not an Avro data file)

 I look into the OpenReader() method:

   // verify magic header
   byte[] magic = new byte[DataFileConstants.Magic.Length];
   inStream.Seek(0, SeekOrigin.Begin);

 It will always seek back to the beginning of the FileStream (which includes
 my own byte), and thus throws an Exception.

 However, in java version, I could use DataFileStream which wouldn't seek
 back and it works.

 Is there a way to make it work in C# version? I also wonder why there isn't
 an equivalent DataFileStream class in C#.


Re: Avro cycle support

2014-05-16 Thread Doug Cutting
On Wed, May 14, 2014 at 10:38 AM, Bernhard Damberger
bdamber...@walmartlabs.com wrote:
   1.  Is there a plan to add cycle support to Avro? I saw this ticket: 
 https://issues.apache.org/jira/browse/AVRO-695. But it hasn't been worked on 
 since 1/2011.
   2.  Why does Avro handle cyclical schema's but not cyclical objects?
   3.  Does anyone know when non-string map keys will be available (i.e. 
 ticket https://issues.apache.org/jira/browse/AVRO-680)?

In general, we don't centrally plan new Avro features.  Rather, if
someone contributes an implementation of a feature that everyone else
agrees does not conflict with their use of Avro then it is integrated.
 So the reason these have not yet been added is simply that no one has
provided an implementation that's compatible.  If these features are
critical to your use then you could either contribute a compatible
implementation yourself, or, if you work with a vendor, encourage your
vendor to contribute one.

   4.  How does Avro handle inheritance in lists? For example: ListBaseClass 
 storing derived class objects?

Avro doesn't really handle inheritance.

You could use a union, explicitly adding new subclasses to the union
as they're implemented.  So if initially you have subclasses A and B
then the list element type could be the union [A,B].  Then when you
implement subclass C, update the union to be [A,B,C].

Alternately you might push the unions deeper.  The top-level record
would directly contain the fields of the base class, and its fields
would have unions corresponding to the subclass variations.

Doug


Re: How to deserialize java resource file in the jar package?

2014-05-02 Thread Doug Cutting
On Tue, Apr 29, 2014 at 10:43 PM, Fengyun RAO raofeng...@gmail.com wrote:
 Could there also be an API requiring only a Stream object in java, which I
 think would be quite convenient.

Please see DataFileStream, which can be constructed using an
InputStream.  This is the base class for DataFileReader.

http://avro.apache.org/docs/current/api/java/org/apache/avro/file/DataFileStream.html

Doug


Re: Go library

2014-03-20 Thread Doug Cutting
I have not heard of any work on an implementation of Avro in go.  It
would make a great addition, even if only data file support.

Doug

On Sat, Mar 15, 2014 at 5:59 AM, Mike Stanley m...@mikestanley.org wrote:
 Anyone know of any avro libraries for go?   I haven't had much luck finding
 anything.  Either Cgo or pure go is fine by me.  I'm a long time user of
 avro and have a considerable amount of data in it. (Avro is our
 serialization format of choice for all archive data, event logs, and other
 data stored on s3, and in hdfs).  Go is quickly becoming a core technology
 in our stack as well and avro support is one of the impeding areas for wider
 adoption.

 Worse case scenario this may be something I take on.  I'd much rather pick
 up where someone else left of though.I dont need any RPC functionality.
 Just read/write (with compression support).


Re: Problem while Converting from JSON=Avro=JSON

2014-03-14 Thread Doug Cutting
To generate a file with a subset of fields you can specify a 'reader'
schema that contains only the desired fields.  For example, if you
have a schema like:

{type:record,name:Event,fields:[
  {name:id,type:int},
  {name:url,type:string},
  
{name:props,type:{type:array,items:{type:record,name:Property,fields:[
  {name:key,type:int},
  {name:value,type:string}
]}]}

And you only want the ids and property values, then you can specify
the following when you create your GenericDatumReader:

{type:record,name:Event,fields:[
  {name:id,type:int},
  
{name:props,type:{type:array,items:{type:record,name:Property,fields:[
  {name:value,type:string}
]}]}

Perhaps we should add a --schema parameter to the tojson command-line
tool that does this?

Doug

On Fri, Mar 14, 2014 at 1:30 AM, Saravanan Nagarajan
saravanan.nagarajan...@gmail.com wrote:
 HI,

 I successfully converted the JSON file to avro format and  i cloud able to
 see the json format using AVRO tool.

 But not i am trying to show only  selected fields from the json file using
 java program and i cloud able to select specific column from the SIMPLE json
 file.

 In case of complex json file, i am not able to select column.

 For example:

 Assume, Employee records contain complex column with department details. Now
 i need to generate the JSON from avro with few column from employee and few
 column from departments.

 My program printed the selected column from the employee table, but not able
 to select from department columns. I used GenericDatumReader for reading the
 avro file.

 Please let me know if you have any suggestions.

 if you need the program, i can share separate mail.

 Thanks,
 Saravanan



Re: what's the efficiency difference between type: string and [string, null]

2014-03-14 Thread Doug Cutting
One small note: the best practice is to place null first when it's
in a union.  This is because the type of a default value for a union
field is the type of the first element of the union, and null is the
most commonly used default value for unions with null.  So the idiom
for a field that defaults to null is:

{name,field name,type:[null,field type],default:null}

I've updated the specification to clarify this point.

https://issues.apache.org/jira/browse/AVRO-1482

Doug

On Fri, Mar 14, 2014 at 1:56 AM, Bertrand Dechoux decho...@gmail.com wrote:
 I think the specification is clear about that.

 Unions
 A union is encoded by first writing a long value indicating the zero-based
 position within the union of the schema of its value. The value is then
 encoded per the indicated schema within the union.
 For example, the union schema [string,null] would encode:

 null as the integer 1 (the index of null in the union, encoded as hex
 02):

 02

 the string a as zero (the index of string in the union), followed by
 the serialized string:

 00 02 61


 http://avro.apache.org/docs/1.7.6/spec.html

 So there is an overhead but that may not be the main issue.

 The issue might be more about defining a correct schema. If a field can be
 null then all clients should handle the case when the field is indeed null.
 That's a 'hygiene issue' (or data quality issue if your prefer), like with a
 database schema.

 Regards

 Bertrand

 Bertrand Dechoux


 On Fri, Mar 14, 2014 at 9:15 AM, Fengyun RAO raofeng...@gmail.com wrote:

 I have some string fields which may be null, while some definitely not
 null.
 The problem is that it takes time to distinguish them.
 There are about 100 fields, 50 of which are string,  10 of which I guess
 could be null.

 Could I just specify all string types [string, null],
 how much is the efficiency difference?





Re: Question about the state of the AVRO C# (csharp) implementation

2014-03-13 Thread Doug Cutting
On Thu, Mar 13, 2014 at 7:34 AM, Longton, Nigel nigel.long...@pimco.com wrote:
 Are there plans to make C# feature equivalent?

As a volunteer-based open-source project, we don't have a long-term
plan.  Rather we, as a group, consider contributions as they arrive
and generally accept those that aren't counter to other's needs.

The major differences seem to be:

 1.   C# doesn’t  support the Avro IDL schema definition

The IDL compiler does not need to be re-implemented per language.  You
can use the existing IDL compiler to convert IDL to protocols 
schemas, then generate C# from those.  A single implementation of the
IDL compiler is easier to maintain.

 2.   C# doesn’t support the JSON serialization codec

 3.   We had a case where we wanted to deserialize a json array which we
 can do in Java with Schema.createArray. The equivalent construct in C# was
 private and so not externally accessible.

These would make great improvements to Avro C#.  Please open issues in Jira.

 We’re making extensive use of Apache avro to communicate between Java and C#
 applications using JSON serialization rather than binary and in some cases
 we are creating schemas to match non-avro generated json from ‘legacy’
 applications. I copied the Java json codec to C# and have got it working (I
 wont claim its fully tested though). More than happy to post this somewhere
 if  its useful.

This sounds useful.  Please attach it as a patch in Jira.

https://cwiki.apache.org/confluence/display/AVRO/How+To+Contribute

 The other tweak we made was to change how ‘default’ values were serialized.
 We created a new writer classes that if the field value equaled the default
 it didn’t serialize it, and a new reader class that set missing fields to
 the schema default if defined (missing non-defaulted fields are still an
 error). The advantage is that the JSON created is much cleaner and can be
 readily consumed by a jackson or any other json framework. (Currently these
 aren’t ported to C# due to lack of time).

As you note, this is a non-standard mode of operation for Avro, but,
if you find it useful, others might too.  So long as it doesn't alter
Avro's standard behavior we should consider adding it to Avro as an
optional capability.

Cheers,

Doug


Re: How to specify the CSharp class name of a field

2014-03-13 Thread Doug Cutting
On Thu, Mar 13, 2014 at 7:59 AM, Longton, Nigel nigel.long...@pimco.com wrote:
 What is the equivalent to ‘java-class’ for the csharp code generator?

There isn't an equivalent at this point.

 Specifically we’re looking at having fields of date and decimal type.

You could perhaps interpret the 'java-class' attribute that's already there.

Long-term we might agree on a language-independent way of identifying
such types, perhaps:
  {type:long, subType:unix-epoch}
  {type:string, subType:ISO-8601-date}
  {type:string, subType:lexicographic-decimal}

If there's interest, this should be fairly easy to specify  implement.

See: https://issues.apache.org/jira/browse/AVRO-739

Doug


Re: Avro 1.7.6 Enums with fields

2014-03-07 Thread Doug Cutting
You can add attributes to the schema, e.g.:

{type:enum, name:MyEnum,
 symbols:[FOO, BAR, BAZ],
 indexes:[10,11,12],
 descriptions[foo is..., bar is.., baz is]
}

Then, if you generate specific code, you can access this with something like:

int fooIndex = 
MyEnum.getClassSchema().getJsonProp(indexes).get(FOO.ordinal()).getIntValue();

Doug

On Wed, Mar 5, 2014 at 8:45 PM, Daniel Jue teamp...@gmail.com wrote:
 Hi, I'm kind of new to Avro, and I couldn't find an example on this by
 googling.  I have a Java enum that looks like this:

 public enum G_CanonicalPropertyType {
 FOO(10, Foo, foo is not food),
 BAR(11, Bar, bar is like a snickers),
 BAZ(12, Baz, baz is a widget with wings);

 private int index;
 private String friendlyName;
 private String description;
  //then constructor and getters
 }

 Where the fields for each enum are important (there are also getters and
 setters for them)

 I'm trying to write an Avro avdl version of this, what's the best way?

 Thanks,

 Dan


Re: Enum backward compatibility in distributed services...

2014-02-27 Thread Doug Cutting
On Tue, Jan 28, 2014 at 9:43 AM, Amihay Zer-Kavod amih...@gmail.com wrote:
 Bottom line, I would go with Flex approach and retire the Specific approach
 entirely.

I filed an issue in Jira for this.

https://issues.apache.org/jira/browse/AVRO-1468

Doug


Re: Create Avro from bytes, not by fields

2014-02-07 Thread Doug Cutting
You might use DataFileWriter#appendEncoded:

http://avro.apache.org/docs/current/api/java/org/apache/avro/file/DataFileWriter.html#appendEncoded(java.nio.ByteBuffer)

If the body has just single instance of the record then you'd call this
once.  If you have multiple instances then you might change the body to
have the schema {type:array, items, bytes}.

Doug


On Fri, Feb 7, 2014 at 12:06 PM, Daniel Rodriguez df.rodriguez...@gmail.com
 wrote:

 Hi all,

 Some context (not an expert Java programmer, and just starting with
 AVRO/Flume):

 I need to transfer avro files from different servers to HDFS I am trying
 to use Flume to do it.
 I have a Flume spooldir source (reading the avro files) with an avro sink
 and avro sink with a HDFS sink. Like this:

servers  |  hadoop
 spooldir src - avro sink    avro src - hdfs

 When Flume spooldir deserialize the avro files creates an flume event with
 two fields: 1) header contains the schema; 2) and in the body field has the
 binary Avro record data, not including the schema or the rest of the
 container file elements. See the flume docs:
 http://flume.apache.org/FlumeUserGuide.html#avro

 So the avro sink creates an avro file like this:

 {headers: {flume.avro.schema.literal:
 {\type\:\record\,\name\:\User\,\namespace\:\example.avro\,\fields\:[{\name\:\name\,\type\:\string\},{\name\:\favorite_number\,\type\:[\int\,\null\]},{\name\:\favorite_color\,\type\:[\string\,\null\]}]}},
 body: {bytes: {BYTES}}}

 So now I am trying to write a serializer since flume only includes an
 FlumeEvent serializer creating avro files like the one above, not the
 original avro files on the servers.

 I am almost there, I got the schema from the header field and the bytes
 from the body field.
 But now I need to create write the AVRO file based on the bytes, not the
 values from the fields, I cannot do: r.put(field, value) since I
 don't have the values, just the bytes.

 This is the code:

 File file = TESTFILE;

 DatumReaderGenericRecord datumReader = new
 GenericDatumReaderGenericRecord();
 DataFileReaderGenericRecord dataFileReader = new
 DataFileReaderGenericRecord(file, datumReader);
 GenericRecord user = null;
 while (dataFileReader.hasNext()) {
 user = dataFileReader.next(user);

 Map headers = (Map) user.get(headers);

 Utf8 schemaHeaderKey = new Utf8(flume.avro.schema.literal);
 String schema = headers.get(schemaHeaderKey).toString();

 ByteBuffer body = (ByteBuffer) user.get(body);


 // Writing...
 Schema.Parser parser = new Schema.Parser();
 Schema schemaSimpleWrapper = parser.parse(schema);
 GenericRecord r =  new GenericData.Record(schemaSimpleWrapper);

 // NOT SURE WHAT COMES NEXT
 }

 Is possible to actually create the AVRO files from the value bytes?

 I appreciate any help.

 Thanks,
 Daniel



Re: Direct conversion from Generic Record to Specific Record

2014-02-06 Thread Doug Cutting
SpecificData#deepCopy will make this conversion.  It currently fails
for enums, but the fix is easy.  Here's a patch that makes that fix
and demonstrates a conversion.  If this change is of interest, please
file an issue in Jira.

Doug

Index: lang/java/avro/src/main/java/org/apache/avro/generic/GenericData.java
===
--- lang/java/avro/src/main/java/org/apache/avro/generic/GenericData.java
(revision 1564561)
+++ lang/java/avro/src/main/java/org/apache/avro/generic/GenericData.java
(working copy)
@@ -933,8 +933,7 @@
   case DOUBLE:
 return (T)new Double((Double)value);
   case ENUM:
-// Enums are immutable; shallow copy will suffice
-return value;
+return (T)createEnum(value.toString(), schema);
   case FIXED:
 return (T)createFixed(null, ((GenericFixed) value).bytes(), schema);
   case FLOAT:
Index: 
lang/java/ipc/src/test/java/org/apache/avro/specific/TestSpecificData.java
===
--- lang/java/ipc/src/test/java/org/apache/avro/specific/TestSpecificData.java
(revision 1564561)
+++ lang/java/ipc/src/test/java/org/apache/avro/specific/TestSpecificData.java
(working copy)
@@ -40,6 +40,8 @@
 import org.apache.avro.test.MD5;
 import org.apache.avro.test.Kind;

+import org.apache.avro.generic.GenericRecord;
+
 public class TestSpecificData {

   @Test
@@ -95,6 +97,16 @@
 new SpecificDatumReaderObject());
 }

+  @Test public void testConvertGenericToSpecific() {
+GenericRecord generic = new GenericData.Record(TestRecord.SCHEMA$);
+generic.put(name, foo);
+generic.put(kind, new GenericData.EnumSymbol(Kind.SCHEMA$, BAR));
+generic.put(hash, new GenericData.Fixed
+(MD5.SCHEMA$, new byte[]{0,1,2,3,4,5,6,7,8,9,0,1,2,3,4,5}));
+TestRecord specific =
+  (TestRecord)SpecificData.get().deepCopy(TestRecord.SCHEMA$, generic);
+  }
+
   @Test public void testGetClassSchema() throws Exception {
 Assert.assertEquals(TestRecord.getClassSchema(), TestRecord.SCHEMA$);
 Assert.assertEquals(MD5.getClassSchema(), MD5.SCHEMA$);

On Thu, Feb 6, 2014 at 8:12 AM, Christophe Taton
christophe.ta...@gmail.com wrote:
 I believe we could eventually make a generic function that can rewrite any
 record or record builder, whether specific or generic, into another record
 or record builder, whether specific or generic.

 Until this happens, Mika's suggestion is probably the best short-term
 solution.

 C.


 On Wed, Feb 5, 2014 at 2:59 PM, Roger Hoover roger.hoo...@gmail.com wrote:

 Hi Christophe,

 Thanks for your reply.  When you say that we could write a generic
 conversion function, do you mean we can write one that works for all
 schemas?  That would be great!

 I'd like an API something like this:

 MyCustomRecord record = MyCustomRecord.newBuilder(GenericRecord
 record).build()

 Thanks,

 Roger


 On Wed, Feb 5, 2014 at 10:59 AM, Christophe Taton
 christophe.ta...@gmail.com wrote:

 Hi Roger,
 With the change proposed in
 https://issues.apache.org/jira/browse/AVRO-1443, you would be able to create
 a specific record using a generic builder API. That means we could write a
 generic conversion function that creates specific records.
 Would that work for you?
 C.


 On Wed, Feb 5, 2014 at 10:23 AM, Roger Hoover roger.hoo...@gmail.com
 wrote:

 Hi,

 I'm working with an existing API that hands me a GenericRecord.  Is
 there an easy way I can covert it into a SpecificRecord.  Really I want to
 get it into the code-generated object.

 Thanks,

 Roger







Re: Facing issue while using avro-maven-plugin

2014-02-05 Thread Doug Cutting
The problem with deepCopy may be a mismatch between the version of
Avro you're using at compile time and the version you're using at
runtime.  When exactly are you getting this error?

Doug

On Wed, Feb 5, 2014 at 9:32 AM, Christophe Taton
christophe.ta...@gmail.com wrote:
 Hi,

 I do not know about 1), but 2) is currently expected : record fields are
 public by default but tagged as deprecated - you can solve this by
 explicitly asking for private fields in the Maven plugin configuration:
   plugin
 groupIdorg.apache.avro/groupId
 artifactIdavro-maven-plugin/artifactId
 configuration
   fieldVisibilityprivate/fieldVisibility
 /configuration
   /plugin

 C.


 On Wed, Feb 5, 2014 at 8:39 AM, AnilKumar B akumarb2...@gmail.com wrote:

 Hi,

 I have created below .avsc schema file and I tried creating corresponding
 classes by using avro tool and with plugin, but there are few errors on
 generated java code. What could be the issue?


 {namespace: test.avro,
  type: record,
  name: Session,
  fields: [
{name:VisitCommon, type: {
type: map, values:string}},
{name:events,
 type: {
 type: array,
 items:{
 name:Event,
 type:map,
 values:string}
 }
 }
  ]
 }


 1) Error: The method deepCopy(Schema,
 ListMapCharSequence,CharSequence) is undefined for the type GenericData
 2) And also observed there is some deprecated code.
  @Deprecated public
 java.util.Mapjava.lang.CharSequence,java.lang.CharSequence VisitCommon;

 I used eclipse plugin as mentioned below.
 http://avro.apache.org/docs/1.7.6/mr.html


 I tried with different versions of avro-maven-plugin, with 1.7.5, 1.7.6
 and with jdk1.7.0_45 version.

 I am unable to resolve it.

 Error message is as below:
 [ERROR] symbol:   method
 deepCopy(org.apache.avro.Schema,java.util.Mapjava.lang.CharSequence,java.lang.CharSequence)
 [ERROR] location: class org.apache.avro.generic.GenericData

 Thanks  Regards,
 B Anil Kumar.




Re: java.lang.ClassCastException: java.util.ArrayList$Itr cannot be cast to org.apache.avro.generic.IndexedRecord

2014-02-04 Thread Doug Cutting
It's hard to tell without more detail, but I strongly suspect this is
a Camel issue rather than a purely Avro issue.  It might thus better
be asked on the Camel user mailing list.

https://camel.apache.org/mailing-lists.html

Doug

On Tue, Feb 4, 2014 at 8:54 AM, Kostas Margaritis
kostas.margari...@inps.co.uk wrote:
 Hi all,

 after many experiments of using camel-avro, I decided to send an email to the 
 list as I'm getting ClassCastException errors, even with simple schemas.
 Java class from AVRO Schema file (avsc) generation works fine and the 
 resulting class builds and is included in the end project, but I can't really 
 make it work.
 Using Avro 1.7.5, and trying to send messages over jetty (using camel-avro 
 and camel-jetty resp), and using this schema:

 {   namespace :   serialisationtests.generated,
 type  :   record,
 name  :   TestPacket,
 fields:   [
 { name: firstname, type: string },
 { name: lastname,  type: string },
 { name: date, type : string },
 { name: features, type: {type: array, items: string}},
 { name: properties, type: {type: map, values: string}}
 ]
 }

 I then use camel-dataset to produce such objects and then pass them through 
 avro marshalling to a jetty listener.
 This is the exception I'm getting when trying to run mvn test:

 ---
  T E S T S
 ---
 Running serialisationtests.test.SerialisationTest
 [  main] SerialisationTest  INFO  
 
 [  main] SerialisationTest  INFO  
 Testing: 
 testReceivedDirectJettyMessages(serialisationtests.test.SerialisationTest)
 [  main] SerialisationTest  INFO  
 
 [  main] SpringCamelContext INFO  Apache 
 Camel 2.12.0.redhat-610336 (CamelContext: camel-1) is starting
 [  main] ManagedManagementStrategy  INFO  JMX is 
 enabled
 [  main] DefaultTypeConverter   INFO  Loaded 
 186 type converters
 [  main] set://myDataSet?produceDelay=0 INFO  
 Endpoint[dataset://myDataSet?produceDelay=0] expecting 6000 messages
 [  main] SpringCamelContext INFO  
 StreamCaching is not in use. If using streams then its recommended to enable 
 stream caching. See more details at 
 http://camel.apache.org/stream-caching.html
 [  main] SpringCamelContext INFO  Route: 
 route1 started and consuming from: 
 Endpoint[dataset://myDataSet?produceDelay=0]
 [  main] JettyHttpComponent WARN  You use 
 localhost interface! It means that no external connections will be available. 
 Don't you want to use 0.0.0.0 instead (all network interfaces)? 
 Endpoint[http://localhost:/myservice]
 [  main] JettyHttpComponent INFO  Using 
 default Jetty continuation timeout for: 
 Endpoint[http://localhost:/myservice]
 [  main] Server INFO  
 jetty-8.1.9.v20130131
 [  main] AbstractConnector  INFO  Started 
 SelectChannelConnector@localhost:
 [  main] SpringCamelContext INFO  Route: 
 route2 started and consuming from: Endpoint[http://localhost:/myservice]
 [  main] SpringCamelContext INFO  Total 2 
 routes, of which 2 is started.
 [  main] SpringCamelContext INFO  Apache 
 Camel 2.12.0.redhat-610336 (CamelContext: camel-1) started in 2.611 seconds
 [  main] MockEndpoint   INFO  
 Asserting: Endpoint[mock://result-jetty] is satisfied
 [hread #0 - dataset://myDataSet] DefaultErrorHandlerERROR Failed 
 delivery for (MessageId: ID-pippin-46409-1391532031804-0-1 on ExchangeId: 
 ID-pippin-46409-1391532031804-0-2). Exhausted after delivery attempt: 1 
 caught: java.lang.ClassCastException: java.util.ArrayList$Itr cannot be cast 
 to org.apache.avro.generic.IndexedRecord

 Message History
 ---
 RouteId  ProcessorId  Processor   
  Elapsed (ms)
 [route1] [route1] [dataset://myDataSet?produceDelay=0 
] [81]
 [route1] [marshal1  ] [marshal[ref:avro]  
] [69]

 Exchange
 

Re: Enum backward compatibility in distributed services...

2014-01-27 Thread Doug Cutting
You'd like the compile-time type-checking of specific, but the
run-time flexibility of generic, right?  Here's a way we might achieve
this.

Given the following schemas:

{type:enum, name:Color, symbols:[RED, GREEN, BLUE]}

{type:record, name:Shape, fields:[
  {name:xPosition, type:int},
  {name:yPosition, type:int},
  {name:color, type:Color},
  ]}

We might generate Java code like:

public class Shape extends GenericData.Record {
  public Shape(Schema schema) { super(schema); }
  public int getXPosition() { return (Number)get(xPosition); }
  public int getYPosition() { return (Number)get(yPosition); }
  public Color getColor { return (Color)get(color); }
}

public class Color extends GenericData.EnumSymbol {
  public Color(Schema schema, String label) {
super(schema, label);
  }
  public static final Color RED = new Color(RED);
  public static final Color GREEN = new Color(GREEN);
  public static final Color BLUE = new Color(BLUE);
}

If one reads data using the writer's schema into such classes, then
missing fields and enum symbols would be preserved in the generic
representation.  For example, you might have a filtering mapper that
removes all red shapes:

public void map(Shape shape, ...) {
  if (!shape.getColor().equals(Color.RED)) {
collect shape;
  }
}

This would still function correctly without recompilation even if the
schema of the input data is very different, e.g., missing xPosition
and yPosition, containing a new color, PURPLE or a new field,
region, etc.

I think Christophe Taton once requested something like this, to permit
one to preserve fields not in the schema used to generate the code
that's reading.  An interesting variation would read things using a
union of the writer's schema and the schema used for code generation,
so that missing fields are given default values.

The actual implementation should probably generate interfaces that
extend the GenericRecord and GenericEnumSymbol interfaces, with
private concrete implementations like the above, and a builder.  This
would permit greater flexibility and optimizations.  One could, e.g.,
when a builder is created, generate, compile and load optimized record
implementations so that little performance penalty is paid.

The end result would be that compiled code would reference interfaces
that don't correspond exactly to the runtime data, but rather provide
a view on that data.  We might not alter specific, but instead add a
new FlexData, FlexDatumReader, etc., that builds on generic.

Thoughts?

Doug


On Sun, Jan 26, 2014 at 2:31 AM, Amihay Zer-Kavod amih...@gmail.com wrote:
 Hi,
 We are using Avro heavily for schema definition of all of the events sent
 through our distributed system.
 The system is a multi service, java based, SaaS system, where the services
 upgraded a lot and in no particular order.
 We are using Enums in some events data and from time to time a new Enum
 value is added.
 In this case we started having problems.
 A producer produce an event with the new enum value, A consumer using old
 schema tries to read the event using java SpecificDatumReader will
 completely fail to read the event .
 These events will not be handled by the consumer until it is upgraded to use
 the new schema generated code.

 Problem is Avro code generation creates a real java enum, and there is no
 way to initialize or represent an unknown enum value in a java enum.
 However in many cases the consumer could still be doing most of its logic
 with the event with unknown enum value.

 Handling enums in Avro is a powerful tool, specificDatumReader is a powerful
 tool, it looks like I'd have to give up usage of one of them!

 Is there any plan/way to handle enums differently in the code generation?
 Any other ideas I can fix this issue with?
 I believe AVRO-1340 reference the same problem, any plans on doing it?
 I would go a step further and allow dynamic access to the original value,
 not just a default value in case enum value is unknown.

 10x
 Amihay





Re: [ANNOUNCE] Avro release 1.7.6

2014-01-27 Thread Doug Cutting
On Sat, Jan 25, 2014 at 8:58 PM, Christophe Taton
christophe.ta...@gmail.com wrote:
 Is it also possible to push the Python3 version to PyPi?

I just did that.

https://pypi.python.org/pypi/avro-python3/1.7.6

I renamed it avro-python3.  Does that seem reasonable?  It seems like
one of several conventions used.

Doug


Re: Schema exclusion from Avro message

2014-01-27 Thread Doug Cutting
If you're using Avro's RPC mechanism, schemas are only sent when the
client and server do not already have each other's schema.  Each
client request is preceded by a hash of the clients schema and the
schema it thinks the server is using.  If the server already has the
client's schema, and the client already has the server's, then the
server can directly respond.  If they do not have the other's schema
then schemas are transmitted and cached.  This way the server's schema
is only transmitted for the first request from a given client, and the
client's schema is only transmitted to the server the first time a
client with that schema connects.

Avro Python does support RPC.

If you're not using Avro RPC but some other messaging mechanism, then
AVRO-1124 as you mention might be useful, but it also has not yet been
released.

If you're storing Avro data in a file, then the Schema is included in
the file, as you mention.

Doug

On Mon, Jan 27, 2014 at 11:00 AM, Wai Yip Tung w...@tungwaiyip.info wrote:
 I found Deepesh's question back in December. I have joined the mailing list
 later. So don't have the message in my inbox and I do not know the proper
 way to reply. Anyway I have include the original message below.

 I have the similar issue. In addition I'm interested to find out about
 Python and Node js library support.

 From what I understand, the avro specification requires avro.schema. So I am
 quite unsure of the status of have the schema in an external repository.

 -  avro.schema contains the schema of objects stored in the file, as JSON
 data (required).

 http://avro.apache.org/docs/1.7.6/spec.html#Object+Container+Files

 Wai Yip


 FromDeepesh Malviya deep0...@gmail.com
 SubjectSchema exclusion from Avro message
 DateSun, 15 Dec 2013 12:58:18 GMT
 Hi,

 I have read at multiple places that we can exclude the schema being packed
 into the Avro message  can only include version to allow schema lookup. I
 have also looked into
 https://issues.apache.org/jira/browse/AVRO-1124however, didn't found
 how to make use of such repository while reading or
 writing Avro messages.

 I just need some heads up related to that to get started. My use-case is
 of
 sending Avro message from a C-based avro client to Flume/Kafka  finally
 storing it to Hadoop.

 --
 _Deepesh









[ANNOUNCE] Avro release 1.7.6

2014-01-24 Thread Doug Cutting
I'd like to announce the availability of Avro release 1.7.6.

Changes are listed at:

  http://s.apache.org/avro176

This release can be downloaded from:

  https://www.apache.org/dyn/closer.cgi/avro/

Java jar files are available from Maven Central.
Ruby artifacts are at RubyGems.
Python is at PyPi.

Thanks to everyone for contributing!

Doug


Re: upgrading to Avro 1.7.5

2014-01-23 Thread Doug Cutting
As others have noted, the data format has not changed and should be compatible.

The performance difference you note is curious.  I wonder if this
could be related to the following issue?

   https://issues.apache.org/jira/browse/AVRO-1348

You could try the 1.7.6 release candidate to test this:

  http://people.apache.org/~cutting/avro-1.7.6-rc0/

The 1.7.6-rc0 Maven staging repository is at:

  https://repository.apache.org/content/repositories/orgapacheavro-1000/

Doug

On Wed, Jan 22, 2014 at 8:29 AM, Jun Rao jun...@gmail.com wrote:
 Hi,

 We are on Avro 1.4.0 and plan to upgrade to Avro 1.7.5. Is the binary
 encoding 100% compatible btw the 2 versions? Also, I did some perf testing
 and it seems that on average, 1.7.5 is about 6% faster on encoding and 20%
 slower on decoding compared with 1.4.0. Is the degradation on decoding
 performance expected (though the absolute decoding time is much smaller,
 typically 1/3 of the encoding time)? Finally, are there enough critical bug
 fixes/improvements to motivate such an upgrade?

 Thanks,

 Jun


Re: Avro C++ RPC support

2014-01-23 Thread Doug Cutting
On Thu, Jan 16, 2014 at 7:34 AM, Sanket Satyaki sanketsaty...@gmail.com wrote:
 I wanted to check, if there is any ongoing request for providing the RPC
 support in C++?

There was some work on this a few years ago, but it was never completed.

https://issues.apache.org/jira/browse/AVRO-484

Perhaps this could serve as a starting point?

 If RPC is missing in c++, is there any technical hurdle (I mean any specific
 reason related to C++ language) ?

I know of no such technical hurdles.

Cheers,

Doug


Re: Nullable Fields

2014-01-21 Thread Doug Cutting
On Mon, Jan 20, 2014 at 3:50 AM, Alparslan Avcı
alparslan.a...@agmlab.com wrote:
 {name: field1, type: [null, {type:map, values:[null,
 string]}],default:null}

 can be represented as like

 {name: field1, type: {type:map, values:string,
 nullable:true}, nullable:true, default:null}

You'd like an alternate syntax for expressing nullability, is that
right?  The semantics would be the same, a value could either be of
the named type or it could be null, right?

The original design was done this way because, once we had unions and
a null type, it seemed better to compose these two concepts than to
invent another equivalent concept.  (Would the above two schemas be
equal?  Probably not.  Would they express exactly the same datatypes?
Probably.)

See also this talk (made after Avro was designed):

  
http://www.infoq.com/presentations/Null-References-The-Billion-Dollar-Mistake-Tony-Hoare

But, whether we like it or not, that design decision is now past.  We
cannot easily make such a change in the JSON schema language since it
would create compatibility problems.  For example, a data file
generated using the new syntax could not be read by code that had not
yet been upgraded to process this syntax.  Existing code would not
know that nullable is a special property and would fail in strange
ways, since it would expect to find items of the specified type, not
union values.

We can devise higher-level schema languages that are translated into
Avro schemas and that handle nullability differently.  So you could
define a pre-processor for JSON schemas that converted them from the
above syntax into Avro's standard syntax, and use this when
interacting with Avro.

 Null types in unions will always cause an exceptional situation since they
 are not 'types' at all.

I don't understand this statement.

Doug


Re: Avro Array and combining schema ?

2014-01-21 Thread Doug Cutting
It sounds like you'd like to reference schemas in other files?  If so,
you can do this with Maven using the import configuration, added in
https://issues.apache.org/jira/browse/AVRO-1188.  Also the
command-line compiler will process files in order, with types defined
earlier on the command line available in those defined later.  Avro
IDL also has an import command that permits inclusion of types from
.avsc or IDL files.

Doug

On Tue, Jan 21, 2014 at 6:32 AM, ÐΞ€ρ@Ҝ (๏̯͡๏) deepuj...@gmail.com wrote:
 I have two schemas as follow

 1) Event.avsc

 {

 type: record, namespace:com.nrt.datum.avro, name : Event,

 fields: [

 {name : guid, type: [string, null]},

 {name : moreId, type: [int, null]}

]

 }


 2. Session.avsc

 {

 type: record, namespace:com.nrt.datum.avro, name : Session,

 fields: [

  {name : isClosed, type: boolean},

  {name : guid, type: [string, null]}

 ]

 }



 1) I want to define a third schema that should SessionContainer, that should
 have all the fields of schema and an array (List) of Event. How do i write
 SessionContainer.avsc  ?
 2) If i could refer Event.avsc and Session.avsc in SessionContainer.avsc
 then if i change Event/Session, i do not have to change SessionContainer.

 I use Java + Avro.
 Any suggestions ?

 --
 Deepak



Re: Handling field names when serializing and deserializing JSON

2014-01-14 Thread Doug Cutting
On Tue, Jan 14, 2014 at 2:06 PM, Pritchard, Charles X. -ND
charles.x.pritchard@disney.com wrote:
 Do I just pop the “original” field name in as an alias and use the “safe”
 (alphanumeric+underscore) one as the primary name?

I'm not exactly sure what you're trying to do.  Aliases in the
reader's schema are matched to names in the writer's schema.  So if
you have Avro data with names that are illegal in Hive then you could
provide Hive with a safely-named schema to use when reading these that
has the original name as an alias.  Conversely, if Hive writes data
with the safe name that you want to read as data using the original
name, then you'd read with the safe name in an alias.  Does that make
sense?

Doug


Re: Handling field names when serializing and deserializing JSON

2014-01-14 Thread Doug Cutting
It might work, I'd have to test it to be sure.  But it's not guaranteed.

Avro names are specified at:

  http://avro.apache.org/docs/current/spec.html#Names

Avro Java accepts more than this, including arbitrary unicode
alphabetic characters.

See https://issues.apache.org/jira/browse/AVRO-1022 for an extensive discussion.

Doug

On Tue, Jan 14, 2014 at 2:45 PM, Pritchard, Charles X. -ND
charles.x.pritchard@disney.com wrote:

 On Jan 14, 2014, at 2:32 PM, Doug Cutting cutt...@apache.org wrote:

 On Tue, Jan 14, 2014 at 2:06 PM, Pritchard, Charles X. -ND
 charles.x.pritchard@disney.com wrote:
 Do I just pop the “original” field name in as an alias and use the “safe”
 (alphanumeric+underscore) one as the primary name?

 you have Avro data with names that are illegal in Hive then you could
 provide Hive with a safely-named schema to use when reading these that
 has the original name as an alias.  Conversely, if Hive writes data
 with the safe name that you want to read as data using the original
 name, then you'd read with the safe name in an alias.  Does that make
 sense?

 Yes; so we just flop the alias/original name between Hive and other sources.
 Really appreciate the clarification there.

 One of the common places this comes up is with hyphens such as: “X-Something” 
 in some JSON schemas.

 Thanks for letting me know how to handle this.

 On that same topic though — if/when someone does something really awful, like 
 using a dot in the key name,
 is that still going to work out fine with record.get() syntax?

 e.g.: { “key”: “val”, “dotted.key”: “val” }

 I know that in the context of avro aliases, the dot has special semantics.

 (I hope I’m not being too obtuse).

 -Charles


Re: Avro + SSL ?

2014-01-10 Thread Doug Cutting
On Thu, Jan 9, 2014 at 10:02 PM, Sid Shetye sid...@outlook.com wrote:
 Does anyone know how to use Avro RPC along with SSL?

The current C# implementation only has a SocketTransceiver
implementation, which does not support secure connections.  If you
want to interoperate with Java and other implementations, then we
should probably implement an HttpTransceiver for C# and use HTTPS.
This should be straightforward to implement using C#'s standard
HttpWebRequest.

Doug


Re: Avro Read with sync() {java.io.IOException: Invalid sync}

2013-12-23 Thread Doug Cutting
This sounds like a bug.

I wonder if it is similar to a related bug in Hadoop?

https://issues.apache.org/jira/browse/HADOOP-9307

If so, please file an issue in Jira.

Doug

On Sat, Dec 21, 2013 at 4:35 AM, ÐΞ€ρ@Ҝ (๏̯͡๏) deepuj...@gmail.com wrote:
 Hello,
 I have a 340 MB avro data file that contains records sorted and identified
 by unique id (duplicate records exists). At the beginning of every unique
 record a synchronization point is created with DataFileWriter.sync(). (I
 cannot or do not want to save the sync points and i do not want to use
 SortedKeyValueFile as output format for M/R job)

 There are at-least 25k synchronization points in a 340 MB file.

 Ex:
 Marker1_RecordA1_RecordA2_RecordA3_Marker2_RecordB1_RecordB2


 As records are sorted, for efficient retrieval, binary search is performed
 using the attached code.

 Most of the times the search is successful, at times the code throws the
 following exception
 --
 org.apache.avro.AvroRuntimeException: java.io.IOException: Invalid sync! at
 org.apache.avro.file.DataFileStream.hasNext(DataFileStream.java:210
 --



 Questions
 1) Is it ok to have 25k sycn points for 300 MB file ? Does it cost in
 performance while reading ?
 2) I note down the position that was used to invoke fileReader.sync(mid);.
 If i catch AvroRuntimeException, close and open the file and sync(mid) i do
 not see exception. Why should Avro throw exception before and not later ?
 3) Is there a limit on number of times sync() is invoked ?
 4) When sync(position) is invoked, are any 0 = position = file.size()
 valid ? If yes why do i see AvroRuntimeException (#2) ?

 Regards,
 Deepak



Re: Effort towards Avro 2.0?

2013-12-03 Thread Doug Cutting
On Mon, Dec 2, 2013 at 1:56 PM, Philip Zeyliger phi...@cloudera.com wrote:
 It sounds like you're proposing to break language API compatibility.  Are
 you also proposing to break wire compatibility for Avro HTTP RPC, Avro Netty
 RPC, and/or Avro datafiles?

We should be able to provide back-compatibility.  When current APIs
cannot be back-compatibly extended, new APIs can be added.  Old APIs
may be deprecated but should be retained for a while.  Data files
written by 1.x should be readable by 2.x.

Forward compatibility may not be possible when new schema features are
used.  Data files written in 2.x may not be readable by 1.x.  Perhaps
we could add a mode that forces 2.x to write a 1.x format file.

RPC interoperability requires that 2.x be able to both read and write
1.x format.  So long as a 1.x protocol is used, then 1.x and 2.x
clients and servers might be able to interoperate using 1.x wire
formats.  But when 2.x schema features are used this may not be
possible.

Perhaps we should proceed by making back-compatibility (ability to
read 1.x) a requirement, then adding interoperabilty features (ability
to write 1.x) as needed?

Should we require that all new schema features (named unions,
extensions, date primitive, etc,) have a lossless translation to a 1.x
schema?

Doug


Re: Correct avsc definition for array of external object's

2013-11-08 Thread Doug Cutting
On Fri, Nov 8, 2013 at 12:19 PM, Lewis John Mcgibbney
lewis.mcgibb...@gmail.com wrote:
 The question I am asking is whether I should embed the ExtractorSpec as a
 nested record? Or is there another way such as importing?

Nesting would certainly work, but may make things harder to maintain
if you reference the type in several different .avsc files.

Avro's schema parser and compiler supports pre-defining types.  E.g.,
with Avro's command-line .avsc to .java compiler, types in files
defined earlier on the command line can be referenced by types in
files later on the command line.  Similarly, the Java APIs and Maven
integrations support this.  But, from what I can tell, the Gora
compiler does not currently support this (although it would not be
hard to add).

Avro IDL does support imports, so you might use Avro's IDL compiler to
convert .avdl files to .avsc files that expand nested types, then use
the Gora compiler to compile these.

Doug


Re: GenericDatumReader and datum reuse

2013-10-31 Thread Doug Cutting
A simple approach to reuse is to pass the value previously returned from read():

GenericRecord record = null;
while (...) {
  record = reader.read(record, decoder);
  ... code that does not retain a pointer to record ...
}

Doug

On Wed, Oct 30, 2013 at 3:07 PM, kulkarni.swar...@gmail.com
kulkarni.swar...@gmail.com wrote:
 Hello,

 While going thorough the API for GenericDatumReader, I came across this
 read[1] method and it wasn't immediately clear to me as to what the reuse
 of the datum really means.

 So for instance if I have an evolving schema, should this datum be created
 from the readerSchema or the writerSchema. So something like:

 record = new GenericData.Record(readerSchema); // created with readerSchema

 GenericDatumReaderGenericRecord gdr = new
 GenericDatumReaderGenericRecord(writerSchema, readerSchema);

 record = gdr.read(record, binaryDecoder);

 Or is it better to simply leave it off as null?

 record = gdr.read(null, binaryDecoder);


 Thanks,


 [1]
 http://avro.apache.org/docs/1.7.4/api/java/org/apache/avro/generic/GenericDatumReader.html#read(D,
 org.apache.avro.io.Decoder)


Re: Partial lookup without full deserialization

2013-10-31 Thread Doug Cutting
You can specify a reader schema of simply {a:int}.  Avro will
efficiently skip missing fields when parsing values.  Note that you
still need the original, full schema (the writer schema).  This is
achieved through the schema resolution rules.

http://avro.apache.org/docs/current/spec.html#Schema+Resolution

Doug

On Thu, Oct 31, 2013 at 5:20 PM, Arvind Kalyan bas...@gmail.com wrote:
 Folks, say I serialize a GenericData.Record with some schema {a: int, b:
 string, c: array[int]} into a byte[] and send it over the wire.

 On the receiving side, once I have this byte[] is it possible for me to
 lookup just the field 'a' without incurring the cost of deserializing all
 the fields?

 Any other thoughts around trying to optimize partial lookups?

 thanks
 --
 Arvind Kalyan
 http://www.linkedin.com/in/base16



Re: Setting field default value's programmatically

2013-10-27 Thread Doug Cutting
On Sat, Oct 26, 2013 at 3:41 AM, Lewis John Mcgibbney
lewis.mcgibb...@gmail.com wrote:
 2) Why org.apache.avro.data.RecordBuilderBase#defaultValue(Field field) is
 not returning null if no default is specified in the schema... which is the
 case?

It returns null when null is the default value, but throws an error
when no default value is specified.  This is required for correct
behavior, since null values are invalid for many types.  That said,
one could define an alternate method that returns a type-specific
default value when none is specified.  Alternately, one could traverse
schemas and populate fields that have no specified default with a
type-specific default.  I haven't looked at the Gora code in question,
but perhaps there's an opportunity to pre-process the schemas there,
prior to Avro code generation?

Cheers,

Doug


Re: expect specific record but get generic

2013-10-21 Thread Doug Cutting
If the generated classes are not on the classpath then the generic
representation is used.  So, yes, this sounds like a classpath
problem.

On Mon, Oct 21, 2013 at 8:41 AM, Koert Kuipers ko...@tresata.com wrote:
 i am observing that on a particular system (spark) my code breaks in that
 avro does not return the specific record i expected but instead returns
 generic records.


 i suspect this is some class loading issue on the distributed system
 (something about how the classpath is constructed for the spark slaves).

 anyone had class loading issues get in the way of avro specific?


Re: expect specific record but get generic

2013-10-21 Thread Doug Cutting
On Mon, Oct 21, 2013 at 1:19 PM, Koert Kuipers ko...@tresata.com wrote:
 doug, could it be a classloader (instead of classpath) issue? looking at
 spark it seems to run the tasks inside the slaves/workers with a custom
 classloader.

Yes, it could be a classloader issue.  Perhaps you need to pass the
Spark task's ClassLoader to SpecificData?  Find where your
SpecificDatumReader is constructed, and pass it a SpecificData that
was constructed with that ClassLoader.

http://avro.apache.org/docs/current/api/java/org/apache/avro/specific/SpecificData.html#SpecificData(java.lang.ClassLoader)
http://avro.apache.org/docs/current/api/java/org/apache/avro/specific/SpecificDatumReader.html#SpecificDatumReader(org.apache.avro.specific.SpecificData)

Doug


Re: default values

2013-10-20 Thread Doug Cutting
Note that builders do supply defaults.

http://avro.apache.org/docs/current/api/java/org/apache/avro/data/RecordBuilder.html

But that might not help you much here.

You might intersect your config json with the reader schema to determine
its implied writer schema.

Doug
On Oct 13, 2013 9:21 PM, Koert Kuipers ko...@tresata.com wrote:

 i am testing parsing a json string using a record class defined in an
 .avdl file.

 i was hoping that if a field in the record has a default value that it is
 not mandatory to provide it in the json. however i get an exception:

 org.apache.avro.AvroTypeException: Expected field name not found: test123

 i guess it was silly of me to expect this?
 are default values only used in schema resolution (so reader vs writer
 schema)? and in this case there is no writer schema, just json data i
 created and a reader schema, so the default values will not help me?

 too bad because this seemed like a really quick and convenient way to
 create configuration objects that can be instantiated from json, but
 without optional fields with default values it loses a lot of it power.




Re: avrogencpp and multiple .avsc

2013-10-15 Thread Doug Cutting
On Tue, Oct 15, 2013 at 3:21 PM, William McKenzie
wsmck...@cartewright.com wrote:
 Is it possible to generate code for multiple schemas at one time, and
 resolve references between them?

The command line 'compile' tool and maven task both support this.  One
can pass multiple schema files on the command line, and each can
reference schemas that precede it on the command line.

With Maven, you can specify imports.  See
https://issues.apache.org/jira/browse/AVRO-1188 for details.

 I tried using an avpr file, and it did not seem to like that.

With a .avpr file you can place multiple schemas in its types
section, leaving the messages section empty.

With an .avdl file you can import other .avsc, .avpr and .avdl files.

http://avro.apache.org/docs/current/idl.html#imports

Doug


Re: GenericRecord and passivity

2013-10-15 Thread Doug Cutting
GenericRecord should work well in this context.  Can you provide a
complete example that fails?

Doug

On Tue, Oct 15, 2013 at 3:43 PM, kulkarni.swar...@gmail.com
kulkarni.swar...@gmail.com wrote:
 Do we know if a GenericRecord is robust to schema evolution? I am currently
 seeing cases where I get an exception like the following if I try to
 deserialize an older record with a newer schema.

 Exception in thread main java.lang.ArrayIndexOutOfBoundsException: 8

 at org.apache.avro.io.parsing.Symbol$Alternative.getSymbol(Symbol.java:364)

 at org.apache.avro.io.ResolvingDecoder.doAction(ResolvingDecoder.java:229)

 at org.apache.avro.io.parsing.Parser.advance(Parser.java:88)

 at org.apache.avro.io.ResolvingDecoder.readIndex(ResolvingDecoder.java:206)

 at
 org.apache.avro.generic.GenericDatumReader.read(GenericDatumReader.java:152)

 at
 org.apache.avro.generic.GenericDatumReader.readRecord(GenericDatumReader.java:177)

 at
 org.apache.avro.generic.GenericDatumReader.read(GenericDatumReader.java:148)

 at
 org.apache.avro.generic.GenericDatumReader.read(GenericDatumReader.java:139)


 The newer schema just has a few more fields added to it and no names from
 the older schema were changed.

 Is this a known issue? Should a SpecificRecord be always considered when
 passivity is important?


 Thanks,


 --
 Swarnim


Re: GenericRecord and passivity

2013-10-15 Thread Doug Cutting
On Tue, Oct 15, 2013 at 4:49 PM, Eric Wasserman ewasser...@247-inc.com wrote:
 Change this line:
 DatumReaderGenericRecord reader = new
 GenericDatumReaderGenericRecord(schema_11);

 to this:
 DatumReaderGenericRecord reader = new
 GenericDatumReaderGenericRecord(schema_10,  schema_11);

Yes, that is the fix.

Or, better yet, use DataFileWriter  DataFileReader, which handle this for you.

Doug


Re: Unable to compile a namespace-less schema

2013-10-11 Thread Doug Cutting
Thanks for the complete example.  I can get this to compile by
removing the package com.company from Main.java.

The Java specification says, It is a compile time error to import a
type from the unnamed package.  So you'll only be able to use these
classes from other classes in the unnamed package, which isn't very
useful.

You can add a namespace to your .avsc file, and alias it to the
non-namespaced version so that it still matches, e,g:

  ..., name:com.company.TUPLE_20, alias:.TUPLE_20, ...

That gets things to compile for me without removing the package in
your Main.java.

Doug

On Fri, Oct 11, 2013 at 1:59 PM, Vitaly Gordon vita...@gmail.com wrote:
 Hi Doug,
 I've attached a maven project that contains code that shows the problem.
 The code is basically the same one from the Avro guide, but what is
 important to observe is that since the Main class has a package define, it
 cannot import the classes generated from the namespace-less schema.

 just run mvn:compile to get the compilation errors

 Thanks,
 Vitaly


 On Thu, Oct 10, 2013 at 1:58 PM, Doug Cutting cutt...@apache.org wrote:

 I encourage you to please provide a complete test, code that fails.
 If maven is involved, create a simple, Maven project that illustrates
 the problem.

 Thanks,

 Doug

 On Wed, Oct 9, 2013 at 11:21 PM, Vitaly Gordon vita...@gmail.com wrote:
  Hi Doug,
  You are right, the code does compile with javac. Apparently it is some
  maven
  error, where it doesn't like to compile package-less files.
 
  Having said that, I still have the issue of not being able to use these
  java
  files in my code, because there is no way to import them. One thing I
  tried
  that sometimes work is adding some arbitrary namespace to the avro
  schema.
  However, when I try to read records using the new (with namespace)
  schema, I
  get in return a generic record instead of a specific one. This behavior
  can
  be observed in the same file I attached by adding an arbitrary namespace
  to
  the schema before generating the Java classes from it.
 
  Is there any way to read specific records when the schema that was used
  to
  write them contains no namespace?
 
  Thanks,
  Vitaly
 
 
  On Wed, Oct 9, 2013 at 6:07 PM, Doug Cutting cutt...@apache.org wrote:
 
  Using the current trunk of Avro I am able to:
   - extract the schema from the data file you provided (using
  avro-tools schema command)
   - generate Java classes for this schema (using the avro-tools compile
  command)
   - compile these generated Java classes (using the javac command)
 
  Can you provide a complete case of what fails for you?
 
  Thanks,
 
  Doug
 
  On Wed, Oct 9, 2013 at 4:56 PM, Vitaly Gordon vita...@gmail.com
  wrote:
   Does anyone else might have an idea how I can resolve this
   namespace-less
   Avro schema code generation?
  
   Thanks,
   Vitaly
  
  
   On Mon, Oct 7, 2013 at 2:04 PM, Vitaly Gordon vita...@gmail.com
   wrote:
  
   Hi Sean,
   Here is a file that contains a single record that I cannot read
   using a
   specific reader.
  
   It's hard for me to add code because the problem is a compilation
   problem
   with the generated Java files.
  
   So to recreate the problem:
   1. Extract the schema from the record
   2. Generate the code from the schema
   3. Compile
  
   Is there another way that I can describe the issue?
  
  
  
   On Mon, Oct 7, 2013 at 10:58 AM, Sean Busbey bus...@cloudera.com
   wrote:
  
   Hi Vitay!
  
   Can you give us a minimal schema and test program that illustrates
   the
   problem you're describing?
  
   --
   Sean
  
   On Oct 7, 2013 12:27 PM, Vitaly Gordon vita...@gmail.com wrote:
  
   Hi All,
   I am trying to read Avro data that its schema does not have a
   namespace.
   The problem is that I cannot compile the classes, because the
   generated Java
   code does not have a package. On the other hand, if I do add some
   arbitrary
   namespace to the schema, the record is resolved as a generic one,
   which then
   fails on ClassCastException to the specific record.
  
   Any ideas on how I can resolve this issue?
  
   Thanks,
   Vitay
  
  
  
 
 




Re: Unable to compile a namespace-less schema

2013-10-10 Thread Doug Cutting
I encourage you to please provide a complete test, code that fails.
If maven is involved, create a simple, Maven project that illustrates
the problem.

Thanks,

Doug

On Wed, Oct 9, 2013 at 11:21 PM, Vitaly Gordon vita...@gmail.com wrote:
 Hi Doug,
 You are right, the code does compile with javac. Apparently it is some maven
 error, where it doesn't like to compile package-less files.

 Having said that, I still have the issue of not being able to use these java
 files in my code, because there is no way to import them. One thing I tried
 that sometimes work is adding some arbitrary namespace to the avro schema.
 However, when I try to read records using the new (with namespace) schema, I
 get in return a generic record instead of a specific one. This behavior can
 be observed in the same file I attached by adding an arbitrary namespace to
 the schema before generating the Java classes from it.

 Is there any way to read specific records when the schema that was used to
 write them contains no namespace?

 Thanks,
 Vitaly


 On Wed, Oct 9, 2013 at 6:07 PM, Doug Cutting cutt...@apache.org wrote:

 Using the current trunk of Avro I am able to:
  - extract the schema from the data file you provided (using
 avro-tools schema command)
  - generate Java classes for this schema (using the avro-tools compile
 command)
  - compile these generated Java classes (using the javac command)

 Can you provide a complete case of what fails for you?

 Thanks,

 Doug

 On Wed, Oct 9, 2013 at 4:56 PM, Vitaly Gordon vita...@gmail.com wrote:
  Does anyone else might have an idea how I can resolve this
  namespace-less
  Avro schema code generation?
 
  Thanks,
  Vitaly
 
 
  On Mon, Oct 7, 2013 at 2:04 PM, Vitaly Gordon vita...@gmail.com wrote:
 
  Hi Sean,
  Here is a file that contains a single record that I cannot read using a
  specific reader.
 
  It's hard for me to add code because the problem is a compilation
  problem
  with the generated Java files.
 
  So to recreate the problem:
  1. Extract the schema from the record
  2. Generate the code from the schema
  3. Compile
 
  Is there another way that I can describe the issue?
 
 
 
  On Mon, Oct 7, 2013 at 10:58 AM, Sean Busbey bus...@cloudera.com
  wrote:
 
  Hi Vitay!
 
  Can you give us a minimal schema and test program that illustrates the
  problem you're describing?
 
  --
  Sean
 
  On Oct 7, 2013 12:27 PM, Vitaly Gordon vita...@gmail.com wrote:
 
  Hi All,
  I am trying to read Avro data that its schema does not have a
  namespace.
  The problem is that I cannot compile the classes, because the
  generated Java
  code does not have a package. On the other hand, if I do add some
  arbitrary
  namespace to the schema, the record is resolved as a generic one,
  which then
  fails on ClassCastException to the specific record.
 
  Any ideas on how I can resolve this issue?
 
  Thanks,
  Vitay
 
 
 




Re: footer info in avro

2013-10-09 Thread Doug Cutting
There's no plan I know of to add this.

Avro's original file format wrote metadata at the end of the file.
This was changed in Avro 1.3 so that files could always be processed
sequentially, without seeking to the end.

Doug

On Wed, Oct 9, 2013 at 5:08 PM, Venkat vramac...@ymail.com wrote:
 Hi All

 Does Avro file format support storing some json metadata at the end of the
 file (footer)? Currently, the header stores schema in json enabling
 consumers to augment it with application specific info. But I'm not sure if
 there is a way consumer can store specific metadata at the end of the file.

 The use case is that data processing applications can compute statistics
 during the scan and write the computed statistics at the end of the file
 before closing the stream. This avoid storing statistics else where.

 If this feature does not exits, is there a plan on adding it sometime in the
 future?

 Thanks a lot.


Re: Unable to compile a namespace-less schema

2013-10-09 Thread Doug Cutting
Using the current trunk of Avro I am able to:
 - extract the schema from the data file you provided (using
avro-tools schema command)
 - generate Java classes for this schema (using the avro-tools compile command)
 - compile these generated Java classes (using the javac command)

Can you provide a complete case of what fails for you?

Thanks,

Doug

On Wed, Oct 9, 2013 at 4:56 PM, Vitaly Gordon vita...@gmail.com wrote:
 Does anyone else might have an idea how I can resolve this namespace-less
 Avro schema code generation?

 Thanks,
 Vitaly


 On Mon, Oct 7, 2013 at 2:04 PM, Vitaly Gordon vita...@gmail.com wrote:

 Hi Sean,
 Here is a file that contains a single record that I cannot read using a
 specific reader.

 It's hard for me to add code because the problem is a compilation problem
 with the generated Java files.

 So to recreate the problem:
 1. Extract the schema from the record
 2. Generate the code from the schema
 3. Compile

 Is there another way that I can describe the issue?



 On Mon, Oct 7, 2013 at 10:58 AM, Sean Busbey bus...@cloudera.com wrote:

 Hi Vitay!

 Can you give us a minimal schema and test program that illustrates the
 problem you're describing?

 --
 Sean

 On Oct 7, 2013 12:27 PM, Vitaly Gordon vita...@gmail.com wrote:

 Hi All,
 I am trying to read Avro data that its schema does not have a namespace.
 The problem is that I cannot compile the classes, because the generated 
 Java
 code does not have a package. On the other hand, if I do add some arbitrary
 namespace to the schema, the record is resolved as a generic one, which 
 then
 fails on ClassCastException to the specific record.

 Any ideas on how I can resolve this issue?

 Thanks,
 Vitay





  1   2   3   4   >