You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@avro.apache.org by 常冰琳 <de...@gmail.com> on 2011/10/13 14:25:31 UTC

How to add optional new record fields and/or new methods in avro-ipc?

Hi,
I try to use avro-ipc in my project and got a problem, if the server side
needs to upgrade,
add a few optional field to Request/Response object, or add a new method,
thus change the
schema.
I use reflect, add a new String field in Request class with @Nullable in
server side, keep client
side unchanged, but it won't to work.
I think protobuf supports this, just by adding new optional fields.
So does it possible or how to support backward compatibility in other way?

Here is the protocol class


old:



public interface SampleService {
  public static class Request {
    int id;
    String name;
    @Nullable
    String address;

    public Request() {
      this(0, "null", null);
    }

    public Request(int id, String name, String address) {
      this.id = id;
      this.name = name;
      this.address = address;
    }

    public int getId() {
      return id;
    }

    public String getName() {
      return name;
    }

    public String getAddress() {
      return address;
    }

  }

  public static class Response {
    @Nullable
    String value;

    public Response() {
      this("null");
    }
    public Response(String value) {
      this.value = value;
    }

    public String getValue() {
      return value;
    }
  }

  int fibonacci(int n);
  int sum(int [] nums, int base);
  Response complexCall(Request req);
}




new:


public interface SampleService {
  public static class Request {
    int id;
    String name;
    @Nullable
    String address;
    @Nullable
    String address2;

    public Request() {
      this(0, "null", null, null);
    }

    public Request(int id, String name, String address, String address2) {
      this.id = id;
      this.name = name;
      this.address = address;
      this.address2 = address2;
    }

    public int getId() {
      return id;
    }

    public String getName() {
      return name;
    }

    public String getAddress() {
      return address;
    }

    public String getAddress2() {
      return address2;
    }

  }

  public static class Response {
    @Nullable
    String value;

    public Response() {
      this("null");
    }
    public Response(String value) {
      this.value = value;
    }

    public String getValue() {
      return value;
    }
  }

  int fibonacci(int n);
  int sum(int [] nums, int base);
  Response complexCall(Request req);
}


Here is the error message:

Exception in thread "main" org.apache.avro.AvroRuntimeException:
org.apache.avro.AvroTypeException: Found {
  "type" : "record",
  "name" : "Request",
  "namespace" : "com.mycompany.app.SampleService$",
  "fields" : [ {
    "name" : "id",
    "type" : "int"
  }, {
    "name" : "name",
    "type" : "string"
  }, {
    "name" : "address",
    "type" : [ "null", "string" ]
  } ]
}, expecting {
  "type" : "record",
  "name" : "Request",
  "namespace" : "com.mycompany.app.SampleService$",
  "fields" : [ {
    "name" : "id",
    "type" : "int"
  }, {
    "name" : "name",
    "type" : "string"
  }, {
    "name" : "address",
    "type" : [ "null", "string" ]
  }, {
    "name" : "address2",
    "type" : [ "null", "string" ]
  } ]
}
at
org.apache.avro.ipc.specific.SpecificRequestor.readError(SpecificRequestor.java:126)
at org.apache.avro.ipc.Requestor$Response.getResponse(Requestor.java:555)
at
org.apache.avro.ipc.Requestor$TransceiverCallback.handleResult(Requestor.java:360)
at
org.apache.avro.ipc.Requestor$TransceiverCallback.handleResult(Requestor.java:323)
at
org.apache.avro.ipc.NettyTransceiver$NettyClientAvroHandler.messageReceived(NettyTransceiver.java:382)
at
org.jboss.netty.channel.SimpleChannelUpstreamHandler.handleUpstream(SimpleChannelUpstreamHandler.java:80)
at
org.apache.avro.ipc.NettyTransceiver$NettyClientAvroHandler.handleUpstream(NettyTransceiver.java:364)
at
org.jboss.netty.channel.DefaultChannelPipeline.sendUpstream(DefaultChannelPipeline.java:545)
at
org.jboss.netty.channel.DefaultChannelPipeline$DefaultChannelHandlerContext.sendUpstream(DefaultChannelPipeline.java:754)
at org.jboss.netty.channel.Channels.fireMessageReceived(Channels.java:302)
at
org.jboss.netty.handler.codec.frame.FrameDecoder.unfoldAndFireMessageReceived(FrameDecoder.java:317)
at
org.jboss.netty.handler.codec.frame.FrameDecoder.callDecode(FrameDecoder.java:299)
at
org.jboss.netty.handler.codec.frame.FrameDecoder.messageReceived(FrameDecoder.java:216)
at
org.jboss.netty.channel.SimpleChannelUpstreamHandler.handleUpstream(SimpleChannelUpstreamHandler.java:80)
at
org.jboss.netty.channel.DefaultChannelPipeline.sendUpstream(DefaultChannelPipeline.java:545)
at
org.jboss.netty.channel.DefaultChannelPipeline.sendUpstream(DefaultChannelPipeline.java:540)
at org.jboss.netty.channel.Channels.fireMessageReceived(Channels.java:274)
at org.jboss.netty.channel.Channels.fireMessageReceived(Channels.java:261)
at org.jboss.netty.channel.socket.nio.NioWorker.read(NioWorker.java:349)
at
org.jboss.netty.channel.socket.nio.NioWorker.processSelectedKeys(NioWorker.java:280)
at org.jboss.netty.channel.socket.nio.NioWorker.run(NioWorker.java:200)
at
org.jboss.netty.util.ThreadRenamingRunnable.run(ThreadRenamingRunnable.java:108)
at
org.jboss.netty.util.internal.DeadLockProofWorker$1.run(DeadLockProofWorker.java:44)
at
java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
at
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
at java.lang.Thread.run(Thread.java:680)

