avro-dev mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Ryan Blue <b...@cloudera.com>
Subject Re: [DISCUSS][JAVA] Generating toBytes/fromBytes methods?
Date Mon, 11 Jan 2016 22:53:56 GMT
What you're bringing up is a common problem for Kafka, but I think we 
can separate a couple of the issues here.

First, I think it is a good idea to include the fingerprint with the 
encoded bytes, regardless of how schemas are coordinated. There's no way 
to reliably read those bytes if you don't know the write-side schema. 
Even if you plan on only ever having one schema and coordinate by never 
changing it, I think it is better to have a check that the schema you're 
reading with matches the write side. It is better to have a reasonable 
failure (unknown write schema) than to read corrupt data or fail while 
deserializing.

The second problem is schema coordination. There are a few approaches 
here, including a schema repository service like you suggest or 
SchemaStore like Martin mentioned. I think we should keep the two 
concerns separate because there are different ways of tackling this 
issue without necessarily running a service.

rb

On 12/23/2015 03:19 AM, Niels Basjes wrote:
> Yes, as a generic crossplatform solution this makes a lot of sense. It s
> easy to build and stops consuming the messages as soon as the fingerprint
> changes.
> In my corporate reality I see that a source of messages puts them into
> Kafka, then several consumers read and deserialize them in near-realtime.
> In the static situation of "the schema doesn't change" having a fingerprint
> of the schema would suffice.
> But now the schema changes. I would like to deploy the new version of the
> source system, and consume these new messages using the schema migration
> idea. I.e. read the records as long as they are backwards compatible.
> As a general assumption: it is impossible to deploy a new version of all
> consuming applications at the same time.
> I want as little down time as possible and I do not want to lose any
> messages.
>
> Is this possible with a simple fingerprint?
> The only direction I can think of right now is by having a central registry
> that contains all schemas ever used, indexed by their fingerprint.
>
> What alternative solutions can we think of?
>
> Niels
>
> On Wed, 23 Dec 2015, 01:42 Sean Busbey <busbey@cloudera.com> wrote:
>
>> Including a schema fingerprint at the start
>>
>> 1) reuses stuff we have
>> 2) gives a language independent notion of compatibility
>> 3) doesn't bind how folks get stuff in/out of the single record form.
>>
>> --
>> Sean Busbey
>> On Dec 22, 2015 06:52, "Niels Basjes" <Niels@basjes.nl> wrote:
>>
>>> I was not clear enough in my previous email.
>>> What I meant is to 'wrap' the application schema in a serialization
>> wrapper
>>> schema that has a field indicating the "schema classname".
>>> That (generic setup) combined with some generated code in the schema
>>> classes should yield a solution that supports schema migration.
>>>
>>> Niels
>>>
>>> On Tue, Dec 22, 2015 at 11:55 AM, Niels Basjes <Niels@basjes.nl> wrote:
>>>
>>>> Thanks for pointing this out.
>>>> This is exactly what I was working on.
>>>>
>>>> The way I solved the 'does the schema match' question at work is by
>>>> requiring that all schema's start with a single text field "schema
>>>> classname" being the full class name of the class that was used to
>>> generate
>>>> it.
>>>> That way we can have newer versions of the schema and still be able to
>>>> unpack them. In this form the classname is essentially an indicator if
>>>> schema migration is possible; even though the schemas are different.
>>>>
>>>> What do you think of this direction?
>>>>
>>>> Niels
>>>>
>>>>
>>>> On Mon, Dec 21, 2015 at 11:30 PM, Ryan Blue <blue@cloudera.com> wrote:
>>>>
>>>>> Niels,
>>>>>
>>>>> This sounds like a good idea to me to have methods like this. I've had
>>> to
>>>>> write those methods several times!
>>>>>
>>>>> The idea is also related to AVRO-1704 [1], which is a suggestion to
>>>>> standardize the encoding that is used for single records. Some
>> projects
>>>>> have been embedding the schema fingerprint at the start of each
>> record,
>>> for
>>>>> example, which would be a helpful thing to do.
>>>>>
>>>>> It may also be a good idea to create a helper object rather than
>>>>> attaching new methods to the datum classes themselves. In your example
>>>>> below, you have to create a new encoder or decoder for each method
>>> call. We
>>>>> could instead keep a backing buffer and encoder/decoder on a class
>> that
>>> the
>>>>> caller instantiates so that they can be reused. At the same time, that
>>>>> would make it possible to reuse the class with any data model and
>> manage
>>>>> the available schemas (if embedding the fingerprint).
>>>>>
>>>>> I'm thinking something like this:
>>>>>
>>>>>    ReflectClass datum = new ReflectClass();
>>>>>    ReflectData model = ReflectData.get();
>>>>>    DatumCodec codec = new DatumCodec(model, schema);
>>>>>
>>>>>    # convert datum to bytes using data model
>>>>>    byte[] asBytes = codec.toBytes(datum);
>>>>>
>>>>>    # convert bytes to datum using data model
>>>>>    ReflectClass copy = codec.fromBytes(asBytes);
>>>>>
>>>>> What do you think?
>>>>>
>>>>> rb
>>>>>
>>>>>
>>>>> [1]: https://issues.apache.org/jira/browse/AVRO-1704
>>>>>
>>>>>
>>>>> On 12/18/2015 05:01 AM, Niels Basjes wrote:
>>>>>
>>>>>> Hi,
>>>>>>
>>>>>> I'm working on a project where I'm putting Avro records into Kafka
>> and
>>> at
>>>>>> the other end pull them out again.
>>>>>> For that purpose I wrote two methods 'toBytes' and 'fromBytes' in
a
>>>>>> separate class (see below).
>>>>>>
>>>>>> I see this as the type of problem many developers run into.
>>>>>> Would it be a good idea to generate methods like these into the
>>> generated
>>>>>> Java code?
>>>>>>
>>>>>> This would make it possible to serialize and deserialize singles
>>> records
>>>>>> like this:
>>>>>>
>>>>>> byte [] someBytes = measurement.toBytes();
>>>>>> Measurement m = Measurement.fromBytes(someBytes);
>>>>>>
>>>>>> Niels Basjes
>>>>>>
>>>>>> P.S. possibly not name it toBytes but getBytes (similar to what the
>>>>>> String
>>>>>> class has)
>>>>>>
>>>>>> public final class MeasurementSerializer {
>>>>>>       private MeasurementSerializer() {
>>>>>>       }
>>>>>>
>>>>>>       public static Measurement fromBytes(byte[] bytes) throws
>>>>>> IOException {
>>>>>>           try {
>>>>>>               DatumReader<Measurement> reader = new
>>>>>> SpecificDatumReader<>(Measurement.getClassSchema());
>>>>>>               Decoder decoder =
>>> DecoderFactory.get().binaryDecoder(bytes,
>>>>>> null);
>>>>>>               return reader.read(null, decoder);
>>>>>>           } catch (RuntimeException rex) {
>>>>>>               throw new IOException(rex.getMessage());
>>>>>>           }
>>>>>>       }
>>>>>>
>>>>>>       public static byte[] toBytes(Measurement measurement) throws
>>>>>> IOException {
>>>>>>           try {
>>>>>>               ByteArrayOutputStream out = new ByteArrayOutputStream();
>>>>>>               Encoder encoder =
>> EncoderFactory.get().binaryEncoder(out,
>>>>>> null);
>>>>>>               SpecificDatumWriter<Measurement> writer = new
>>>>>> SpecificDatumWriter<>(Measurement.class);
>>>>>>               writer.write(measurement, encoder);
>>>>>>               encoder.flush();
>>>>>>               out.close();
>>>>>>               return out.toByteArray();
>>>>>>           } catch (RuntimeException rex) {
>>>>>>               throw new IOException(rex.getMessage());
>>>>>>           }
>>>>>>       }
>>>>>> }
>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>>
>>>>> --
>>>>> Ryan Blue
>>>>> Software Engineer
>>>>> Cloudera, Inc.
>>>>>
>>>>
>>>>
>>>>
>>>> --
>>>> Best regards / Met vriendelijke groeten,
>>>>
>>>> Niels Basjes
>>>>
>>>
>>>
>>>
>>> --
>>> Best regards / Met vriendelijke groeten,
>>>
>>> Niels Basjes
>>>
>>
>


-- 
Ryan Blue
Software Engineer
Cloudera, Inc.

Mime
View raw message