On 06/07/2010 01:53 PM, Bill de hOra wrote:
I.m wondering if I'm missing an incantation for 'compile
schema/protocol' in Avro tools to handle defaults. Given this schema,
In Avro, default values are used by a reader when a writer's schema
lacks a field. This permits one to add a new field and
On 06/07/2010 03:11 PM, Bill de hOra wrote:
This means writers can't leverage schema defaults, so writers should do
something like this?
Message message = new Message();
// no defaults set
String quux = message
.getSchema()
.getField("foo")
.defaultValue()
.getTextValue();
message.foo=new Utf8(q
On 06/07/2010 03:40 PM, Scott Carey wrote:
If the specific compiler generated a couple constructors --
* A default empty argument constructor -- fills fields with defaults.
* A constructor with all fields passed in -- assigns fields from the
constructor and does nothing with defaults.
Then I ca
On 06/08/2010 11:10 AM, Markus Weimer wrote:
Is there a way to "stream" the doubles into the output without holding a
copy in memory? Or is there another way to encode a double[] in a schema?
Avro arrays and maps are written in a blocked representation, so the
binary encoding does support arbi
On 06/10/2010 09:27 AM, Scott Carey wrote:
I propose:
* We use getters/setters, and allow setters and constructors to be
package protected with an option so the objects can be safely passed
around outside the package without stateful wrapper objects. Users
can write factory methods or other stat
n the enum. At code generation
time we can simply write, e.g., Integer.class wherever you put INT.type.
Doug
On 06/10/2010 02:09 PM, Scott Carey wrote:
On Jun 10, 2010, at 11:49 AM, Doug Cutting wrote:
On 06/10/2010 09:27 AM, Scott Carey wrote:
I propose:
* We use getters/setters, and al
On 06/10/2010 02:13 PM, Scott Carey wrote:
Also, it would be good to have the type state in one place atomically
-- the value itself. If its in two places then race conditions
become a problem.
That would be a nice feature, but I don't see it as a requirement. A
setter method need not always
On 06/10/2010 03:44 PM, Scott Carey wrote:
The getBar$Type() requires the cascaded if/else though.
Right, that's the cost I was alluding too. Some unions are reasonably
large, e.g., http://s.apache.org/Events.avpr, where the union at the end
has 16 branches, and the cost of the cascaded if/e
On 06/14/2010 02:22 PM, R. Tyler Ballance wrote:
In my case, I already have "string" as part of my errors union (passing back
errors as JSON in some cases) so now my schemas rase SchemaParseExceptions
(http://avro.pastebin.com/pfSempR7).
The intent in the spec was that user errors would all be
Sadayuki,
Avro does not currently support such an API, but it might not be
difficult to add one and it would be very useful to have.
The Transciever API might to be extended with a method like:
Future> transceiveFuture(List);
and the Requestor API might add a method like:
Future requestFutu
ign.
I'll try to test how much performance changes in actuality using another system.
Thanks,
Sadayuki
--
Sadayuki Furuhashi
frsy...@gmail.com
http://msgpack.sourceforge.net/
http://kumofs.sourceforge.net/
On 2010/06/15, at 11:51, Doug Cutting wrote:
Sadayuki,
Avro does not currently supp
xer.add(f3);
// proceeds I/Os using Selector.
multiplexer.run();
// gathers results.
ResultChunk result1 = f1.get();
ResultChunk result2 = f2.get();
ResultChunk result3 = f3.get();
Thanks,
Sadayuki
On 2010/06/15, at 16:10, Doug Cutting wrote:
Sadayuki,
Couldn&
This should work.
There are existing tests for schema properties are in TestSchema.java.
The checkProp() method is called to add a property to every type of
schema, print the schema with that property, then re-parse it and check
that the property is still there.
Can you provide a complete te
On 07/19/2010 11:43 AM, Ran Tavory wrote:
I'd like to use avro in several use cases:
1. RPC. Both using json over HTTP and binary over TCP (will use both
modes). Async RPC clients/servers isn't a showstopper but could also be nice
Binary RPC over HTTP is the supported mode today. We'd eventual
On 07/22/2010 11:40 AM, Sharad Agarwal wrote:
I don't find a way to to share complex type in more than one protocol. I
want to do something like:
@namespace ("x.y")
protocol Protocol1 {
record MyRecord {
string name;
}
void message1(MyRecord r);
}
@namespace ("x.y.z")
protocol Protocol2 {
void
David,
The Java implementation currently preserves extra JSON values at the
schema level, but not at the field level. It wouldn't be hard to change
this. Please file an issue in Jira if this would be useful to you.
Thanks,
Doug
On 07/27/2010 01:33 PM, David Rosenstrauch wrote:
It looks a
On 07/27/2010 03:15 PM, Sharad Agarwal wrote:
Is there a reason for SpecificCompiler to use GenericArray rather than
java's native array or list ?
GenericArray is designed to support object reuse, while native arrays
and List make reuse difficult. Probably GenericArray should be made to
impl
On 07/28/2010 02:07 AM, Nick Palmer wrote:
It would be very nice if GenericArray implemented List. I need get,
set, and remove in GenericData.Array for my application and have
already added these to my Avro code so I can continue developing. I
was planning to file a patch in JIRA for this change.
The sourceComponentTypeField should look something like:
{"name": "sourceComponentType",
"type": {"type": "enum", "symbols": [ ... ] }
}
This is the same error message that confused Sam in:
https://issues.apache.org/jira/browse/AVRO-583
Doug
On 08/02/2010 08:46 AM, Tim Robertson wrote:
Hi a
On 08/02/2010 09:51 AM, Tim Robertson wrote:
http://www.mail-archive.com/avro-...@hadoop.apache.org/msg04063.html
tells me that defaults aren't used at write time, but it means that I
have to initiate all the Strings with "=new Utf8();" each time I
create a message payload - is this correct?
De
That sounds like something that should work. Can you submit a bug
report, ideally with a complete test case? Thanks!
Doug
On 08/02/2010 04:28 PM, Markus Weimer wrote:
Hi,
I added the following line to a schema, recreated the static java classes
for it and compiled my code:
{"name": "bias",
On 08/12/2010 05:10 PM, David Rosenstrauch wrote:
Just wondering: is there a release date planned yet for Avro 1.4.0?
Not a hard date, but I'll be very disappointed if we don't get something
out this month. I've got a few more Java things that I hope to wrap up
in the next week. Other than
On 08/16/2010 11:34 AM, John Kristian wrote:
It seems tricky to add a nullable field with a default (for compatibility
with previous versions). I find that the type of the default value must be
declared first in the union. For example, this works:
{"name":"myField", "type":["int", "null"], "de
On 08/16/2010 01:08 PM, John Kristian wrote:
It seems like it would better for schema parsing (or perhaps resolution) to
fail if the default value doesn't match the first type in the union. That
would require people to clarify existing schemas. Would it cause trouble in
other ways?
Defaults c
On 08/18/2010 10:18 AM, ey-chih chow wrote:
Thanks. But by doing this way, what kind of advantage we can get from Avro?
The Avro MapReduce API is easiest to use when both inputs and outputs
are Avro data.
If inputs are not Avro data, but you want to use the rest of the Avro MR
API, then you
On 08/18/2010 10:49 AM, Harsh J wrote:
We hope to add more such tools for such conversion/ingest, e.g.:
https://issues.apache.org/jira/browse/AVRO-458
Offtopic, but is there any work being done on this already? I saw one
of them tagged with 'GSOC', so wish to know before I sink something
down.
On 08/16/2010 02:46 PM, Patrick Linehan wrote:
does anyone have any suggestions for dealing with large lists/arrays of
primitive values in avro?
in my case (numerical algorithms), my naive mapping of a vector type
(mathematical vectors, not java Vectors) to an avro specific type
generates a Gene
Alex,
This search is much more comprehensive, since it combines mailing lists,
website, wiki, etc., so I'd be in favor of switching the Avro website to
use it. Please submit a patch in Jira.
Thanks,
Doug
On 08/22/2010 07:06 AM, Alex Baranau wrote:
Hello,
Over at http://search-hadoop.com
ain/webapp/resources/schema/relaxng/sitemap-v06.rng:2097:30:
error: datatype library "http://www.w3.org/2001/XMLSchema-datatypes"; not
recognized
[exec]
/root/sematext/tools/apache-forrest-0.8/main/webapp/resources/schema/relaxng/sitemap-v06.rng:2107:29:
error: datatype library "
On 08/25/2010 09:27 AM, Jeff Zhang wrote:
Since the purpose of named type is to allow it to be reference in
other types, and Record, enums and fixed are named types, so why Array
and map is not named type ? What's the design consideration ? I think
if array and map is named type, I can create mor
On 08/25/2010 05:54 PM, R. Tyler Ballance wrote:
I'm wondering if there was any specific reason for using httplib over urllib2
in the first place?
Not that I know of, and hammer's on vacation.
[0] http://github.com/rtyler/avro/tree/1.3.3-redirect
This is mostly compatible, since the use of
Field attributes are supported in Avro 1.4.
https://issues.apache.org/jira/browse/AVRO-601
Doug
On 08/31/2010 11:55 AM, Yang wrote:
sometimes we need a little bit of extra info about a Schema Field.
for example, for a string field, we could say "the max length is 100
chars",
or we could record
On 09/03/2010 07:09 PM, Eric Evans wrote:
This covers Java when generated classes are used, but the other writers
would need to be changed as well. I'm willing to work on patches for
this, is this still something you'd considering merging? Should it be
made optional?
I'd love to see a feature
Avro release 1.4.0 is now available.
Release highlights include:
- PHP implementation of Avro
- RPC tracing for Java
- Java MapReduce API for Avro data
- Utf8 and GenericArray now implement standard Java interfaces
- GenAvro renamed Avro IDL, with many new features added
Full changes are lis
On 09/13/2010 03:53 PM, Lin Guo wrote:
I am wondering why Avro has a special type NULL instead of allowing
each type has null value? What benefit can we gain from this design?
When serialized, if any value may be null then it must be noted that it
is non-null, adding at least a bit to the size
On 09/16/2010 09:25 AM, Scott Carey wrote:
Generally, Avro recommends storing the schema with the data. For a
file that means in the header of the file, for a key/value store that
means in some system metadata. Any individual store can only keep
data serialized with one schema.
Another good
On 10/01/2010 05:45 PM, Patrick Linehan wrote:
am i misunderstanding the documentation? is the behavior i'm seeing
expected? when does a record name/namespace conflict actually cause an
error to be thrown?
The alias feature in Avro 1.4 will let you read records whose name or
namespace differ
On 10/03/2010 07:39 PM, Scott Banachowski wrote:
Hi. I have some uncommitted code that is a start. I haven't had time to
work on it lately, but I can send you a patch if you want to work with it.
Scott, maybe you could attach the code to a Jira issue?
Thanks,
Doug
On 10/04/2010 02:06 PM, Patrick Linehan wrote:
is this the expected behavior in this case? does this not seem to
contradict the schema resolution portions of the spec?
This is not the expected behavior. The expected behavior is that names
must match, but that doesn't seem to be enforced in t
On 10/05/2010 12:08 PM, Patrick Linehan wrote:
should i file a bug about the current behavior, then? i'd be happy to
do so, and i now have a fully-working self-contained code sample which
demonstrates the problem.
Sure, that'd be good. Thanks!
Doug
On 10/25/2010 10:10 AM, Alexander Gordeev wrote:
Please tell me if Avro is able to do a two-way RPC over a single TCP
connection? By two-way RPC I mean that both client and server can send
requests to each other at any moment. It was not clear from the
documentation and I didn't also find any inf
On 10/25/2010 11:55 AM, Yuduo wrote:
Therefore, in this case, we need to create two TCP connections for a
two-way RPC, is that correct?
Yes, if each side must be able to initiate calls, both sides must run a
server, listening for connections.
Doug
On 10/29/2010 07:28 PM, Mark Lewandowski wrote:
org.apache.avro.AvroRuntimeException: Not both one-way: append
My guess is that Python is not preserving this attribute when it renders
the protocol as JSON. Also, perhaps it is a bug that Java requires that
this attribute matches even whe
On 11/01/2010 10:50 AM, Doug Cutting wrote:
we could fix this either by
fixing Python to preserve this attribute in the protocol's JSON or for
Java to ignore this attribute when using a stateless transport like HTTP.
A third short-term option might be to remove the one-way specification
On 11/03/2010 11:24 PM, Jeff Hammerbacher wrote:
Yes. See https://issues.apache.org/jira/browse/*AVRO*-495.
Documented at:
http://avro.apache.org/docs/current/idl.html#imports
Doug
On 11/03/2010 03:11 PM, Douglas Campbell wrote:
Are there any tools for this because as far as we can tell 1.2 datafiles
are unreadable by 1.4 apis?
Not that I know of. I didn't think anyone had used the 1.2 datafile
format! But, if some did, a format converter would be a good tool to have.
On 11/06/2010 11:25 AM, Douglas Campbell wrote:
So our logic for replacement could be something like
upon datafile read request,
try to load with 1.4
if fail, try with 1.2 reader you provided,
get records, save out with 1.4 after update
Should that be doable with what you've provided?
Yes. I
On 11/09/2010 02:46 PM, Doug Cutting wrote:
For example, we could add a datafile reader interface that both the 1.2
and 1.4 version implement, then have a factory that creates one or the
other after reading the file header. Then you'd not need to rewrite
files: the Java implementation wou
These are printed because they're logged at the INFO level and the
default logger is slf4j's SimpleLogger:
http://www.slf4j.org/api/org/slf4j/impl/SimpleLogger.html
To disable them you could put a different slf4j logging implementation
jar on your classpath. E.g., instead of slf4j-simple.jar,
On 11/12/2010 03:08 PM, Christopher Hunt wrote:
I'd say that's definitely DEBUG/TRACE level information.
I think it may depend on the application. For server-side and
distributed applications it's common to log each network transaction by
default, no? But if folks feel strongly about it, fe
On 11/22/2010 08:17 PM, Ryan Holmes wrote:
Avro 1.4.1 doesn't appear to be in the Maven Central repo (or any other
repo I can find). Any idea why?
Thanks for noticing this. This was my first release through Nexus. It
looks like I "closed" the artifact but didn't "release" it. I just
releas
On 11/23/2010 01:54 PM, Douglas Campbell wrote:
just logged a vote in favor 692.
I've started work on it again today. I'm making it so that the
command-line DataFileReadTool and MapReduce jobs will both automatically
handle 1.2-format files.
Is there an ETA for 1.5?
I hope to get it rel
On 11/24/2010 06:53 AM, Harsh J wrote:
Is it possible to parse an "anonymous" record schema (back into a
proper Schema object)?
If I've created an anonymous record schema, I'm getting an error (No
"name" found, via Schema.parse() of Java API) when I de-serialize its
JSON to form a Schema object
On 11/28/2010 08:50 PM, David Jeske wrote:
However, it's
important for me to have a contingency plan in case somehow, someday
there is ever corruption that disconnected the schema-ID from the actual
schema.
If this worst-case transpired, I don't think it would be too difficult
for most dataset
On 11/29/2010 11:04 AM, David Jeske wrote:
I don't follow how this would be possible with Avro. With no type
information, how would you tell the difference between an array of ints,
a bunch of enums, a binary chunk of data, or even just a string? Thrift
and Protobufs have the types so understandi
On 01/04/2011 02:53 AM, Gilles Gaillard wrote:
Our requirements do not include to have a full 'block-mode' RPC and
therefore we would be happy with an asynchronous mode where no return
value is expected when a message is sent. So using the IDL, that's
equivalent to defining messages as methods wi
On 12/20/2010 02:30 PM, Chase Bradford wrote:
I get an UnsupportedOperationException when trying to use set() on a
GenericData.Array backed list. Is this by design or can I submit a
patch that implements it?
A patch would be welcome.
Thanks,
Doug
On 12/07/2010 03:06 AM, Xingen Wang wrote:
I just want my rpc framework be able to transport any objects through
network without any modification in the existing schema definition.
Can I achieve this objective with Avro
Perhaps. A way to do this is to define a general schema. If, for
exam
Tatsuya,
Are you sure that the schema in weather.avro file is named
"test.Weather"? In trunk, if SpecificDatumReader cannot find the class
named in a schema, it will use GenericData$Record. What version of Avro
are you using?
Doug
On 01/08/2011 12:35 AM, Tatsuya Mori wrote:
Hi,
I am try
Dev,
What you describe should work. Can you perhaps provide a simple code
example to illustrate the problem you are having?
Thanks,
Doug
On 01/17/2011 04:53 AM, Devajyoti Sarkar wrote:
Hi,
I am just beginning to use Avro, so I apologize if this is a silly question.
I would like to set a
The way that I have imagined doing this is to specify a standard schema
for dates, then implementations can optionally map this to a native date
type.
The schema could be a record containing a long, e.g.:
{"type": "record", "name":"org.apache.avro.lib.Date", "fields" : [
{"name": "time", "ty
On 01/18/2011 09:19 AM, Jeremy Custenborder wrote:
I agree with storing it as a long. How would you handle this in code
generation and serialization? Would you envision hooks during code
generation that would generate a member that is the native date time
for the language?
Yes. Just as "bytes"
On 02/17/2011 01:33 AM, Douglas Campbell wrote:
> Any update on 1.5?
Soon, I hope. I'll revive that discussion on the dev list.
Doug
On 02/18/2011 01:16 PM, Wade Chandler wrote:
> I have had different thoughts. I'm wanting to use as much of Avro which is
> already available as possible. The top two seem to be:
>
> 1) Write my own server implementation which handles multiple responders.
This seems like a sub-optimal approach.
On 03/01/2011 09:05 PM, felix gao wrote:
> I am running some comparison tests on a data set that I converted to
> avro with deflator set to level 6. The original logs consists of 2880
> uncompressed http access logs with a total size of 1.4TB. The Compressed
> avro log is about 2/3 of the size. Ho
On 03/10/2011 08:04 PM, Aleksey Maslov wrote:
> 1. Is it true that all objects MUST be defined in 1 huge file if I want to
> reference previously defined ones?
> (sounds a bit messy if you have dozens of objects to define, but ok);
You can import schemas, protocols and other IDL files into an ID
On 03/11/2011 10:28 AM, Scott Carey wrote:
> You need an array of JSON objects in the avsc file, something like:
An .avsc file can only contain a single top-level JSON object, not an
array. So, if you wanted to define this in an .avsc file then you'd
need to use a nested definition, like:
--
A protocol definition can include multiple schema definitions, where
each schema defined can refer to previously defined schema definitions.
A protocol need not include any messages, but can be used simply as a
vehicle to define a set of schemas.
Avro IDL permits imports of external schemas and p
On 03/11/2011 04:45 PM, Scott Carey wrote:
> One of the .avsc file I use is an array of 7 top level JSon objects,
> starting with:
>
> [
> {"name": "com.rr.avro.Fixed16", "type": "fixed", "size":16},
> {"name": "com.rr.avro.Fixed4", "type": "fixed", "size":4},
> {"name": "com.rr.avro.Variable", "t
Not all JSON parsers are required to support comments. The parser that
Avro Java uses (Jackson) does support comments and we enable that
feature, but I don't know about the parser for Python. However in
general one should probably not rely on comments working in JSON.
Doug
On 03/13/2011 03:01 P
I'd like to announce the availability of Avro release 1.5.0.
Selected highlights of this release include:
INCOMPATIBLE CHANGES
AVRO-647. Java: Break avro.jar up into multiple parts: avro.jar,
avro-compiler.jar, avro-ipc.jar, avro-mapred.jar, avro-tools.jar,
and avro-maven-plugin.ja
The best practice is not to modify the generated classes directly, but
rather to author classes that wrap instances of generated classes,
adding methods that enforce such constraints to those. The wrapper
classes can then also maintain back-compatible APIs. Does that make sense?
Doug
On 03/14/2
On 03/17/2011 08:13 PM, 幻 wrote:
> Currently,I have two avro files with different schema. I found that
> I have to set the schema before running a M/R job if the files are in
> avro format.But the schema of the files are probably not the same.How
> can I do that without setting the schema befo
On 03/18/2011 09:54 AM, Harsh J wrote:
> Would it help if the provided JSON schemae were added to the JobConf
> with the given path(s) as a prefix to the key used to retrieve them?
> This would help use with MultipleInputs and such (but it may get
> complicated to do if globs were involved?).
Not
On 03/18/2011 11:31 AM, Harsh J wrote:
> Probably a small case, in which I would require reading from multiple
> sources in my job (perhaps even process them differently until the Map
> phase), with special reader-schemas for each of my sources.
How would your mapper detect which schema was in use
You can use the schema David provided directly for arrays of the User
record. But if you want to have a named type for an array of User then
you need to define a record with that as a field, e.g., something like:
{"type": "record",
"name": "Users",
"fields": [
"name": "userArray",
"type":
On 03/21/2011 10:47 AM, Curtis Jensen wrote:
> In one of my iterations of trying to get this to work, I tried the
> straight array schema you suggested. However, in Java, how do I
> create a "User" record? After parsing the schema, the only schema
> object I can create from it is an Array object.
Sounds like:
https://issues.apache.org/jira/browse/VELOCITY-720
Which would be reduced to a one-line warning if we upgrade to Velocity 1.7.
Can you please file a bug in Jira for this?
Thanks,
Doug
On 03/21/2011 11:38 AM, Arun Ramakrishnan wrote:
> Hi
> I am encountering this problem with av
Avro in several places requires that schemas are self-contained. For
example, when reading a data file the schema that was used to write it
must be available and should not be dynamically re-constructed from
references to schemas in the reader's environment. So, if such a
registry were implemente
On 03/21/2011 10:46 AM, Aleksey Maslov wrote:
> If I would like to place common data into a parent avro object, and some
> additional detailed data into 2 logically separate children data objects –
> could I specify such relationship of the objects in avro IDL, or I can keep
> the relationship impl
On 03/25/2011 10:34 AM, Aurora Skarra-Gallagher wrote:
> I would like to be able to validate the schema. Is there a way for me to
> take my schema and also a JSON output string from my API and validate
> that the output matches the schema?
I don't know of an explicit command-line validator, but yo
Maps are not a named type in Avro. The "name" field you specify in the
"map" schemas is ignored. Only record, enum, and fixed are named types.
If you wish to refer to a map by name in schemas, define a record with
a map as its single field. This adds no storage overhead: it will
serialize to th
Enrico,
This looks like it could be a bug. What version of Avro are you using?
Can you please file a Jira issue for this?
Thanks,
Doug
On 04/07/2011 03:04 AM, Enrico Saviano wrote:
> Hello,
>
> I'm using Avro RPC software to send messages, via
> SaslSocketServer/Transciever, and the Reflect
On 04/08/2011 01:14 PM, Markus Weimer wrote:
> I seem to hit a case not covered by the mapred package documentation:
> I'd like to read from a TextInputFormat and produce AVRO data in a
> map-only job. How Do I do that?
In short, the way to do this is to:
- use a org.apache.hadoop.mapred.Mapper,Nu
On 04/12/2011 02:18 PM, Markus Weimer wrote:
>> In short, the way to do this is to:
>> - use a org.apache.hadoop.mapred.Mapper,NullWritable>
>> - call AvroJob.setOutputSchema(job,schema) with O's schema
>>
>> Does that make sense? If that works for you, I can add it to the
>> javadoc.
>
> Yes, it
What output schema did you specify for the job?
Doug
On 04/13/2011 04:29 AM, Felix.徐 wrote:
> Hi,all. I try to use GenericRecord as value in mapreduce:
>
> public void map(GenericRecord record,
> AvroCollector> collector, Reporter reporter)
>
> collector.collect(new Pair(new Utf8(key),valu
It's hard to tell what's wrong without a complete program and a stack
trace, but I believe this error message is from the GenericData.Record()
constructor. When you construct an instance of this you must pass in a
record schema, not the union schema.
Doug
On 04/15/2011 07:19 PM, Vivek Hungund wr
On 04/27/2011 02:54 PM, Markus Weimer wrote:
> I am trying to read some data in Text format and produce avro files in the
> reducers. The documentation says that I should implement mapper with AvroKey
> and AvroValue as the output. I did that, but get the following error:
>
>> java.lang.IllegalA
The data file format is currently defined to always use the binary
encoding. The json encoding is optional.
For json, a text file with one record per line is a common format,
perhaps where the first line in the file is the schema.
Doug
On 05/01/2011 04:18 AM, David Shimon wrote:
> Hi,
>
>
>
On 05/06/2011 10:34 AM, Miki Tebeka wrote:
> I'm using the avro python package (1.5.0), and it is slow.
> It takes about 1min to process 33K records file. For comparison the
> Java packages process the same file in 1sec.
>
> Any ideas on how to speed that up?
Does the schema have unions? Last I
On 05/06/2011 11:18 AM, Miki Tebeka wrote:
> BTW: It'll be nice to have a __version__ in avro/__init__.py
Please file an issue in Jira and submit a patch, if you are able.
https://issues.apache.org/jira/browse/AVRO
Thanks,
Doug
On 05/05/2011 10:29 AM, Joe Crobak wrote:
> We've recently come across a situation where we have two data files with
> different schemas that we'd like to process together using
> GenericDatumReader. One schema is promotable to the other, but not vice
> versa. We'd like to programmatically determ
I'd like to announce the availability of Avro release 1.5.1.
Selected highlights of this release include:
NEW FEATURES
AVRO-533. Add a C# implementation.
(Jeremy Custenborder, Dona Alvarez and thiru)
AVRO-788. Java: Add Snappy compression for data files, including
MapReduce AP
On 05/06/2011 12:31 PM, Miki Tebeka wrote:
> BTW: When is 1.5.1 coming out?
It's out today!
Doug
On 05/06/2011 04:20 PM, Miki Tebeka wrote:
>> It should be possible to determine the union
>> branch to write much more efficiently.
> Can you elaborate on how? I'll try to code this and patch.
> Also, I'm talking about reading the avro file, not writing to it.
The optimization I was speaking of
What platform are you running the tests on?
Doug
On 05/18/2011 09:18 AM, stan lee wrote:
> Hi Guys,
>
> I am a new commer to Avro and I am trying to build and run testcases fro
> Avro 1.5.1 myself. Everything goes well until run the testcase
> TestSaslDigestMd5(actually I run "mvn test" under
t; Thanks!
>
> Stan
>
> On Wed, May 18, 2011 at 4:49 PM, Doug Cutting <mailto:cutt...@apache.org>> wrote:
>
> What platform are you running the tests on?
>
> Doug
>
> On 05/18/2011 09:18 AM, stan lee wrote:
> > Hi Guys,
> &g
Yes, this is intended to work. The client and server perform a
handshake that ensures that each has the other's protocol. Message
parameters are treated as a record. So, in your example, the server
would ignore the date parameter. If the server adds a parameter then it
should supply a default v
I believe this is already supported by Avro, as described in my previous
message.
Doug
On 05/25/2011 08:15 PM, Yang wrote:
> generally schema/method signature changes are unavoidable in a real
> production system.
>
> so once I setup an avro RPC server, what is the best strategy to
> handle such
On 05/26/2011 10:10 PM, Yang wrote:
> I guess the schema resolution is done only on input params, and not on
> return value?
The same resolution should be done on return values. For example,
changing the return value from one record to another would not work
(unless you used aliases) but adding o
1 - 100 of 492 matches
Mail list logo