Re: How to add optional new record fields and/or new methods in avro-ipc?

Posted by Doug Cutting <cu...@apache.org>.
On 10/18/2011 10:51 AM, Scott Carey wrote:
> So, any ideas what is causing the original User's problem?  @Nullable is
> in use with Reflect (does not work), Specific works (with default values
> but not without -- it appears to have "null" first but not confirmed).
> I suspect there is something else going on.

Sorry, I should have looked back and noticed that he was already using
@Nullable!  This does look like a bug then.  It would be useful to see
the server-side stack trace that created the AvroTypeException.  My
guess is that this is logged on the server side as a "system error".

Doug

Re: How to add optional new record fields and/or new methods in avro-ipc?

Posted by Scott Carey <sc...@apache.org>.

On 10/18/11 10:38 AM, "Doug Cutting" <cu...@apache.org> wrote:

>On 10/18/2011 10:09 AM, Scott Carey wrote:
>> On 10/18/11 9:47 AM, "Doug Cutting" <cu...@apache.org> wrote:
>>> To amend this, you can use Avro's @Nullable annotation:
>> 
>> The problem is that this does not provide the ability to evolve schemas
>>if
>> you add a field
>> since you would need @Default or something similar, as well:
>> @Nullable
>> @Default(null)
>
>I don't think this is required.  The default value for a union is the
>default value for its first branch.  A "null" schema needs no default.
>So the schema ["null", "string"] needs to specify no default value while
>the schema ["string", "null"] does.  Thus the best practice for nullable
>values is to place the null first in the union.  This is what is done by
>the @Nullable annotation.
>
>Perhaps we should clarify this in the Specification?  We might state
>that a "null" schema implicitly has a default value of null since that's
>the only value its ever permitted to have anyway.

Good to know.

So, any ideas what is causing the original User's problem?  @Nullable is
in use with Reflect (does not work), Specific works (with default values
but not without -- it appears to have "null" first but not confirmed).
I suspect there is something else going on.

>
>> Does reflect have any concept of default values?
>
>No.  We could add an @Default annotation, I suppose.  But I don't think
>this is needed for nullable stuff.
>
>Doug



Re: How to add optional new record fields and/or new methods in avro-ipc?

Posted by Binglin Chang <de...@gmail.com>.
Thanks for the reply.
I did some simple tests related to default "default value" for UnionSchema,
and traced into source code:

Schema.java

static Schema parse(JsonNode schema, Names names)

1147          Field f = new Field(fieldName, fieldSchema,

1148                              fieldDoc, field.get("default"), order);
It seems that there is no special handle for UnionSchema Field to get a
proper default value, and
ReflectData do not infer default values for fields, at least union fields(I
think this is a bug):
ReflectData.java

291              fields.add(new Schema.Field(field.getName(),

292                  fieldSchema, null /* doc */, null));
And finally, I think annotation like @Default(..) is needed.


On Wed, Oct 19, 2011 at 5:36 AM, Doug Cutting <cu...@apache.org> wrote:

> On 10/18/2011 11:40 AM, Scott Carey wrote:
> > I'm not sure that I understand "The default value for a union is the
> > default value for its first branch."
> > Defaults don't apply to any types in a union, only to fields on a record.
> > So the Schema ["Foo", "Bar"] can have no default, nor can any of its
> > branches.
>
> From the spec's definition of default values:
>
> "Permitted values depend on the field's schema type, according to the
> table below. Default values for union fields correspond to the first
> schema in the union."
>
> So the default value for a field with a union type as its value is
> assumed to be of the type of the first element of that union.  If the
> first element of that union is "null", then the type of the default
> value must be "null" hence the default value itself can only be "null".
>  So the question is whether we require that to be stated explicitly in
> the schema.  I assumed that we did not.  Either way, we should clarify
> the spec around this.
>
> We could continue to insist that, if no default value is explicitly
> specified in the reader's schema, and the writer's schema lacks a field,
> then an error is thrown.  Or we could say that the default value for
> default values is null, so that if the reader adds a field that's a
> union with "null" as its first branch then no default value need be
> present.
>
> Doug
>

Re: How to add optional new record fields and/or new methods in avro-ipc?

Posted by Doug Cutting <cu...@apache.org>.
On 10/19/2011 10:06 AM, Scott Carey wrote:
> In that case, Avro Java does not adhere to the spec.  I have had things
> like this as a field in Schemas I use for a long time (> 1 year, likely
> nearly 2)
> 
> {"name": "clientEnvironment", "type": [
>   {"name": "BrowserData", "type": "record", "fields" : [
>     {"name": "osName", "type": "string", "default": ""},
>     {"name": "browserVersion", "type": "string", "default": ""},
>     {"name": "flashVersion", "type": "string", "default": ""}
>   ]}, 
>   "null"
> ], "default":null}
> 
> 
> The default, null, is definitely not of type BrowserData.  Changing it to
> be strict about this would break much archived data.

Have you checked that this actually does what you think it should?
Several times folks have reported problems when "null" is not first in a
union for this very reason.  TestSchema#testUnion explicity tests that
such things fail.  So I'd be surprised if that default value actually works.

Doug

