[jira] [Updated] (AVRO-1810) GenericDatumWriter broken with Enum

2016-03-10 Thread Sean Busbey (JIRA)

 [ 
https://issues.apache.org/jira/browse/AVRO-1810?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Sean Busbey updated AVRO-1810:
--
Fix Version/s: 1.8.1

> GenericDatumWriter broken with Enum
> ---
>
> Key: AVRO-1810
> URL: https://issues.apache.org/jira/browse/AVRO-1810
> Project: Avro
>  Issue Type: Bug
>  Components: java
>Affects Versions: 1.8.0
>Reporter: Ryon Day
>Priority: Blocker
> Fix For: 1.8.1
>
>
> {panel:title=Description|titleBGColor=#3FA|bgColor=#DDD}
> Using the GenericDatumWriter with either Generic OR SpecificRecord will break 
> if an Enum is present.
> {panel}
> {panel:title=Steps To Reproduce|titleBGColor=#8DB|bgColor=#DDD}
> I have been tracking Avro decoding oddities for a while.
> The tests for this issue can be found 
> [here|https://github.com/ryonday/avroDecodingHelp/blob/master/src/test/java/com/ryonday/test/Avro180EnumFail.java]
> {panel}
> {panel:title=Notes|titleBGColor=#3AF|bgColor=#DDD}
> Due to the debacle that is the Avro "UTF8" object, we have been avoiding it 
> by using the following scheme:
> * Write incoming records to a byte array using the GenericDatumWriter
> * Read back the byte array to our compiled Java domain objects using a 
> SpecificDatumWriter
> This worked great with Avro 1.7.7, and this is a binary-incompatable breaking 
> change with 1.8.0.
> This would appear to be caused by an addition in the 
> {{GenericDatumWriter:163-164}}:
> {code}
>   if (!data.isEnum(datum))
>   throw new AvroTypeException("Not an enum: "+datum);
> {code}
> {panel}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (AVRO-1810) GenericDatumWriter broken with Enum

2016-03-10 Thread Sean Busbey (JIRA)

[ 
https://issues.apache.org/jira/browse/AVRO-1810?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15190164#comment-15190164
 ] 

Sean Busbey commented on AVRO-1810:
---

This looks like an impedance mismatch between the Generic and Specific APIs. 
The Specific API generates bare java enums for avro enum types, while the 
Generic API now requires GenericEnumSymbol. (technically the switch to 
GenericEnumSymbol happened in 1.4.0, but enforcement has been inconsistent.)

Short term, the 
{{genericDatumWriter_failsForGenericRecord_populatedWithRawEnum}} test can be 
fixed similar to the string one by wrapping things in GenericData.EnumSymbol.

It looks like the root cause for your helper utility is that SpecificRecordBase 
claims to implement GenericRecord, but then returns the raw java enums for the 
two avro enum fields rather than the correct o.a.avro.generic type.

> GenericDatumWriter broken with Enum
> ---
>
> Key: AVRO-1810
> URL: https://issues.apache.org/jira/browse/AVRO-1810
> Project: Avro
>  Issue Type: Bug
>  Components: java
>Affects Versions: 1.8.0
>Reporter: Ryon Day
>Priority: Blocker
>
> {panel:title=Description|titleBGColor=#3FA|bgColor=#DDD}
> Using the GenericDatumWriter with either Generic OR SpecificRecord will break 
> if an Enum is present.
> {panel}
> {panel:title=Steps To Reproduce|titleBGColor=#8DB|bgColor=#DDD}
> I have been tracking Avro decoding oddities for a while.
> The tests for this issue can be found 
> [here|https://github.com/ryonday/avroDecodingHelp/blob/master/src/test/java/com/ryonday/test/Avro180EnumFail.java]
> {panel}
> {panel:title=Notes|titleBGColor=#3AF|bgColor=#DDD}
> Due to the debacle that is the Avro "UTF8" object, we have been avoiding it 
> by using the following scheme:
> * Write incoming records to a byte array using the GenericDatumWriter
> * Read back the byte array to our compiled Java domain objects using a 
> SpecificDatumWriter
> This worked great with Avro 1.7.7, and this is a binary-incompatable breaking 
> change with 1.8.0.
> This would appear to be caused by an addition in the 
> {{GenericDatumWriter:163-164}}:
> {code}
>   if (!data.isEnum(datum))
>   throw new AvroTypeException("Not an enum: "+datum);
> {code}
> {panel}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (AVRO-1810) GenericDatumWriter broken with Enum

2016-03-10 Thread Sean Busbey (JIRA)

[ 
https://issues.apache.org/jira/browse/AVRO-1810?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15189944#comment-15189944
 ] 

Sean Busbey commented on AVRO-1810:
---

{quote}
Due to the debacle that is the Avro "UTF8" object, we have been avoiding it by 
using the following scheme:
{quote}

I am curious to hear what the debacle is, perhaps you could send a description 
to user@avro or dev@avro?

{quote}
This worked great with Avro 1.7.7, and this is a binary-incompatable breaking 
change with 1.8.0.
This would appear to be caused by an addition in the GenericDatumWriter:163-164:
{code}
  if (!data.isEnum(datum))
  throw new AvroTypeException("Not an enum: "+datum);
{code}
{quote}

This was the breaking change documented in AVRO-997.

The example in 
{{genericDatumWriter_andSpecificDatumWriter_failForGenericRecord_populatedWithTextualEnum}}
 is using Strings for enum types. Those should be GenericEnumSymbol, as 
documented in the [javadocs for type 
mapping|http://avro.apache.org/docs/1.8.0/api/java/org/apache/avro/generic/package-summary.html]
 and the release notes for AVRO-997.

The other two examples are based on what look like generated specific classes 
from [the schemas in your test 
project|https://github.com/ryonday/avroDecodingHelp/tree/master/src/main/avro], 
is that right?

If I build the test project at the top level is it straight forward to examine 
hte generated classes?

> GenericDatumWriter broken with Enum
> ---
>
> Key: AVRO-1810
> URL: https://issues.apache.org/jira/browse/AVRO-1810
> Project: Avro
>  Issue Type: Bug
>  Components: java
>Affects Versions: 1.8.0
>Reporter: Ryon Day
>Priority: Blocker
>
> {panel:title=Description|titleBGColor=#3FA|bgColor=#DDD}
> Using the GenericDatumWriter with either Generic OR SpecificRecord will break 
> if an Enum is present.
> {panel}
> {panel:title=Steps To Reproduce|titleBGColor=#8DB|bgColor=#DDD}
> I have been tracking Avro decoding oddities for a while.
> The tests for this issue can be found 
> [here|https://github.com/ryonday/avroDecodingHelp/blob/master/src/test/java/com/ryonday/test/Avro180EnumFail.java]
> {panel}
> {panel:title=Notes|titleBGColor=#3AF|bgColor=#DDD}
> Due to the debacle that is the Avro "UTF8" object, we have been avoiding it 
> by using the following scheme:
> * Write incoming records to a byte array using the GenericDatumWriter
> * Read back the byte array to our compiled Java domain objects using a 
> SpecificDatumWriter
> This worked great with Avro 1.7.7, and this is a binary-incompatable breaking 
> change with 1.8.0.
> This would appear to be caused by an addition in the 
> {{GenericDatumWriter:163-164}}:
> {code}
>   if (!data.isEnum(datum))
>   throw new AvroTypeException("Not an enum: "+datum);
> {code}
> {panel}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Created] (AVRO-1810) GenericDatumWriter broken with Enum

2016-03-10 Thread Ryon Day (JIRA)
Ryon Day created AVRO-1810:
--

 Summary: GenericDatumWriter broken with Enum
 Key: AVRO-1810
 URL: https://issues.apache.org/jira/browse/AVRO-1810
 Project: Avro
  Issue Type: Bug
  Components: java
Affects Versions: 1.8.0
Reporter: Ryon Day
Priority: Blocker


{panel:title=Description|titleBGColor=#3FA|bgColor=#DDD}
Using the GenericDatumWriter with either Generic OR SpecificRecord will break 
if an Enum is present.
{panel}

{panel:title=Steps To Reproduce|titleBGColor=#8DB|bgColor=#DDD}
I have been tracking Avro decoding oddities for a while.
The tests for this issue can be found 
[here|https://github.com/ryonday/avroDecodingHelp/blob/master/src/test/java/com/ryonday/test/Avro180EnumFail.java]
{panel}

{panel:title=Notes|titleBGColor=#3AF|bgColor=#DDD}
Due to the debacle that is the Avro "UTF8" object, we have been avoiding it by 
using the following scheme:

* Write incoming records to a byte array using the GenericDatumWriter
* Read back the byte array to our compiled Java domain objects using a 
SpecificDatumWriter

This worked great with Avro 1.7.7, and this is a binary-incompatable breaking 
change with 1.8.0.

This would appear to be caused by an addition in the 
{{GenericDatumWriter:163-164}}:

{code}
  if (!data.isEnum(datum))
  throw new AvroTypeException("Not an enum: "+datum);
{code}
{panel}




--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (AVRO-1809) I wish to remove optimization from GenericDatumReader.getResolver

2016-03-10 Thread Doug Cutting (JIRA)

[ 
https://issues.apache.org/jira/browse/AVRO-1809?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15189629#comment-15189629
 ] 

Doug Cutting commented on AVRO-1809:


Can you please benchmark this change, ideally including its impact on 
multithreaded performance?

Generally, each past performance optimization was only made only after 
demonstrating measurable improvement.  For single-threaded performance, this is 
usually with Perf.java.  There aren't yet standard benchmarks for 
multi-threaded performance.

The changes that led to the current implementation are primarily discussed in 
the following issues:

https://issues.apache.org/jira/browse/AVRO-743
https://issues.apache.org/jira/browse/AVRO-557

> I wish to remove optimization from GenericDatumReader.getResolver
> -
>
> Key: AVRO-1809
> URL: https://issues.apache.org/jira/browse/AVRO-1809
> Project: Avro
>  Issue Type: Wish
>  Components: java
>Reporter: Konstantin Usachev
>Priority: Minor
>
> There is an optimization at 
> org.apache.avro.generic.GenericDatumReader.getResolver, when we cache creator 
> thread and it's first returned value. At first, It looks redundant, because 
> it saves three calls to Map.get, which is unmeasurable, especially after 
> Schema's hashcode calculation optimization, made by the same author 
> [~cutting], it's not obvious and adds additional complexity. Also caching of 
> current thread whould be a source of bugs in case of different green threads 
> libraries integration (which, actually, occurred during integration with 
> Quasar).



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (AVRO-1704) Standardized format for encoding messages with Avro

2016-03-10 Thread Doug Cutting (JIRA)

[ 
https://issues.apache.org/jira/browse/AVRO-1704?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15189594#comment-15189594
 ] 

Doug Cutting commented on AVRO-1704:


bq. remove the things that do not impact the binary form of the record

This is already done as part of fingerprint calculation.

https://avro.apache.org/docs/current/spec.html#Parsing+Canonical+Form+for+Schemas

Also, if we opt for a prefix, we might use something like 'A'+'v'+'r'+0, where 
the last character also indicates the format version, including schema hash 
function.  That's similar to what's used to label the file format, and has a 
side benefit of clearly demonstrating that this is binary, non-textual data.

> Standardized format for encoding messages with Avro
> ---
>
> Key: AVRO-1704
> URL: https://issues.apache.org/jira/browse/AVRO-1704
> Project: Avro
>  Issue Type: Improvement
>Reporter: Daniel Schierbeck
>
> I'm currently using the Datafile format for encoding messages that are 
> written to Kafka and Cassandra. This seems rather wasteful:
> 1. I only encode a single record at a time, so there's no need for sync 
> markers and other metadata related to multi-record files.
> 2. The entire schema is inlined every time.
> However, the Datafile format is the only one that has been standardized, 
> meaning that I can read and write data with minimal effort across the various 
> languages in use in my organization. If there was a standardized format for 
> encoding single values that was optimized for out-of-band schema transfer, I 
> would much rather use that.
> I think the necessary pieces of the format would be:
> 1. A format version number.
> 2. A schema fingerprint type identifier, i.e. Rabin, MD5, SHA256, etc.
> 3. The actual schema fingerprint (according to the type.)
> 4. Optional metadata map.
> 5. The encoded datum.
> The language libraries would implement a MessageWriter that would encode 
> datums in this format, as well as a MessageReader that, given a SchemaStore, 
> would be able to decode datums. The reader would decode the fingerprint and 
> ask its SchemaStore to return the corresponding writer's schema.
> The idea is that SchemaStore would be an abstract interface that allowed 
> library users to inject custom backends. A simple, file system based one 
> could be provided out of the box.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[GitHub] avro pull request: AVRO-1809 Removed creator's thread caching from...

2016-03-10 Thread DrVirtual
GitHub user DrVirtual opened a pull request:

https://github.com/apache/avro/pull/76

AVRO-1809 Removed creator's thread caching from 
GenericDatumReader.getResolver

For more explanation please read 
https://issues.apache.org/jira/browse/AVRO-1809

You can merge this pull request into a Git repository by running:

$ git pull https://github.com/DrVirtual/avro 
AVRO-1809-Removed-redundant-GenericDatumReader.getResolver-optimization

Alternatively you can review and apply these changes as the patch at:

https://github.com/apache/avro/pull/76.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #76


commit 54efba3b7be9408337479477a83fcf7d4260ac20
Author: k.usachev 
Date:   2016-03-10T16:19:00Z

AVRO-1809 Removed creator's thread caching from 
GenericDatumReader.getResolver




---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[jira] [Commented] (AVRO-1809) I wish to remove optimization from GenericDatumReader.getResolver

2016-03-10 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/AVRO-1809?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15189529#comment-15189529
 ] 

ASF GitHub Bot commented on AVRO-1809:
--

GitHub user DrVirtual opened a pull request:

https://github.com/apache/avro/pull/76

AVRO-1809 Removed creator's thread caching from 
GenericDatumReader.getResolver

For more explanation please read 
https://issues.apache.org/jira/browse/AVRO-1809

You can merge this pull request into a Git repository by running:

$ git pull https://github.com/DrVirtual/avro 
AVRO-1809-Removed-redundant-GenericDatumReader.getResolver-optimization

Alternatively you can review and apply these changes as the patch at:

https://github.com/apache/avro/pull/76.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #76


commit 54efba3b7be9408337479477a83fcf7d4260ac20
Author: k.usachev 
Date:   2016-03-10T16:19:00Z

AVRO-1809 Removed creator's thread caching from 
GenericDatumReader.getResolver




> I wish to remove optimization from GenericDatumReader.getResolver
> -
>
> Key: AVRO-1809
> URL: https://issues.apache.org/jira/browse/AVRO-1809
> Project: Avro
>  Issue Type: Wish
>  Components: java
>Reporter: Konstantin Usachev
>Priority: Minor
>
> There is an optimization at 
> org.apache.avro.generic.GenericDatumReader.getResolver, when we cache creator 
> thread and it's first returned value. At first, It looks redundant, because 
> it saves three calls to Map.get, which is unmeasurable, especially after 
> Schema's hashcode calculation optimization, made by the same author 
> [~cutting], it's not obvious and adds additional complexity. Also caching of 
> current thread whould be a source of bugs in case of different green threads 
> libraries integration (which, actually, occurred during integration with 
> Quasar).



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Created] (AVRO-1809) I wish to remove optimization from GenericDatumReader.getResolver

2016-03-10 Thread Konstantin Usachev (JIRA)
Konstantin Usachev created AVRO-1809:


 Summary: I wish to remove optimization from 
GenericDatumReader.getResolver
 Key: AVRO-1809
 URL: https://issues.apache.org/jira/browse/AVRO-1809
 Project: Avro
  Issue Type: Wish
  Components: java
Reporter: Konstantin Usachev
Priority: Minor


There is an optimization at 
org.apache.avro.generic.GenericDatumReader.getResolver, when we cache creator 
thread and it's first returned value. At first, It looks redundant, because it 
saves three calls to Map.get, which is unmeasurable, especially after Schema's 
hashcode calculation optimization, made by the same author [~cutting], it's not 
obvious and adds additional complexity. Also caching of current thread whould 
be a source of bugs in case of different green threads libraries integration 
(which, actually, occurred during integration with Quasar).



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (AVRO-1704) Standardized format for encoding messages with Avro

2016-03-10 Thread Niels Basjes (JIRA)

[ 
https://issues.apache.org/jira/browse/AVRO-1704?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15189473#comment-15189473
 ] 

Niels Basjes commented on AVRO-1704:


Note that having the "AVRO" prefix will also limit the number of needless calls 
to the Schema registry when bad records are put into the stream (like the Timer 
ticks example).

> Standardized format for encoding messages with Avro
> ---
>
> Key: AVRO-1704
> URL: https://issues.apache.org/jira/browse/AVRO-1704
> Project: Avro
>  Issue Type: Improvement
>Reporter: Daniel Schierbeck
>
> I'm currently using the Datafile format for encoding messages that are 
> written to Kafka and Cassandra. This seems rather wasteful:
> 1. I only encode a single record at a time, so there's no need for sync 
> markers and other metadata related to multi-record files.
> 2. The entire schema is inlined every time.
> However, the Datafile format is the only one that has been standardized, 
> meaning that I can read and write data with minimal effort across the various 
> languages in use in my organization. If there was a standardized format for 
> encoding single values that was optimized for out-of-band schema transfer, I 
> would much rather use that.
> I think the necessary pieces of the format would be:
> 1. A format version number.
> 2. A schema fingerprint type identifier, i.e. Rabin, MD5, SHA256, etc.
> 3. The actual schema fingerprint (according to the type.)
> 4. Optional metadata map.
> 5. The encoded datum.
> The language libraries would implement a MessageWriter that would encode 
> datums in this format, as well as a MessageReader that, given a SchemaStore, 
> would be able to decode datums. The reader would decode the fingerprint and 
> ask its SchemaStore to return the corresponding writer's schema.
> The idea is that SchemaStore would be an abstract interface that allowed 
> library users to inject custom backends. A simple, file system based one 
> could be provided out of the box.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[GitHub] avro pull request: AVRO-1808. Added possibility of overriding lock...

2016-03-10 Thread DrVirtual
GitHub user DrVirtual opened a pull request:

https://github.com/apache/avro/pull/75

AVRO-1808. Added possibility of overriding lock mechanics for 
Requestor.handshakeLock

Look at https://issues.apache.org/jira/browse/AVRO-1808 for more information

You can merge this pull request into a Git repository by running:

$ git pull https://github.com/DrVirtual/avro 
AVRO-1808-Requestor-handshake-lock

Alternatively you can review and apply these changes as the patch at:

https://github.com/apache/avro/pull/75.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #75


commit cf3ea721b6e47eec7ec6284a31bb057c9858e8c6
Author: k.usachev 
Date:   2016-03-10T15:08:32Z

AVRO-1808. Added possibility of overriding lock mechanics for 
Requestor.handshakeLock




---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[jira] [Commented] (AVRO-1808) It would be useful to be able to change lock mechanics for handshake at Requestor

2016-03-10 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/AVRO-1808?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15189403#comment-15189403
 ] 

ASF GitHub Bot commented on AVRO-1808:
--

GitHub user DrVirtual opened a pull request:

https://github.com/apache/avro/pull/75

AVRO-1808. Added possibility of overriding lock mechanics for 
Requestor.handshakeLock

Look at https://issues.apache.org/jira/browse/AVRO-1808 for more information

You can merge this pull request into a Git repository by running:

$ git pull https://github.com/DrVirtual/avro 
AVRO-1808-Requestor-handshake-lock

Alternatively you can review and apply these changes as the patch at:

https://github.com/apache/avro/pull/75.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #75


commit cf3ea721b6e47eec7ec6284a31bb057c9858e8c6
Author: k.usachev 
Date:   2016-03-10T15:08:32Z

AVRO-1808. Added possibility of overriding lock mechanics for 
Requestor.handshakeLock




> It would be useful to be able to change lock mechanics for handshake at 
> Requestor 
> --
>
> Key: AVRO-1808
> URL: https://issues.apache.org/jira/browse/AVRO-1808
> Project: Avro
>  Issue Type: Improvement
>  Components: java
>Reporter: Konstantin Usachev
>Priority: Minor
>
> For now there is no option to change lock mechanics for handshake at 
> org.apache.avro.ipc.Requestor - it uses ReentrantLock. We can't use this type 
> of lock in our system, so we need some way to override it, like the way you 
> have already done for org.apache.avro.ipc.Transceiver.channelLock.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (AVRO-1704) Standardized format for encoding messages with Avro

2016-03-10 Thread Niels Basjes (JIRA)

[ 
https://issues.apache.org/jira/browse/AVRO-1704?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15189402#comment-15189402
 ] 

Niels Basjes commented on AVRO-1704:


I've been looking into what kind of solution would work here since I'm working 
on a project where we need datastructures going into Kafka and be available to 
multiple consumers.

The fundamental problem we need to solve is that of "Schema Evolution" in a 
streaming environment (Let's assume Kafka with the built in persistence of 
records).
We need three things to make this happen:
# A way to recognize a 'blob' is a serialized AVRO record.
#* We can simply assume it is always an AVRO record. 
#* I think we should simply let such a record start with "AVRO" to ensure we 
can cleanly catch problems like this STORM-512 (Summary: Timer ticks we written 
into Kafka which caused a lot of deserialization errors in reading the AVRO 
records.)
# A way to determine the schema this was written with.
#* As indicated above I vote for using the CRC-64-AVRO. 
#** I noticed that a simple typo fix in the documentation of a Schema causes a 
new fingerprint to be generated. 
#** Proposal: I think we should 'clean' the schema before calculating the 
fingerprint. I.e. remove the things that do not impact the binary form of the 
record (like the doc field).
# Have a place where we can find the schemas using the fingerprint as the key.
#* Here I think (looking at AVRO-1124 and the fact that there are ready to run 
implementations like this [Schema 
Registry|http://docs.confluent.io/current/schema-registry/docs/index.html]) we 
should limit what we keep inside Avro to something like a "SchemaFactory" 
interface (as the storage/retrieval interface to get a Schema) and a very basic 
implementation that simply reads the available schema's from a (set of) 
property file(s). Using this others can write additional implementations that 
can read/write to things like databases or the above mentioned Schema Registry.

So to summarize my proposal on the standard for the {{Single record 
serialization format}} can be written as:
{code}"AVRO"{code}

[~rdblue], I'm seeking feedback from you guys on this proposal. 


> Standardized format for encoding messages with Avro
> ---
>
> Key: AVRO-1704
> URL: https://issues.apache.org/jira/browse/AVRO-1704
> Project: Avro
>  Issue Type: Improvement
>Reporter: Daniel Schierbeck
>
> I'm currently using the Datafile format for encoding messages that are 
> written to Kafka and Cassandra. This seems rather wasteful:
> 1. I only encode a single record at a time, so there's no need for sync 
> markers and other metadata related to multi-record files.
> 2. The entire schema is inlined every time.
> However, the Datafile format is the only one that has been standardized, 
> meaning that I can read and write data with minimal effort across the various 
> languages in use in my organization. If there was a standardized format for 
> encoding single values that was optimized for out-of-band schema transfer, I 
> would much rather use that.
> I think the necessary pieces of the format would be:
> 1. A format version number.
> 2. A schema fingerprint type identifier, i.e. Rabin, MD5, SHA256, etc.
> 3. The actual schema fingerprint (according to the type.)
> 4. Optional metadata map.
> 5. The encoded datum.
> The language libraries would implement a MessageWriter that would encode 
> datums in this format, as well as a MessageReader that, given a SchemaStore, 
> would be able to decode datums. The reader would decode the fingerprint and 
> ask its SchemaStore to return the corresponding writer's schema.
> The idea is that SchemaStore would be an abstract interface that allowed 
> library users to inject custom backends. A simple, file system based one 
> could be provided out of the box.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Created] (AVRO-1808) It would be useful to be able to change lock mechanics for handshake at Requestor

2016-03-10 Thread Konstantin Usachev (JIRA)
Konstantin Usachev created AVRO-1808:


 Summary: It would be useful to be able to change lock mechanics 
for handshake at Requestor 
 Key: AVRO-1808
 URL: https://issues.apache.org/jira/browse/AVRO-1808
 Project: Avro
  Issue Type: Improvement
  Components: java
Reporter: Konstantin Usachev
Priority: Minor


For now there is no option to change lock mechanics for handshake at 
org.apache.avro.ipc.Requestor - it uses ReentrantLock. We can't use this type 
of lock in our system, so we need some way to override it, like the way you 
have already done for org.apache.avro.ipc.Transceiver.channelLock.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)