Re: How to add optional new record fields and/or new methods in avro-ipc?

Posted by Scott Carey <sc...@apache.org>.

On 10/18/11 2:36 PM, "Doug Cutting" <cu...@apache.org> wrote:

>On 10/18/2011 11:40 AM, Scott Carey wrote:
>> I'm not sure that I understand "The default value for a union is the
>> default value for its first branch."
>> Defaults don't apply to any types in a union, only to fields on a
>>record.
>> So the Schema ["Foo", "Bar"] can have no default, nor can any of its
>> branches.
>
>From the spec's definition of default values:
>
>"Permitted values depend on the field's schema type, according to the
>table below. Default values for union fields correspond to the first
>schema in the union."

In that case, Avro Java does not adhere to the spec.  I have had things
like this as a field in Schemas I use for a long time (> 1 year, likely
nearly 2)

{"name": "clientEnvironment", "type": [
  {"name": "BrowserData", "type": "record", "fields" : [
    {"name": "osName", "type": "string", "default": ""},
    {"name": "browserVersion", "type": "string", "default": ""},
    {"name": "flashVersion", "type": "string", "default": ""}
  ]}, 
  "null"
], "default":null}


The default, null, is definitely not of type BrowserData.  Changing it to
be strict about this would break much archived data.  It can't be
completely flexible since there is ambiguity on  matching defaults to
types (e.g. the string "\u0040" could be a string, bytes, or fixed).
Perhaps changing the language to "first matching branch" or "first
compatible branch" would work, but then we have to define how the
compatibility works.

>
>So the default value for a field with a union type as its value is
>assumed to be of the type of the first element of that union.  If the
>first element of that union is "null", then the type of the default
>value must be "null" hence the default value itself can only be "null".
> So the question is whether we require that to be stated explicitly in
>the schema.  I assumed that we did not.  Either way, we should clarify
>the spec around this.
>
>We could continue to insist that, if no default value is explicitly
>specified in the reader's schema, and the writer's schema lacks a field,
>then an error is thrown.  Or we could say that the default value for
>default values is null, so that if the reader adds a field that's a
>union with "null" as its first branch then no default value need be
>present.
>
>Doug



Re: How to add optional new record fields and/or new methods in avro-ipc?

Posted by Doug Cutting <cu...@apache.org>.
On 10/18/2011 11:40 AM, Scott Carey wrote:
> I'm not sure that I understand "The default value for a union is the
> default value for its first branch."
> Defaults don't apply to any types in a union, only to fields on a record.
> So the Schema ["Foo", "Bar"] can have no default, nor can any of its
> branches.

>From the spec's definition of default values:

"Permitted values depend on the field's schema type, according to the
table below. Default values for union fields correspond to the first
schema in the union."

So the default value for a field with a union type as its value is
assumed to be of the type of the first element of that union.  If the
first element of that union is "null", then the type of the default
value must be "null" hence the default value itself can only be "null".
 So the question is whether we require that to be stated explicitly in
the schema.  I assumed that we did not.  Either way, we should clarify
the spec around this.

We could continue to insist that, if no default value is explicitly
specified in the reader's schema, and the writer's schema lacks a field,
then an error is thrown.  Or we could say that the default value for
default values is null, so that if the reader adds a field that's a
union with "null" as its first branch then no default value need be present.

Doug

Re: How to add optional new record fields and/or new methods in avro-ipc?

Posted by Scott Carey <sc...@apache.org>.

On 10/18/11 10:38 AM, "Doug Cutting" <cu...@apache.org> wrote:

>On 10/18/2011 10:09 AM, Scott Carey wrote:
>> On 10/18/11 9:47 AM, "Doug Cutting" <cu...@apache.org> wrote:
>>> To amend this, you can use Avro's @Nullable annotation:
>> 
>> The problem is that this does not provide the ability to evolve schemas
>>if
>> you add a field
>> since you would need @Default or something similar, as well:
>> @Nullable
>> @Default(null)
>
>I don't think this is required.  The default value for a union is the
>default value for its first branch.  A "null" schema needs no default.
>So the schema ["null", "string"] needs to specify no default value while
>the schema ["string", "null"] does.  Thus the best practice for nullable
>values is to place the null first in the union.  This is what is done by
>the @Nullable annotation.

I have been successfully using default values for union fields with a
default of "null" and the first branch being a record with the Specific
API.
I'm not sure that I understand "The default value for a union is the
default value for its first branch."
Defaults don't apply to any types in a union, only to fields on a record.
So the Schema ["Foo", "Bar"] can have no default, nor can any of its
branches.
A field with that schema as its type can have a default that is either a
"Foo" or a "Bar".

I think we may need to clarify all of this in the spec, the section on
records does not have an example using defaults, or mention what is a
permissible default for a union field.


>
>Perhaps we should clarify this in the Specification?  We might state
>that a "null" schema implicitly has a default value of null since that's
>the only value its ever permitted to have anyway.
>
>> Does reflect have any concept of default values?
>
>No.  We could add an @Default annotation, I suppose.  But I don't think
>this is needed for nullable stuff.
>
>Doug



Re: How to add optional new record fields and/or new methods in avro-ipc?

Posted by Doug Cutting <cu...@apache.org>.
On 10/18/2011 10:09 AM, Scott Carey wrote:
> On 10/18/11 9:47 AM, "Doug Cutting" <cu...@apache.org> wrote:
>> To amend this, you can use Avro's @Nullable annotation:
> 
> The problem is that this does not provide the ability to evolve schemas if
> you add a field
> since you would need @Default or something similar, as well:
> @Nullable
> @Default(null)

I don't think this is required.  The default value for a union is the
default value for its first branch.  A "null" schema needs no default.
So the schema ["null", "string"] needs to specify no default value while
the schema ["string", "null"] does.  Thus the best practice for nullable
values is to place the null first in the union.  This is what is done by
the @Nullable annotation.

Perhaps we should clarify this in the Specification?  We might state
that a "null" schema implicitly has a default value of null since that's
the only value its ever permitted to have anyway.

> Does reflect have any concept of default values?

No.  We could add an @Default annotation, I suppose.  But I don't think
this is needed for nullable stuff.

Doug

Re: How to add optional new record fields and/or new methods in avro-ipc?

Posted by Scott Carey <sc...@apache.org>.
On 10/18/11 9:47 AM, "Doug Cutting" <cu...@apache.org> wrote:

>On 10/17/2011 08:14 PM, 常冰琳 wrote:
>> What I do in the demo is add a new nullable string in server side, not
>> change a string to nullable string.
>> I add a new field with default value using specific, and it works fine,
>> so I suspect the reason that reflect doesn't work is that I didn't add
>> default value to the nullable string field.
>> Perhaps the default value for nullable field should be null by default?
>
>Reflect by default assumes that all values are not nullable.  This is
>perhaps a bug, but the alternative is to make every non-numeric value
>nullable, which would result in verbose schemas.
>
>To amend this, you can use Avro's @Nullable annotation:
>
>http://avro.apache.org/docs/current/api/java/org/apache/avro/reflect/Nulla
>ble.html
>
>This can be applied to parameters, return types and fields.
>
>For example:
>
>import org.apache.avro.reflect.Nullable;
>
>public class Foo {
>  @Nullable String x;
>  public void setX(@Nullable String x) { this.x = x; }
>  @Nullable public String getX() { return x; }
>}


The problem is that this does not provide the ability to evolve schemas if
you add a field
since you would need @Default or something similar, as well:
@Nullable
@Default(null)

Does reflect have any concept of default values?



>
>Doug



Re: How to add optional new record fields and/or new methods in avro-ipc?

Posted by Doug Cutting <cu...@apache.org>.
On 10/17/2011 08:14 PM, 常冰琳 wrote:
> What I do in the demo is add a new nullable string in server side, not
> change a string to nullable string. 
> I add a new field with default value using specific, and it works fine,
> so I suspect the reason that reflect doesn't work is that I didn't add
> default value to the nullable string field. 
> Perhaps the default value for nullable field should be null by default?

Reflect by default assumes that all values are not nullable.  This is
perhaps a bug, but the alternative is to make every non-numeric value
nullable, which would result in verbose schemas.

To amend this, you can use Avro's @Nullable annotation:

http://avro.apache.org/docs/current/api/java/org/apache/avro/reflect/Nullable.html

This can be applied to parameters, return types and fields.

For example:

import org.apache.avro.reflect.Nullable;

public class Foo {
  @Nullable String x;
  public void setX(@Nullable String x) { this.x = x; }
  @Nullable public String getX() { return x; }
}

Doug

Re: How to add optional new record fields and/or new methods in avro-ipc?

Posted by 常冰琳 <de...@gmail.com>.
One more thing, how to add default value information in reflect?

On Tue, Oct 18, 2011 at 11:14 AM, 常冰琳 <de...@gmail.com> wrote:

> This should work in general.  An avro field can be promoted from a String
>> to a nullable String.  The inverse is not true.  A client that is expecting
>> a non-nullable string field response cannot read data serialized with a
>> nullable string.
>
>
> What I do in the demo is add a new nullable string in server side, not
> change a string to nullable string.
> I add a new field with default value using specific, and it works fine, so
> I suspect the reason that reflect doesn't work is that I didn't add default
> value to the nullable string field.
> Perhaps the default value for nullable field should be null by default?
>
>
> 2011/10/17 Scott Carey <sc...@apache.org>
>
>> On 10/13/11 5:25 AM, "常冰琳" <de...@gmail.com> wrote:
>>
>> Hi,
>> I try to use avro-ipc in my project and got a problem, if the server side
>> needs to upgrade,
>> add a few optional field to Request/Response object, or add a new method,
>> thus change the
>> schema.
>> I use reflect, add a new String field in Request class with @Nullable in
>> server side, keep client
>> side unchanged, but it won't to work.
>> I think protobuf supports this, just by adding new optional fields.
>> So does it possible or how to support backward compatibility in other way?
>>
>>
>>
>>  This should work in general.  An avro field can be promoted from a
>> String to a nullable String.  The inverse is not true.  A client that is
>> expecting a non-nullable string field response cannot read data serialized
>> with a nullable string.
>>
>> I am not familiar enough with the Protocol requestor or how it interacts
>> with the Reflect API to spot the issue.  I can't tell from the stack trace
>> if a resolving decoder is in use — which is required to resolve from one
>> schema to another.
>>
>> If you want the server side to evolve you can:
>> add new fields
>> remove fields (only if the field has a default on the client side)
>> promote fields as permitted (int -> long; string -> nullable string; etc.)
>> rename fields or other named types with aliases.
>>
>> Here is the protocol class
>>
>>
>> old:
>>
>>
>>
>> public interface SampleService {
>>   public static class Request {
>>     int id;
>>     String name;
>>     @Nullable
>>     String address;
>>
>>     public Request() {
>>       this(0, "null", null);
>>     }
>>
>>     public Request(int id, String name, String address) {
>>       this.id = id;
>>       this.name = name;
>>       this.address = address;
>>     }
>>
>>     public int getId() {
>>       return id;
>>     }
>>
>>     public String getName() {
>>       return name;
>>     }
>>
>>     public String getAddress() {
>>       return address;
>>     }
>>
>>   }
>>
>>   public static class Response {
>>     @Nullable
>>     String value;
>>
>>     public Response() {
>>       this("null");
>>     }
>>     public Response(String value) {
>>       this.value = value;
>>     }
>>
>>     public String getValue() {
>>       return value;
>>     }
>>   }
>>
>>   int fibonacci(int n);
>>   int sum(int [] nums, int base);
>>   Response complexCall(Request req);
>> }
>>
>>
>>
>>
>> new:
>>
>>
>> public interface SampleService {
>>   public static class Request {
>>     int id;
>>     String name;
>>     @Nullable
>>     String address;
>>     @Nullable
>>     String address2;
>>
>>     public Request() {
>>       this(0, "null", null, null);
>>     }
>>
>>     public Request(int id, String name, String address, String address2) {
>>       this.id = id;
>>       this.name = name;
>>       this.address = address;
>>       this.address2 = address2;
>>     }
>>
>>     public int getId() {
>>       return id;
>>     }
>>
>>     public String getName() {
>>       return name;
>>     }
>>
>>     public String getAddress() {
>>       return address;
>>     }
>>
>>     public String getAddress2() {
>>       return address2;
>>     }
>>
>>   }
>>
>>   public static class Response {
>>     @Nullable
>>     String value;
>>
>>     public Response() {
>>       this("null");
>>     }
>>     public Response(String value) {
>>       this.value = value;
>>     }
>>
>>     public String getValue() {
>>       return value;
>>     }
>>   }
>>
>>   int fibonacci(int n);
>>   int sum(int [] nums, int base);
>>   Response complexCall(Request req);
>> }
>>
>>
>> Here is the error message:
>>
>> Exception in thread "main" org.apache.avro.AvroRuntimeException:
>> org.apache.avro.AvroTypeException: Found {
>>   "type" : "record",
>>   "name" : "Request",
>>   "namespace" : "com.mycompany.app.SampleService$",
>>   "fields" : [ {
>>     "name" : "id",
>>     "type" : "int"
>>   }, {
>>     "name" : "name",
>>     "type" : "string"
>>   }, {
>>     "name" : "address",
>>     "type" : [ "null", "string" ]
>>   } ]
>> }, expecting {
>>   "type" : "record",
>>   "name" : "Request",
>>   "namespace" : "com.mycompany.app.SampleService$",
>>   "fields" : [ {
>>     "name" : "id",
>>     "type" : "int"
>>   }, {
>>     "name" : "name",
>>     "type" : "string"
>>   }, {
>>     "name" : "address",
>>     "type" : [ "null", "string" ]
>>   }, {
>>     "name" : "address2",
>>     "type" : [ "null", "string" ]
>>   } ]
>> }
>> at
>> org.apache.avro.ipc.specific.SpecificRequestor.readError(SpecificRequestor.java:126)
>>  at
>> org.apache.avro.ipc.Requestor$Response.getResponse(Requestor.java:555)
>> at
>> org.apache.avro.ipc.Requestor$TransceiverCallback.handleResult(Requestor.java:360)
>>  at
>> org.apache.avro.ipc.Requestor$TransceiverCallback.handleResult(Requestor.java:323)
>> at
>> org.apache.avro.ipc.NettyTransceiver$NettyClientAvroHandler.messageReceived(NettyTransceiver.java:382)
>>  at
>> org.jboss.netty.channel.SimpleChannelUpstreamHandler.handleUpstream(SimpleChannelUpstreamHandler.java:80)
>> at
>> org.apache.avro.ipc.NettyTransceiver$NettyClientAvroHandler.handleUpstream(NettyTransceiver.java:364)
>>  at
>> org.jboss.netty.channel.DefaultChannelPipeline.sendUpstream(DefaultChannelPipeline.java:545)
>> at
>> org.jboss.netty.channel.DefaultChannelPipeline$DefaultChannelHandlerContext.sendUpstream(DefaultChannelPipeline.java:754)
>>  at
>> org.jboss.netty.channel.Channels.fireMessageReceived(Channels.java:302)
>> at
>> org.jboss.netty.handler.codec.frame.FrameDecoder.unfoldAndFireMessageReceived(FrameDecoder.java:317)
>>  at
>> org.jboss.netty.handler.codec.frame.FrameDecoder.callDecode(FrameDecoder.java:299)
>> at
>> org.jboss.netty.handler.codec.frame.FrameDecoder.messageReceived(FrameDecoder.java:216)
>>  at
>> org.jboss.netty.channel.SimpleChannelUpstreamHandler.handleUpstream(SimpleChannelUpstreamHandler.java:80)
>> at
>> org.jboss.netty.channel.DefaultChannelPipeline.sendUpstream(DefaultChannelPipeline.java:545)
>>  at
>> org.jboss.netty.channel.DefaultChannelPipeline.sendUpstream(DefaultChannelPipeline.java:540)
>> at org.jboss.netty.channel.Channels.fireMessageReceived(Channels.java:274)
>>  at
>> org.jboss.netty.channel.Channels.fireMessageReceived(Channels.java:261)
>> at org.jboss.netty.channel.socket.nio.NioWorker.read(NioWorker.java:349)
>>  at
>> org.jboss.netty.channel.socket.nio.NioWorker.processSelectedKeys(NioWorker.java:280)
>> at org.jboss.netty.channel.socket.nio.NioWorker.run(NioWorker.java:200)
>>  at
>> org.jboss.netty.util.ThreadRenamingRunnable.run(ThreadRenamingRunnable.java:108)
>> at
>> org.jboss.netty.util.internal.DeadLockProofWorker$1.run(DeadLockProofWorker.java:44)
>>  at
>> java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
>> at
>> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
>>  at java.lang.Thread.run(Thread.java:680)
>>
>>
>

Re: How to add optional new record fields and/or new methods in avro-ipc?

Posted by 常冰琳 <de...@gmail.com>.
>
> This should work in general.  An avro field can be promoted from a String
> to a nullable String.  The inverse is not true.  A client that is expecting
> a non-nullable string field response cannot read data serialized with a
> nullable string.


What I do in the demo is add a new nullable string in server side, not
change a string to nullable string.
I add a new field with default value using specific, and it works fine, so I
suspect the reason that reflect doesn't work is that I didn't add default
value to the nullable string field.
Perhaps the default value for nullable field should be null by default?


2011/10/17 Scott Carey <sc...@apache.org>

> On 10/13/11 5:25 AM, "常冰琳" <de...@gmail.com> wrote:
>
> Hi,
> I try to use avro-ipc in my project and got a problem, if the server side
> needs to upgrade,
> add a few optional field to Request/Response object, or add a new method,
> thus change the
> schema.
> I use reflect, add a new String field in Request class with @Nullable in
> server side, keep client
> side unchanged, but it won't to work.
> I think protobuf supports this, just by adding new optional fields.
> So does it possible or how to support backward compatibility in other way?
>
>
>
> This should work in general.  An avro field can be promoted from a String
> to a nullable String.  The inverse is not true.  A client that is expecting
> a non-nullable string field response cannot read data serialized with a
> nullable string.
>
> I am not familiar enough with the Protocol requestor or how it interacts
> with the Reflect API to spot the issue.  I can't tell from the stack trace
> if a resolving decoder is in use — which is required to resolve from one
> schema to another.
>
> If you want the server side to evolve you can:
> add new fields
> remove fields (only if the field has a default on the client side)
> promote fields as permitted (int -> long; string -> nullable string; etc.)
> rename fields or other named types with aliases.
>
> Here is the protocol class
>
>
> old:
>
>
>
> public interface SampleService {
>   public static class Request {
>     int id;
>     String name;
>     @Nullable
>     String address;
>
>     public Request() {
>       this(0, "null", null);
>     }
>
>     public Request(int id, String name, String address) {
>       this.id = id;
>       this.name = name;
>       this.address = address;
>     }
>
>     public int getId() {
>       return id;
>     }
>
>     public String getName() {
>       return name;
>     }
>
>     public String getAddress() {
>       return address;
>     }
>
>   }
>
>   public static class Response {
>     @Nullable
>     String value;
>
>     public Response() {
>       this("null");
>     }
>     public Response(String value) {
>       this.value = value;
>     }
>
>     public String getValue() {
>       return value;
>     }
>   }
>
>   int fibonacci(int n);
>   int sum(int [] nums, int base);
>   Response complexCall(Request req);
> }
>
>
>
>
> new:
>
>
> public interface SampleService {
>   public static class Request {
>     int id;
>     String name;
>     @Nullable
>     String address;
>     @Nullable
>     String address2;
>
>     public Request() {
>       this(0, "null", null, null);
>     }
>
>     public Request(int id, String name, String address, String address2) {
>       this.id = id;
>       this.name = name;
>       this.address = address;
>       this.address2 = address2;
>     }
>
>     public int getId() {
>       return id;
>     }
>
>     public String getName() {
>       return name;
>     }
>
>     public String getAddress() {
>       return address;
>     }
>
>     public String getAddress2() {
>       return address2;
>     }
>
>   }
>
>   public static class Response {
>     @Nullable
>     String value;
>
>     public Response() {
>       this("null");
>     }
>     public Response(String value) {
>       this.value = value;
>     }
>
>     public String getValue() {
>       return value;
>     }
>   }
>
>   int fibonacci(int n);
>   int sum(int [] nums, int base);
>   Response complexCall(Request req);
> }
>
>
> Here is the error message:
>
> Exception in thread "main" org.apache.avro.AvroRuntimeException:
> org.apache.avro.AvroTypeException: Found {
>   "type" : "record",
>   "name" : "Request",
>   "namespace" : "com.mycompany.app.SampleService$",
>   "fields" : [ {
>     "name" : "id",
>     "type" : "int"
>   }, {
>     "name" : "name",
>     "type" : "string"
>   }, {
>     "name" : "address",
>     "type" : [ "null", "string" ]
>   } ]
> }, expecting {
>   "type" : "record",
>   "name" : "Request",
>   "namespace" : "com.mycompany.app.SampleService$",
>   "fields" : [ {
>     "name" : "id",
>     "type" : "int"
>   }, {
>     "name" : "name",
>     "type" : "string"
>   }, {
>     "name" : "address",
>     "type" : [ "null", "string" ]
>   }, {
>     "name" : "address2",
>     "type" : [ "null", "string" ]
>   } ]
> }
> at
> org.apache.avro.ipc.specific.SpecificRequestor.readError(SpecificRequestor.java:126)
>  at org.apache.avro.ipc.Requestor$Response.getResponse(Requestor.java:555)
> at
> org.apache.avro.ipc.Requestor$TransceiverCallback.handleResult(Requestor.java:360)
>  at
> org.apache.avro.ipc.Requestor$TransceiverCallback.handleResult(Requestor.java:323)
> at
> org.apache.avro.ipc.NettyTransceiver$NettyClientAvroHandler.messageReceived(NettyTransceiver.java:382)
>  at
> org.jboss.netty.channel.SimpleChannelUpstreamHandler.handleUpstream(SimpleChannelUpstreamHandler.java:80)
> at
> org.apache.avro.ipc.NettyTransceiver$NettyClientAvroHandler.handleUpstream(NettyTransceiver.java:364)
>  at
> org.jboss.netty.channel.DefaultChannelPipeline.sendUpstream(DefaultChannelPipeline.java:545)
> at
> org.jboss.netty.channel.DefaultChannelPipeline$DefaultChannelHandlerContext.sendUpstream(DefaultChannelPipeline.java:754)
>  at
> org.jboss.netty.channel.Channels.fireMessageReceived(Channels.java:302)
> at
> org.jboss.netty.handler.codec.frame.FrameDecoder.unfoldAndFireMessageReceived(FrameDecoder.java:317)
>  at
> org.jboss.netty.handler.codec.frame.FrameDecoder.callDecode(FrameDecoder.java:299)
> at
> org.jboss.netty.handler.codec.frame.FrameDecoder.messageReceived(FrameDecoder.java:216)
>  at
> org.jboss.netty.channel.SimpleChannelUpstreamHandler.handleUpstream(SimpleChannelUpstreamHandler.java:80)
> at
> org.jboss.netty.channel.DefaultChannelPipeline.sendUpstream(DefaultChannelPipeline.java:545)
>  at
> org.jboss.netty.channel.DefaultChannelPipeline.sendUpstream(DefaultChannelPipeline.java:540)
> at org.jboss.netty.channel.Channels.fireMessageReceived(Channels.java:274)
>  at
> org.jboss.netty.channel.Channels.fireMessageReceived(Channels.java:261)
> at org.jboss.netty.channel.socket.nio.NioWorker.read(NioWorker.java:349)
>  at
> org.jboss.netty.channel.socket.nio.NioWorker.processSelectedKeys(NioWorker.java:280)
> at org.jboss.netty.channel.socket.nio.NioWorker.run(NioWorker.java:200)
>  at
> org.jboss.netty.util.ThreadRenamingRunnable.run(ThreadRenamingRunnable.java:108)
> at
> org.jboss.netty.util.internal.DeadLockProofWorker$1.run(DeadLockProofWorker.java:44)
>  at
> java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
> at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
>  at java.lang.Thread.run(Thread.java:680)
>
>

Re: How to add optional new record fields and/or new methods in avro-ipc?

Posted by Doug Cutting <cu...@apache.org>.
On 10/16/2011 11:29 PM, Scott Carey wrote:
> If you want the server side to evolve you can:
> add new fields
> remove fields (only if the field has a default on the client side)
> promote fields as permitted (int -> long; string -> nullable string; etc.)
> rename fields or other named types with aliases.

Should we add this summary to the documentation somewhere?  Perhaps the
specification?

The other item to note is that you can add new messages to protocols but
shouldn't remove them or change their semantics incompatibly, and that
message parameters should be treated like fields above.

Doug

Re: How to add optional new record fields and/or new methods in avro-ipc?

Posted by Scott Carey <sc...@apache.org>.
On 10/13/11 5:25 AM, "常冰琳" <de...@gmail.com> wrote:

> Hi,
> I try to use avro-ipc in my project and got a problem, if the server side
> needs to upgrade,
> add a few optional field to Request/Response object, or add a new method, thus
> change the 
> schema. 
> I use reflect, add a new String field in Request class with @Nullable in
> server side, keep client
> side unchanged, but it won't to work.
> I think protobuf supports this, just by adding new optional fields.
> So does it possible or how to support backward compatibility in other way?

> 
This should work in general.  An avro field can be promoted from a String to
a nullable String.  The inverse is not true.  A client that is expecting a
non-nullable string field response cannot read data serialized with a
nullable string.   

I am not familiar enough with the Protocol requestor or how it interacts
with the Reflect API to spot the issue.  I can't tell from the stack trace
if a resolving decoder is in use ― which is required to resolve from one
schema to another.

If you want the server side to evolve you can:
add new fields
remove fields (only if the field has a default on the client side)
promote fields as permitted (int -> long; string -> nullable string; etc.)
rename fields or other named types with aliases.

> Here is the protocol class
> 
> 
> old:
> 
> 
> 
> public interface SampleService {
>   public static class Request {
>     int id;
>     String name;
>     @Nullable
>     String address;
> 
>     public Request() {
>       this(0, "null", null);
>     }
> 
>     public Request(int id, String name, String address) {
>       this.id <http://this.id>  = id;
>       this.name <http://this.name>  = name;
>       this.address = address;
>     }
> 
>     public int getId() {
>       return id;
>     }
> 
>     public String getName() {
>       return name;
>     }
> 
>     public String getAddress() {
>       return address;
>     }
>     
>   }
>   
>   public static class Response {
>     @Nullable
>     String value;
> 
>     public Response() {
>       this("null");
>     }
>     public Response(String value) {
>       this.value = value;
>     }
> 
>     public String getValue() {
>       return value;
>     }
>   }
> 
>   int fibonacci(int n);
>   int sum(int [] nums, int base);
>   Response complexCall(Request req);
> }
> 
> 
> 
> 
> new:
> 
> 
> public interface SampleService {
>   public static class Request {
>     int id;
>     String name;
>     @Nullable
>     String address;
>     @Nullable
>     String address2;
> 
>     public Request() {
>       this(0, "null", null, null);
>     }
> 
>     public Request(int id, String name, String address, String address2) {
>       this.id <http://this.id>  = id;
>       this.name <http://this.name>  = name;
>       this.address = address;
>       this.address2 = address2;
>     }
> 
>     public int getId() {
>       return id;
>     }
> 
>     public String getName() {
>       return name;
>     }
> 
>     public String getAddress() {
>       return address;
>     }
> 
>     public String getAddress2() {
>       return address2;
>     }
>     
>   }
>   
>   public static class Response {
>     @Nullable
>     String value;
> 
>     public Response() {
>       this("null");
>     }
>     public Response(String value) {
>       this.value = value;
>     }
> 
>     public String getValue() {
>       return value;
>     }
>   }
> 
>   int fibonacci(int n);
>   int sum(int [] nums, int base);
>   Response complexCall(Request req);
> }
> 
> 
> Here is the error message:
> 
> Exception in thread "main" org.apache.avro.AvroRuntimeException:
> org.apache.avro.AvroTypeException: Found {
>   "type" : "record",
>   "name" : "Request",
>   "namespace" : "com.mycompany.app.SampleService$",
>   "fields" : [ {
>     "name" : "id",
>     "type" : "int"
>   }, {
>     "name" : "name",
>     "type" : "string"
>   }, {
>     "name" : "address",
>     "type" : [ "null", "string" ]
>   } ]
> }, expecting {
>   "type" : "record",
>   "name" : "Request",
>   "namespace" : "com.mycompany.app.SampleService$",
>   "fields" : [ {
>     "name" : "id",
>     "type" : "int"
>   }, {
>     "name" : "name",
>     "type" : "string"
>   }, {
>     "name" : "address",
>     "type" : [ "null", "string" ]
>   }, {
>     "name" : "address2",
>     "type" : [ "null", "string" ]
>   } ]
> }
> at 
> org.apache.avro.ipc.specific.SpecificRequestor.readError(SpecificRequestor.jav
> a:126)
> at org.apache.avro.ipc.Requestor$Response.getResponse(Requestor.java:555)
> at 
> org.apache.avro.ipc.Requestor$TransceiverCallback.handleResult(Requestor.java:
> 360)
> at 
> org.apache.avro.ipc.Requestor$TransceiverCallback.handleResult(Requestor.java:
> 323)
> at 
> org.apache.avro.ipc.NettyTransceiver$NettyClientAvroHandler.messageReceived(Ne
> ttyTransceiver.java:382)
> at 
> org.jboss.netty.channel.SimpleChannelUpstreamHandler.handleUpstream(SimpleChan
> nelUpstreamHandler.java:80)
> at 
> org.apache.avro.ipc.NettyTransceiver$NettyClientAvroHandler.handleUpstream(Net
> tyTransceiver.java:364)
> at 
> org.jboss.netty.channel.DefaultChannelPipeline.sendUpstream(DefaultChannelPipe
> line.java:545)
> at 
> org.jboss.netty.channel.DefaultChannelPipeline$DefaultChannelHandlerContext.se
> ndUpstream(DefaultChannelPipeline.java:754)
> at org.jboss.netty.channel.Channels.fireMessageReceived(Channels.java:302)
> at 
> org.jboss.netty.handler.codec.frame.FrameDecoder.unfoldAndFireMessageReceived(
> FrameDecoder.java:317)
> at 
> org.jboss.netty.handler.codec.frame.FrameDecoder.callDecode(FrameDecoder.java:
> 299)
> at 
> org.jboss.netty.handler.codec.frame.FrameDecoder.messageReceived(FrameDecoder.
> java:216)
> at 
> org.jboss.netty.channel.SimpleChannelUpstreamHandler.handleUpstream(SimpleChan
> nelUpstreamHandler.java:80)
> at 
> org.jboss.netty.channel.DefaultChannelPipeline.sendUpstream(DefaultChannelPipe
> line.java:545)
> at 
> org.jboss.netty.channel.DefaultChannelPipeline.sendUpstream(DefaultChannelPipe
> line.java:540)
> at org.jboss.netty.channel.Channels.fireMessageReceived(Channels.java:274)
> at org.jboss.netty.channel.Channels.fireMessageReceived(Channels.java:261)
> at org.jboss.netty.channel.socket.nio.NioWorker.read(NioWorker.java:349)
> at 
> org.jboss.netty.channel.socket.nio.NioWorker.processSelectedKeys(NioWorker.jav
> a:280)
> at org.jboss.netty.channel.socket.nio.NioWorker.run(NioWorker.java:200)
> at 
> org.jboss.netty.util.ThreadRenamingRunnable.run(ThreadRenamingRunnable.java:10
> 8)
> at 
> org.jboss.netty.util.internal.DeadLockProofWorker$1.run(DeadLockProofWorker.ja
> va:44)
> at 
> java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java
> :886)
> at 
> 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908>
)
> at java.lang.Thread.run(Thread.java:680)
>