This is the multi-page printable view of this section.
Click here to print.
Return to the regular view of this page.
Documentation
Introduction
Apache Avro™ is a data serialization system.
Avro provides:
- Rich data structures.
- A compact, fast, binary data format.
- A container file, to store persistent data.
- Remote procedure call (RPC).
- Simple integration with dynamic languages. Code generation is not required to read or write data files nor to use or implement RPC protocols. Code generation as an optional optimization, only worth implementing for statically typed languages.
Schemas
Avro relies on schemas. When Avro data is read, the schema used when writing it is always present. This permits each datum to be written with no per-value overheads, making serialization both fast and small. This also facilitates use with dynamic, scripting languages, since data, together with its schema, is fully self-describing.
When Avro data is stored in a file, its schema is stored with it, so that files may be processed later by any program. If the program reading the data expects a different schema this can be easily resolved, since both schemas are present.
When Avro is used in RPC, the client and server exchange schemas in the connection handshake. (This can be optimized so that, for most calls, no schemas are actually transmitted.) Since both client and server both have the other’s full schema, correspondence between same named fields, missing fields, extra fields, etc. can all be easily resolved.
Avro schemas are defined with JSON . This facilitates implementation in languages that already have JSON libraries.
Comparison with other systems
Avro provides functionality similar to systems such as Thrift, Protocol Buffers, etc. Avro differs from these systems in the following fundamental aspects.
- Dynamic typing: Avro does not require that code be generated. Data is always accompanied by a schema that permits full processing of that data without code generation, static datatypes, etc. This facilitates construction of generic data-processing systems and languages.
- Untagged data: Since the schema is present when data is read, considerably less type information need be encoded with data, resulting in smaller serialization size.
- No manually-assigned field IDs: When a schema changes, both the old and new schema are always present when processing data, so differences may be resolved symbolically, using field names.
1 - Apache Avro™ 1.11.1(Current) Documentation
Introduction
Apache Avro™ is a data serialization system.
Avro provides:
- Rich data structures.
- A compact, fast, binary data format.
- A container file, to store persistent data.
- Remote procedure call (RPC).
- Simple integration with dynamic languages. Code generation is not required to read or write data files nor to use or implement RPC protocols. Code generation as an optional optimization, only worth implementing for statically typed languages.
Schemas
Avro relies on schemas. When Avro data is read, the schema used when writing it is always present. This permits each datum to be written with no per-value overheads, making serialization both fast and small. This also facilitates use with dynamic, scripting languages, since data, together with its schema, is fully self-describing.
When Avro data is stored in a file, its schema is stored with it, so that files may be processed later by any program. If the program reading the data expects a different schema this can be easily resolved, since both schemas are present.
When Avro is used in RPC, the client and server exchange schemas in the connection handshake. (This can be optimized so that, for most calls, no schemas are actually transmitted.) Since both client and server both have the other’s full schema, correspondence between same named fields, missing fields, extra fields, etc. can all be easily resolved.
Avro schemas are defined with JSON . This facilitates implementation in languages that already have JSON libraries.
Comparison with other systems
Avro provides functionality similar to systems such as Thrift, Protocol Buffers, etc. Avro differs from these systems in the following fundamental aspects.
- Dynamic typing: Avro does not require that code be generated. Data is always accompanied by a schema that permits full processing of that data without code generation, static datatypes, etc. This facilitates construction of generic data-processing systems and languages.
- Untagged data: Since the schema is present when data is read, considerably less type information need be encoded with data, resulting in smaller serialization size.
- No manually-assigned field IDs: When a schema changes, both the old and new schema are always present when processing data, so differences may be resolved symbolically, using field names.
1.1 - Getting Started (Java)
This is a short guide for getting started with Apache Avro™ using Java. This guide only covers using Avro for data serialization; see Patrick Hunt’s Avro RPC Quick Start for a good introduction to using Avro for RPC.
Download
Avro implementations for C, C++, C#, Java, PHP, Python, and Ruby can be downloaded from the Apache Avro™ Download page. This guide uses Avro 1.11.1, the latest version at the time of writing. For the examples in this guide, download avro-1.11.1.jar and avro-tools-1.11.1.jar.
Alternatively, if you are using Maven, add the following dependency to your POM:
<dependency>
<groupId>org.apache.avro</groupId>
<artifactId>avro</artifactId>
<version>1.11.1</version>
</dependency>
As well as the Avro Maven plugin (for performing code generation):
<plugin>
<groupId>org.apache.avro</groupId>
<artifactId>avro-maven-plugin</artifactId>
<version>1.11.1</version>
<executions>
<execution>
<phase>generate-sources</phase>
<goals>
<goal>schema</goal>
</goals>
<configuration>
<sourceDirectory>${project.basedir}/src/main/avro/</sourceDirectory>
<outputDirectory>${project.basedir}/src/main/java/</outputDirectory>
</configuration>
</execution>
</executions>
</plugin>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-compiler-plugin</artifactId>
<configuration>
<source>1.8</source>
<target>1.8</target>
</configuration>
</plugin>
You may also build the required Avro jars from source. Building Avro is beyond the scope of this guide; see the Build Documentation page in the wiki for more information.
Defining a schema
Avro schemas are defined using JSON. Schemas are composed of primitive types (null, boolean, int, long, float, double, bytes, and string) and complex types (record, enum, array, map, union, and fixed). You can learn more about Avro schemas and types from the specification, but for now let’s start with a simple schema example, user.avsc:
{"namespace": "example.avro",
"type": "record",
"name": "User",
"fields": [
{"name": "name", "type": "string"},
{"name": "favorite_number", "type": ["int", "null"]},
{"name": "favorite_color", "type": ["string", "null"]}
]
}
This schema defines a record representing a hypothetical user. (Note that a schema file can only contain a single schema definition.) At minimum, a record definition must include its type (“type”: “record”), a name (“name”: “User”), and fields, in this case name, favorite_number, and favorite_color. We also define a namespace (“namespace”: “example.avro”), which together with the name attribute defines the “full name” of the schema (example.avro.User in this case).
Fields are defined via an array of objects, each of which defines a name and type (other attributes are optional, see the record specification for more details). The type attribute of a field is another schema object, which can be either a primitive or complex type. For example, the name field of our User schema is the primitive type string, whereas the favorite_number and favorite_color fields are both unions, represented by JSON arrays. unions are a complex type that can be any of the types listed in the array; e.g., favorite_number can either be an int or null, essentially making it an optional field.
Serializing and deserializing with code generation
Compiling the schema
Code generation allows us to automatically create classes based on our previously-defined schema. Once we have defined the relevant classes, there is no need to use the schema directly in our programs. We use the avro-tools jar to generate code as follows:
java -jar /path/to/avro-tools-1.11.1.jar compile schema <schema file> <destination>
This will generate the appropriate source files in a package based on the schema’s namespace in the provided destination folder. For instance, to generate a User class in package example.avro from the schema defined above, run
java -jar /path/to/avro-tools-1.11.1.jar compile schema user.avsc .
Note that if you using the Avro Maven plugin, there is no need to manually invoke the schema compiler; the plugin automatically performs code generation on any .avsc files present in the configured source directory.
Creating Users
Now that we’ve completed the code generation, let’s create some Users, serialize them to a data file on disk, and then read back the file and deserialize the User objects.
First let’s create some Users and set their fields.
User user1 = new User();
user1.setName("Alyssa");
user1.setFavoriteNumber(256);
// Leave favorite color null
// Alternate constructor
User user2 = new User("Ben", 7, "red");
// Construct via builder
User user3 = User.newBuilder()
.setName("Charlie")
.setFavoriteColor("blue")
.setFavoriteNumber(null)
.build();
As shown in this example, Avro objects can be created either by invoking a constructor directly or by using a builder. Unlike constructors, builders will automatically set any default values specified in the schema. Additionally, builders validate the data as it set, whereas objects constructed directly will not cause an error until the object is serialized. However, using constructors directly generally offers better performance, as builders create a copy of the datastructure before it is written.
Note that we do not set user1’s favorite color. Since that record is of type [“string”, “null”], we can either set it to a string or leave it null; it is essentially optional. Similarly, we set user3’s favorite number to null (using a builder requires setting all fields, even if they are null).
Serializing
Now let’s serialize our Users to disk.
// Serialize user1, user2 and user3 to disk
DatumWriter<User> userDatumWriter = new SpecificDatumWriter<User>(User.class);
DataFileWriter<User> dataFileWriter = new DataFileWriter<User>(userDatumWriter);
dataFileWriter.create(user1.getSchema(), new File("users.avro"));
dataFileWriter.append(user1);
dataFileWriter.append(user2);
dataFileWriter.append(user3);
dataFileWriter.close();
We create a DatumWriter, which converts Java objects into an in-memory serialized format. The SpecificDatumWriter class is used with generated classes and extracts the schema from the specified generated type.
Next we create a DataFileWriter, which writes the serialized records, as well as the schema, to the file specified in the dataFileWriter.create call. We write our users to the file via calls to the dataFileWriter.append method. When we are done writing, we close the data file.
Deserializing
Finally, let’s deserialize the data file we just created.
// Deserialize Users from disk
DatumReader<User> userDatumReader = new SpecificDatumReader<User>(User.class);
DataFileReader<User> dataFileReader = new DataFileReader<User>(file, userDatumReader);
User user = null;
while (dataFileReader.hasNext()) {
// Reuse user object by passing it to next(). This saves us from
// allocating and garbage collecting many objects for files with
// many items.
user = dataFileReader.next(user);
System.out.println(user);
}
This snippet will output:
{"name": "Alyssa", "favorite_number": 256, "favorite_color": null}
{"name": "Ben", "favorite_number": 7, "favorite_color": "red"}
{"name": "Charlie", "favorite_number": null, "favorite_color": "blue"}
Deserializing is very similar to serializing. We create a SpecificDatumReader, analogous to the SpecificDatumWriter we used in serialization, which converts in-memory serialized items into instances of our generated class, in this case User. We pass the DatumReader and the previously created File to a DataFileReader, analogous to the DataFileWriter, which reads both the schema used by the writer as well as the data from the file on disk. The data will be read using the writer’s schema included in the file and the schema provided by the reader, in this case the User class. The writer’s schema is needed to know the order in which fields were written, while the reader’s schema is needed to know what fields are expected and how to fill in default values for fields added since the file was written. If there are differences between the two schemas, they are resolved according to the Schema Resolution specification.
Next we use the DataFileReader to iterate through the serialized Users and print the deserialized object to stdout. Note how we perform the iteration: we create a single User object which we store the current deserialized user in, and pass this record object to every call of dataFileReader.next. This is a performance optimization that allows the DataFileReader to reuse the same User object rather than allocating a new User for every iteration, which can be very expensive in terms of object allocation and garbage collection if we deserialize a large data file. While this technique is the standard way to iterate through a data file, it’s also possible to use for (User user : dataFileReader) if performance is not a concern.
Compiling and running the example code
This example code is included as a Maven project in the examples/java-example directory in the Avro docs. From this directory, execute the following commands to build and run the example:
$ mvn compile # includes code generation via Avro Maven plugin
$ mvn -q exec:java -Dexec.mainClass=example.SpecificMain
Beta feature: Generating faster code
In release 1.9.0, we introduced a new approach to generating code that speeds up decoding of objects by more than 10% and encoding by more than 30% (future performance enhancements are underway). To ensure a smooth introduction of this change into production systems, this feature is controlled by a feature flag, the system property org.apache.avro.specific.use_custom_coders. In this first release, this feature is off by default. To turn it on, set the system flag to true at runtime. In the sample above, for example, you could enable the fater coders as follows:
$ mvn -q exec:java -Dexec.mainClass=example.SpecificMain
-Dorg.apache.avro.specific.use_custom_coders=true
Note that you do not have to recompile your Avro schema to have access to this feature. The feature is compiled and built into your code, and you turn it on and off at runtime using the feature flag. As a result, you can turn it on during testing, for example, and then off in production. Or you can turn it on in production, and quickly turn it off if something breaks.
We encourage the Avro community to exercise this new feature early to help build confidence. (For those paying one-demand for compute resources in the cloud, it can lead to meaningful cost savings.) As confidence builds, we will turn this feature on by default, and eventually eliminate the feature flag (and the old code).
Serializing and deserializing without code generation
Data in Avro is always stored with its corresponding schema, meaning we can always read a serialized item regardless of whether we know the schema ahead of time. This allows us to perform serialization and deserialization without code generation.
Let’s go over the same example as in the previous section, but without using code generation: we’ll create some users, serialize them to a data file on disk, and then read back the file and deserialize the users objects.
Creating users
First, we use a Parser to read our schema definition and create a Schema object.
Schema schema = new Schema.Parser().parse(new File("user.avsc"));
Using this schema, let’s create some users.
GenericRecord user1 = new GenericData.Record(schema);
user1.put("name", "Alyssa");
user1.put("favorite_number", 256);
// Leave favorite color null
GenericRecord user2 = new GenericData.Record(schema);
user2.put("name", "Ben");
user2.put("favorite_number", 7);
user2.put("favorite_color", "red");
Since we’re not using code generation, we use GenericRecords to represent users. GenericRecord uses the schema to verify that we only specify valid fields. If we try to set a non-existent field (e.g., user1.put(“favorite_animal”, “cat”)), we’ll get an AvroRuntimeException when we run the program.
Note that we do not set user1’s favorite color. Since that record is of type [“string”, “null”], we can either set it to a string or leave it null; it is essentially optional.
Serializing
Now that we’ve created our user objects, serializing and deserializing them is almost identical to the example above which uses code generation. The main difference is that we use generic instead of specific readers and writers.
First we’ll serialize our users to a data file on disk.
// Serialize user1 and user2 to disk
File file = new File("users.avro");
DatumWriter<GenericRecord> datumWriter = new GenericDatumWriter<GenericRecord>(schema);
DataFileWriter<GenericRecord> dataFileWriter = new DataFileWriter<GenericRecord>(datumWriter);
dataFileWriter.create(schema, file);
dataFileWriter.append(user1);
dataFileWriter.append(user2);
dataFileWriter.close();
We create a DatumWriter, which converts Java objects into an in-memory serialized format. Since we are not using code generation, we create a GenericDatumWriter. It requires the schema both to determine how to write the GenericRecords and to verify that all non-nullable fields are present.
As in the code generation example, we also create a DataFileWriter, which writes the serialized records, as well as the schema, to the file specified in the dataFileWriter.create call. We write our users to the file via calls to the dataFileWriter.append method. When we are done writing, we close the data file.
Deserializing
Finally, we’ll deserialize the data file we just created.
// Deserialize users from disk
DatumReader<GenericRecord> datumReader = new GenericDatumReader<GenericRecord>(schema);
DataFileReader<GenericRecord> dataFileReader = new DataFileReader<GenericRecord>(file, datumReader);
GenericRecord user = null;
while (dataFileReader.hasNext()) {
// Reuse user object by passing it to next(). This saves us from
// allocating and garbage collecting many objects for files with
// many items.
user = dataFileReader.next(user);
System.out.println(user);
This outputs:
{"name": "Alyssa", "favorite_number": 256, "favorite_color": null}
{"name": "Ben", "favorite_number": 7, "favorite_color": "red"}
Deserializing is very similar to serializing. We create a GenericDatumReader, analogous to the GenericDatumWriter we used in serialization, which converts in-memory serialized items into GenericRecords. We pass the DatumReader and the previously created File to a DataFileReader, analogous to the DataFileWriter, which reads both the schema used by the writer as well as the data from the file on disk. The data will be read using the writer’s schema included in the file, and the reader’s schema provided to the GenericDatumReader. The writer’s schema is needed to know the order in which fields were written, while the reader’s schema is needed to know what fields are expected and how to fill in default values for fields added since the file was written. If there are differences between the two schemas, they are resolved according to the Schema Resolution specification.
Next, we use the DataFileReader to iterate through the serialized users and print the deserialized object to stdout. Note how we perform the iteration: we create a single GenericRecord object which we store the current deserialized user in, and pass this record object to every call of dataFileReader.next. This is a performance optimization that allows the DataFileReader to reuse the same record object rather than allocating a new GenericRecord for every iteration, which can be very expensive in terms of object allocation and garbage collection if we deserialize a large data file. While this technique is the standard way to iterate through a data file, it’s also possible to use for (GenericRecord user : dataFileReader) if performance is not a concern.
Compiling and running the example code
This example code is included as a Maven project in the examples/java-example directory in the Avro docs. From this directory, execute the following commands to build and run the example:
$ mvn compile
$ mvn -q exec:java -Dexec.mainClass=example.GenericMain
1.2 - Getting Started (Python)
This is a short guide for getting started with Apache Avro™ using Python. This guide only covers using Avro for data serialization; see Patrick Hunt’s Avro RPC Quick Start for a good introduction to using Avro for RPC.
Notice for Python 3 users
A package called “avro-python3” had been provided to support Python 3 previously, but the codebase was consolidated into the “avro” package and that supports both Python 2 and 3 now. The avro-python3 package will be removed in the near future, so users should use the “avro” package instead. They are mostly API compatible, but there’s a few minor difference (e.g., function name capitalization, such as avro.schema.Parse vs avro.schema.parse).
Download
For Python, the easiest way to get started is to install it from PyPI. Python’s Avro API is available over PyPi.
$ python3 -m pip install avro
The official releases of the Avro implementations for C, C++, C#, Java, PHP, Python, and Ruby can be downloaded from the Apache Avro™ Releases page. This guide uses Avro 1.11.1, the latest version at the time of writing. Download and unzip avro-1.11.1.tar.gz, and install via python setup.py (this will probably require root privileges). Ensure that you can import avro from a Python prompt.
$ tar xvf avro-1.11.1.tar.gz
$ cd avro-1.11.1
$ python setup.py install
$ python
>>> import avro # should not raise ImportError
Alternatively, you may build the Avro Python library from source. From your the root Avro directory, run the commands
$ cd lang/py/
$ python3 -m pip install -e .
$ python
Defining a schema
Avro schemas are defined using JSON. Schemas are composed of primitive types (null, boolean, int, long, float, double, bytes, and string) and complex types (record, enum, array, map, union, and fixed). You can learn more about Avro schemas and types from the specification, but for now let’s start with a simple schema example, user.avsc:
{"namespace": "example.avro",
"type": "record",
"name": "User",
"fields": [
{"name": "name", "type": "string"},
{"name": "favorite_number", "type": ["int", "null"]},
{"name": "favorite_color", "type": ["string", "null"]}
]
}
This schema defines a record representing a hypothetical user. (Note that a schema file can only contain a single schema definition.) At minimum, a record definition must include its type (“type”: “record”), a name (“name”: “User”), and fields, in this case name, favorite_number, and favorite_color. We also define a namespace (“namespace”: “example.avro”), which together with the name attribute defines the “full name” of the schema (example.avro.User in this case).
Fields are defined via an array of objects, each of which defines a name and type (other attributes are optional, see the record specification for more details). The type attribute of a field is another schema object, which can be either a primitive or complex type. For example, the name field of our User schema is the primitive type string, whereas the favorite_number and favorite_color fields are both unions, represented by JSON arrays. unions are a complex type that can be any of the types listed in the array; e.g., favorite_number can either be an int or null, essentially making it an optional field.
Serializing and deserializing without code generation
Data in Avro is always stored with its corresponding schema, meaning we can always read a serialized item, regardless of whether we know the schema ahead of time. This allows us to perform serialization and deserialization without code generation. Note that the Avro Python library does not support code generation.
Try running the following code snippet, which serializes two users to a data file on disk, and then reads back and deserializes the data file:
import avro.schema
from avro.datafile import DataFileReader, DataFileWriter
from avro.io import DatumReader, DatumWriter
schema = avro.schema.parse(open("user.avsc", "rb").read())
writer = DataFileWriter(open("users.avro", "wb"), DatumWriter(), schema)
writer.append({"name": "Alyssa", "favorite_number": 256})
writer.append({"name": "Ben", "favorite_number": 7, "favorite_color": "red"})
writer.close()
reader = DataFileReader(open("users.avro", "rb"), DatumReader())
for user in reader:
print(user)
reader.close()
This outputs:
{u'favorite_color': None, u'favorite_number': 256, u'name': u'Alyssa'}
{u'favorite_color': u'red', u'favorite_number': 7, u'name': u'Ben'}
Do make sure that you open your files in binary mode (i.e. using the modes wb or rb respectively). Otherwise you might generate corrupt files due to automatic replacement of newline characters with the platform-specific representations.
Let’s take a closer look at what’s going on here.
schema = avro.schema.parse(open("user.avsc", "rb").read())
avro.schema.parse takes a string containing a JSON schema definition as input and outputs a avro.schema.Schema object (specifically a subclass of Schema, in this case RecordSchema). We’re passing in the contents of our user.avsc schema file here.
writer = DataFileWriter(open("users.avro", "wb"), DatumWriter(), schema)
We create a DataFileWriter, which we’ll use to write serialized items to a data file on disk. The DataFileWriter constructor takes three arguments:
- The file we’ll serialize to
- A DatumWriter, which is responsible for actually serializing the items to Avro’s binary format (DatumWriters can be used separately from DataFileWriters, e.g., to perform IPC with Avro).
- The schema we’re using. The DataFileWriter needs the schema both to write the schema to the data file, and to verify that the items we write are valid items and write the appropriate fields.
writer.append({"name": "Alyssa", "favorite_number": 256})
writer.append({"name": "Ben", "favorite_number": 7, "favorite_color": "red"})
We use DataFileWriter.append to add items to our data file. Avro records are represented as Python dicts. Since the field favorite_color has type [“string”, “null”], we are not required to specify this field, as shown in the first append. Were we to omit the required name field, an exception would be raised. Any extra entries not corresponding to a field are present in the dict are ignored.
reader = DataFileReader(open("users.avro", "rb"), DatumReader())
We open the file again, this time for reading back from disk. We use a DataFileReader and DatumReader analagous to the DataFileWriter and DatumWriter above.
for user in reader:
print(user)
The DataFileReader is an iterator that returns dicts corresponding to the serialized items.
1.3 - Specification
Introduction
This document defines Apache Avro. It is intended to be the authoritative specification. Implementations of Avro must adhere to this document.
Schema Declaration
A Schema is represented in JSON by one of:
- A JSON string, naming a defined type.
- A JSON object, of the form:
{"type": "typeName", ...attributes...}
where typeName is either a primitive or derived type name, as defined below. Attributes not defined in this document are permitted as metadata, but must not affect the format of serialized data.
- A JSON array, representing a union of embedded types.
Primitive Types
The set of primitive type names is:
- null: no value
- boolean: a binary value
- int: 32-bit signed integer
- long: 64-bit signed integer
- float: single precision (32-bit) IEEE 754 floating-point number
- double: double precision (64-bit) IEEE 754 floating-point number
- bytes: sequence of 8-bit unsigned bytes
- string: unicode character sequence
Primitive types have no specified attributes.
Primitive type names are also defined type names. Thus, for example, the schema “string” is equivalent to:
Complex Types
Avro supports six kinds of complex types: records, enums, arrays, maps, unions and fixed.
Records
Records use the type name “record” and support the following attributes:
- name: a JSON string providing the name of the record (required).
- namespace, a JSON string that qualifies the name (optional);
- doc: a JSON string providing documentation to the user of this schema (optional).
- aliases: a JSON array of strings, providing alternate names for this record (optional).
- fields: a JSON array, listing fields (required). Each field is a JSON object with the following attributes:
- name: a JSON string providing the name of the field (required), and
- doc: a JSON string describing this field for users (optional).
- type: a schema, as defined above
- default: A default value for this field, only used when reading instances that lack the field for schema evolution purposes. The presence of a default value does not make the field optional at encoding time. 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. Default values for bytes and fixed fields are JSON strings, where Unicode code points 0-255 are mapped to unsigned 8-bit byte values 0-255. Avro encodes a field even if its value is equal to its default.
field default values
| avro type |
json type |
example |
| null |
null |
null |
| boolean |
boolean |
true |
| int,long |
integer |
1 |
| float,double |
number |
1.1 |
| bytes |
string |
"\u00FF" |
| string |
string |
"foo" |
| record |
object |
{"a": 1} |
| enum |
string |
"FOO" |
| array |
array |
[1] |
| map |
object |
{"a": 1} |
| fixed |
string |
"\u00ff" |
- order: specifies how this field impacts sort ordering of this record (optional). Valid values are “ascending” (the default), “descending”, or “ignore”. For more details on how this is used, see the sort order section below.
- aliases: a JSON array of strings, providing alternate names for this field (optional).
For example, a linked-list of 64-bit values may be defined with:
{
"type": "record",
"name": "LongList",
"aliases": ["LinkedLongs"], // old name for this
"fields" : [
{"name": "value", "type": "long"}, // each element has a long
{"name": "next", "type": ["null", "LongList"]} // optional next element
]
}
Enums
Enums use the type name “enum” and support the following attributes:
- name: a JSON string providing the name of the enum (required).
- namespace, a JSON string that qualifies the name (optional);
- aliases: a JSON array of strings, providing alternate names for this enum (optional).
- doc: a JSON string providing documentation to the user of this schema (optional).
- symbols: a JSON array, listing symbols, as JSON strings (required). All symbols in an enum must be unique; duplicates are prohibited. Every symbol must match the regular expression [A-Za-z_][A-Za-z0-9_]* (the same requirement as for names).
- default: A default value for this enumeration, used during resolution when the reader encounters a symbol from the writer that isn’t defined in the reader’s schema (optional). The value provided here must be a JSON string that’s a member of the symbols array. See documentation on schema resolution for how this gets used.
For example, playing card suits might be defined with:
{
"type": "enum",
"name": "Suit",
"symbols" : ["SPADES", "HEARTS", "DIAMONDS", "CLUBS"]
}
Arrays
Arrays use the type name “array” and support a single attribute:
- items: the schema of the array’s items.
For example, an array of strings is declared with:
{
"type": "array",
"items" : "string",
"default": []
}
Maps
Maps use the type name “map” and support one attribute:
- values: the schema of the map’s values.
Map keys are assumed to be strings.
For example, a map from string to long is declared with:
{
"type": "map",
"values" : "long",
"default": {}
}
Unions
Unions, as mentioned above, are represented using JSON arrays. For example, ["null", "string"] declares a schema which may be either a null or string.
(Note that when a default value is specified for a record field whose type is a union, the type of the default value must match the first element of the union. Thus, for unions containing “null”, the “null” is usually listed first, since the default value of such unions is typically null.)
Unions may not contain more than one schema with the same type, except for the named types record, fixed and enum. For example, unions containing two array types or two map types are not permitted, but two types with different names are permitted. (Names permit efficient resolution when reading and writing unions.)
Unions may not immediately contain other unions.
Fixed
Fixed uses the type name “fixed” and supports the following attributes:
- name: a string naming this fixed (required).
- namespace, a string that qualifies the name (optional);
- aliases: a JSON array of strings, providing alternate names for this enum (optional).
- size: an integer, specifying the number of bytes per value (required).
For example, 16-byte quantity may be declared with:
{"type": "fixed", "size": 16, "name": "md5"}
Names
Record, enums and fixed are named types. Each has a fullname that is composed of two parts; a name and a namespace, separated by a dot. Equality of names is defined on the fullname.
Record fields and enum symbols have names as well (but no namespace). Equality of fields and enum symbols is defined on the name of the field/symbol within its scope (the record/enum that defines it). Fields and enum symbols across scopes are never equal.
The name portion of the fullname of named types, record field names, and enum symbols must:
- start with [A-Za-z_]
- subsequently contain only [A-Za-z0-9_]
A namespace is a dot-separated sequence of such names. The empty string may also be used as a namespace to indicate the null namespace. Equality of names (including field names and enum symbols) as well as fullnames is case-sensitive.
The null namespace may not be used in a dot-separated sequence of names. So the grammar for a namespace is:
<empty> | <name>[(<dot><name>)*]
In record, enum and fixed definitions, the fullname is determined according to the algorithm below the example:
{
"type": "record",
"name": "Example",
"doc": "A simple name (attribute) and no namespace attribute: use the null namespace (\"\"); the fullname is 'Example'.",
"fields": [
{
"name": "inheritNull",
"type": {
"type": "enum",
"name": "Simple",
"doc": "A simple name (attribute) and no namespace attribute: inherit the null namespace of the enclosing type 'Example'. The fullname is 'Simple'.",
"symbols": ["a", "b"]
}
}, {
"name": "explicitNamespace",
"type": {
"type": "fixed",
"name": "Simple",
"namespace": "explicit",
"doc": "A simple name (attribute) and a namespace (attribute); the fullname is 'explicit.Simple' (this is a different type than of the 'inheritNull' field).",
"size": 12
}
}, {
"name": "fullName",
"type": {
"type": "record",
"name": "a.full.Name",
"namespace": "ignored",
"doc": "A name attribute with a fullname, so the namespace attribute is ignored. The fullname is 'a.full.Name', and the namespace is 'a.full'.",
"fields": [
{
"name": "inheritNamespace",
"type": {
"type": "enum",
"name": "Understanding",
"doc": "A simple name (attribute) and no namespace attribute: inherit the namespace of the enclosing type 'a.full.Name'. The fullname is 'a.full.Understanding'.",
"symbols": ["d", "e"]
}
}
]
}
}
]
}
The fullname of a record, enum or fixed definition is determined by the required name and optional namespace attributes like this:
- A fullname is specified. If the name specified contains a dot, then it is assumed to be a fullname, and any namespace also specified is ignored. For example, use “name”: “org.foo.X” to indicate the fullname org.foo.X.
- A simple name (a name that contains no dots) and namespace are both specified. For example, one might use “name”: “X”, “namespace”: “org.foo” to indicate the fullname org.foo.X.
- A simple name only is specified (a name that contains no dots). In this case the namespace is taken from the most tightly enclosing named schema or protocol, and the fullname is constructed from that namespace and the name. For example, if “name”: “X” is specified, and this occurs within a field of the record definition of org.foo.Y, then the fullname is org.foo.X. This also happens if there is no enclosing namespace (i.e., the enclosing schema definition has the null namespace).
References to previously defined names are as in the latter two cases above: if they contain a dot they are a fullname, if they do not contain a dot, the namespace is the namespace of the enclosing definition.
Primitive type names (null, boolean, int, long, float, double, bytes, string) have no namespace and their names may not be defined in any namespace.
Complex types (record, enum, array, map, fixed) have no namespace, but their names (as well as union) are permitted to be reused as type names. This can be confusing to the human reader, but is always unambiguous for binary serialization. Due to the limitations of JSON encoding, it is a best practice to use a namespace when using these names.
A schema or protocol may not contain multiple definitions of a fullname. Further, a name must be defined before it is used (“before” in the depth-first, left-to-right traversal of the JSON parse tree, where the types attribute of a protocol is always deemed to come “before” the messages attribute.)
Aliases
Named types and fields may have aliases. An implementation may optionally use aliases to map a writer’s schema to the reader’s. This facilitates both schema evolution as well as processing disparate datasets.
Aliases function by re-writing the writer’s schema using aliases from the reader’s schema. For example, if the writer’s schema was named “Foo” and the reader’s schema is named “Bar” and has an alias of “Foo”, then the implementation would act as though “Foo” were named “Bar” when reading. Similarly, if data was written as a record with a field named “x” and is read as a record with a field named “y” with alias “x”, then the implementation would act as though “x” were named “y” when reading.
A type alias may be specified either as a fully namespace-qualified, or relative to the namespace of the name it is an alias for. For example, if a type named “a.b” has aliases of “c” and “x.y”, then the fully qualified names of its aliases are “a.c” and “x.y”.
Data Serialization and Deserialization
Binary encoded Avro data does not include type information or field names. The benefit is that the serialized data is small, but as a result a schema must always be used in order to read Avro data correctly. The best way to ensure that the schema is structurally identical to the one used to write the data is to use the exact same schema.
Therefore, files or systems that store Avro data should always include the writer’s schema for that data. Avro-based remote procedure call (RPC) systems must also guarantee that remote recipients of data have a copy of the schema used to write that data. In general, it is advisable that any reader of Avro data should use a schema that is the same (as defined more fully in Parsing Canonical Form for Schemas) as the schema that was used to write the data in order to deserialize it correctly. Deserializing data into a newer schema is accomplished by specifying an additional schema, the results of which are described in Schema Resolution.
In general, both serialization and deserialization proceed as a depth-first, left-to-right traversal of the schema, serializing or deserializing primitive types as they are encountered. Therefore, it is possible, though not advisable, to read Avro data with a schema that does not have the same Parsing Canonical Form as the schema with which the data was written. In order for this to work, the serialized primitive values must be compatible, in order value by value, with the items in the deserialization schema. For example, int and long are always serialized the same way, so an int could be deserialized as a long. Since the compatibility of two schemas depends on both the data and the serialization format (eg. binary is more permissive than JSON because JSON includes field names, eg. a long that is too large will overflow an int), it is simpler and more reliable to use schemas with identical Parsing Canonical Form.
Encodings
Avro specifies two serialization encodings: binary and JSON. Most applications will use the binary encoding, as it is smaller and faster. But, for debugging and web-based applications, the JSON encoding may sometimes be appropriate.
Binary Encoding
Binary encoding does not include field names, self-contained information about the types of individual bytes, nor field or record separators. Therefore readers are wholly reliant on the schema used when the data was encoded.
Primitive Types
Primitive types are encoded in binary as follows:
- null is written as zero bytes.
- a boolean is written as a single byte whose value is either 0 (false) or 1 (true).
- int and long values are written using variable-length zig-zag coding. Some examples:
| value |
hex |
| 0 |
00 |
| -1 |
01 |
| 1 |
02 |
| -2 |
03 |
| 2 |
04 |
| … |
… |
| -64 |
7f |
| 64 |
80 01 |
| … |
… |
- a float is written as 4 bytes. The float is converted into a 32-bit integer using a method equivalent to Java’s floatToIntBits and then encoded in little-endian format.
- a double is written as 8 bytes. The double is converted into a 64-bit integer using a method equivalent to Java’s doubleToLongBits and then encoded in little-endian format.
- bytes are encoded as a long followed by that many bytes of data.
- a string is encoded as a long followed by that many bytes of UTF-8 encoded character data.
For example, the three-character string “foo” would be encoded as the long value 3 (encoded as hex 06) followed by the UTF-8 encoding of ‘f’, ‘o’, and ‘o’ (the hex bytes 66 6f 6f):
06 66 6f 6f
Complex Types
Complex types are encoded in binary as follows:
Records
A record is encoded by encoding the values of its fields in the order that they are declared. In other words, a record is encoded as just the concatenation of the encodings of its fields. Field values are encoded per their schema.
For example, the record schema
{
"type": "record",
"name": "test",
"fields" : [
{"name": "a", "type": "long"},
{"name": "b", "type": "string"}
]
}
An instance of this record whose a field has value 27 (encoded as hex 36) and whose b field has value “foo” (encoded as hex bytes 06 66 6f 6f), would be encoded simply as the concatenation of these, namely the hex byte sequence:
36 06 66 6f 6f
Enums
An enum is encoded by a int, representing the zero-based position of the symbol in the schema.
For example, consider the enum:
{"type": "enum", "name": "Foo", "symbols": ["A", "B", "C", "D"] }
This would be encoded by an int between zero and three, with zero indicating “A”, and 3 indicating “D”.
Arrays
Arrays are encoded as a series of blocks. Each block consists of a long count value, followed by that many array items. A block with count zero indicates the end of the array. Each item is encoded per the array’s item schema.
If a block’s count is negative, its absolute value is used, and the count is followed immediately by a long block size indicating the number of bytes in the block. This block size permits fast skipping through data, e.g., when projecting a record to a subset of its fields.
For example, the array schema
{"type": "array", "items": "long"}
an array containing the items 3 and 27 could be encoded as the long value 2 (encoded as hex 04) followed by long values 3 and 27 (encoded as hex 06 36) terminated by zero:
04 06 36 00
The blocked representation permits one to read and write arrays larger than can be buffered in memory, since one can start writing items without knowing the full length of the array.
Maps
Maps are encoded as a series of blocks. Each block consists of a long count value, followed by that many key/value pairs. A block with count zero indicates the end of the map. Each item is encoded per the map’s value schema.
If a block’s count is negative, its absolute value is used, and the count is followed immediately by a long block size indicating the number of bytes in the block. This block size permits fast skipping through data, e.g., when projecting a record to a subset of its fields.
The blocked representation permits one to read and write maps larger than can be buffered in memory, since one can start writing items without knowing the full length of the map.
Unions
A union is encoded by first writing an int value indicating the zero-based position within the union of the schema of its value. The value is then encoded per the indicated schema within the union.
For example, the union schema ["null","string"] would encode:
- null as zero (the index of “null” in the union):
00
- the string “a” as one (the index of “string” in the union, 1, encoded as hex 02), followed by the serialized string:
02 02 61
NOTE: Currently for C/C++ implementations, the positions are practically an int, but theoretically a long. In reality, we don’t expect unions with 215M members
Fixed
Fixed instances are encoded using the number of bytes declared in the schema.
JSON Encoding
Except for unions, the JSON encoding is the same as is used to encode field default values.
The value of a union is encoded in JSON as follows:
- if its type is null, then it is encoded as a JSON null;
- otherwise it is encoded as a JSON object with one name/value pair whose name is the type’s name and whose value is the recursively encoded value. For Avro’s named types (record, fixed or enum) the user-specified name is used, for other types the type name is used.
For example, the union schema ["null","string","Foo"], where Foo is a record name, would encode:
- null as null;
- the string “a” as
{"string": "a"} and
- a Foo instance as
{"Foo": {...}}, where {...} indicates the JSON encoding of a Foo instance.
Note that the original schema is still required to correctly process JSON-encoded data. For example, the JSON encoding does not distinguish between int and long, float and double, records and maps, enums and strings, etc.
Single-object encoding
In some situations a single Avro serialized object is to be stored for a longer period of time. One very common example is storing Avro records for several weeks in an Apache Kafka topic.
In the period after a schema change this persistence system will contain records that have been written with different schemas. So the need arises to know which schema was used to write a record to support schema evolution correctly. In most cases the schema itself is too large to include in the message, so this binary wrapper format supports the use case more effectively.
Single object encoding specification
Single Avro objects are encoded as follows:
- A two-byte marker,
C3 01, to show that the message is Avro and uses this single-record format (version 1).
- The 8-byte little-endian CRC-64-AVRO fingerprint of the object’s schema.
- The Avro object encoded using Avro’s binary encoding.
Implementations use the 2-byte marker to determine whether a payload is Avro. This check helps avoid expensive lookups that resolve the schema from a fingerprint, when the message is not an encoded Avro payload.
Sort Order
Avro defines a standard sort order for data. This permits data written by one system to be efficiently sorted by another system. This can be an important optimization, as sort order comparisons are sometimes the most frequent per-object operation. Note also that Avro binary-encoded data can be efficiently ordered without deserializing it to objects.
Data items may only be compared if they have identical schemas. Pairwise comparisons are implemented recursively with a depth-first, left-to-right traversal of the schema. The first mismatch encountered determines the order of the items.
Two items with the same schema are compared according to the following rules.
- null data is always equal.
- boolean data is ordered with false before true.
- int, long, float and double data is ordered by ascending numeric value.
- bytes and fixed data are compared lexicographically by unsigned 8-bit values.
- string data is compared lexicographically by Unicode code point. Note that since UTF-8 is used as the binary encoding for strings, sorting of bytes and string binary data is identical.
- array data is compared lexicographically by element.
- enum data is ordered by the symbol’s position in the enum schema. For example, an enum whose symbols are
["z", "a"] would sort “z” values before “a” values.
- union data is first ordered by the branch within the union, and, within that, by the type of the branch. For example, an
["int", "string"] union would order all int values before all string values, with the ints and strings themselves ordered as defined above.
- record data is ordered lexicographically by field. If a field specifies that its order is:
- “ascending”, then the order of its values is unaltered.
- “descending”, then the order of its values is reversed.
- “ignore”, then its values are ignored when sorting.
- map data may not be compared. It is an error to attempt to compare data containing maps unless those maps are in an
"order":"ignore" record field.
Object Container Files
Avro includes a simple object container file format. A file has a schema, and all objects stored in the file must be written according to that schema, using binary encoding. Objects are stored in blocks that may be compressed. Syncronization markers are used between blocks to permit efficient splitting of files for MapReduce processing.
Files may include arbitrary user-specified metadata.
A file consists of:
- A file header, followed by
- one or more file data blocks.
A file header consists of:
- Four bytes, ASCII ‘O’, ‘b’, ‘j’, followed by 1.
- file metadata, including the schema.
- The 16-byte, randomly-generated sync marker for this file.
File metadata is written as if defined by the following map schema:
{"type": "map", "values": "bytes"}
All metadata properties that start with “avro.” are reserved. The following file metadata properties are currently used:
- avro.schema contains the schema of objects stored in the file, as JSON data (required).
- avro.codec the name of the compression codec used to compress blocks, as a string. Implementations are required to support the following codecs: “null” and “deflate”. If codec is absent, it is assumed to be “null”. The codecs are described with more detail below.
A file header is thus described by the following schema:
{"type": "record", "name": "org.apache.avro.file.Header",
"fields" : [
{"name": "magic", "type": {"type": "fixed", "name": "Magic", "size": 4}},
{"name": "meta", "type": {"type": "map", "values": "bytes"}},
{"name": "sync", "type": {"type": "fixed", "name": "Sync", "size": 16}}
]
}
A file data block consists of:
- A long indicating the count of objects in this block.
- A long indicating the size in bytes of the serialized objects in the current block, after any codec is applied
- The serialized objects. If a codec is specified, this is compressed by that codec.
- The file’s 16-byte sync marker.
A file data block is thus described by the following schema:
{"type": "record", "name": "org.apache.avro.file.DataBlock",
"fields" : [
{"name": "count", "type": "long"},
{"name": "data", "type": "bytes"},
{"name": "sync", "type": {"type": "fixed", "name": "Sync", "size": 16}}
]
}
Each block’s binary data can be efficiently extracted or skipped without deserializing the contents. The combination of block size, object counts, and sync markers enable detection of corrupt blocks and help ensure data integrity.
Required Codecs
null
The “null” codec simply passes through data uncompressed.
deflate
The “deflate” codec writes the data block using the deflate algorithm as specified in RFC 1951, and typically implemented using the zlib library. Note that this format (unlike the “zlib format” in RFC 1950) does not have a checksum.
Optional Codecs
bzip2
The “bzip2” codec uses the bzip2 compression library.
snappy
The “snappy” codec uses Google’s Snappy compression library. Each compressed block is followed by the 4-byte, big-endian CRC32 checksum of the uncompressed data in the block.
xz
The “xz” codec uses the XZ compression library.
zstandard
The “zstandard” codec uses Facebook’s Zstandard compression library.
Protocol Declaration
Avro protocols describe RPC interfaces. Like schemas, they are defined with JSON text.
A protocol is a JSON object with the following attributes:
- protocol, a string, the name of the protocol (required);
- namespace, an optional string that qualifies the name (optional);
- doc, an optional string describing this protocol;
- types, an optional list of definitions of named types (records, enums, fixed and errors). An error definition is just like a record definition except it uses “error” instead of “record”. Note that forward references to named types are not permitted.
- messages, an optional JSON object whose keys are message names and whose values are objects whose attributes are described below. No two messages may have the same name.
The name and namespace qualification rules defined for schema objects apply to protocols as well.
Messages
A message has attributes:
- a doc, an optional description of the message,
- a request, a list of named, typed parameter schemas (this has the same form as the fields of a record declaration);
- a response schema;
- an optional union of declared error schemas. The effective union has “string” prepended to the declared union, to permit transmission of undeclared “system” errors. For example, if the declared error union is
["AccessError"], then the effective union is ["string", "AccessError"]. When no errors are declared, the effective error union is ["string"]. Errors are serialized using the effective union; however, a protocol’s JSON declaration contains only the declared union.
- an optional one-way boolean parameter.
A request parameter list is processed equivalently to an anonymous record. Since record field lists may vary between reader and writer, request parameters may also differ between the caller and responder, and such differences are resolved in the same manner as record field differences.
The one-way parameter may only be true when the response type is "null" and no errors are listed.
Sample Protocol
For example, one may define a simple HelloWorld protocol with:
{
"namespace": "com.acme",
"protocol": "HelloWorld",
"doc": "Protocol Greetings",
"types": [
{"name": "Greeting", "type": "record", "fields": [
{"name": "message", "type": "string"}]},
{"name": "Curse", "type": "error", "fields": [
{"name": "message", "type": "string"}]}
],
"messages": {
"hello": {
"doc": "Say hello.",
"request": [{"name": "greeting", "type": "Greeting" }],
"response": "Greeting",
"errors": ["Curse"]
}
}
}
Message Transport
Messages may be transmitted via different transport mechanisms.
To the transport, a message is an opaque byte sequence.
A transport is a system that supports:
- transmission of request messages
- receipt of corresponding response messages
Servers may send a response message back to the client corresponding to a request message. The mechanism of correspondence is transport-specific. For example, in HTTP it is implicit, since HTTP directly supports requests and responses. But a transport that multiplexes many client threads over a single socket would need to tag messages with unique identifiers.
Transports may be either stateless or stateful. In a stateless transport, messaging assumes no established connection state, while stateful transports establish connections that may be used for multiple messages. This distinction is discussed further in the handshake section below.
HTTP as Transport
When HTTP is used as a transport, each Avro message exchange is an HTTP request/response pair. All messages of an Avro protocol should share a single URL at an HTTP server. Other protocols may also use that URL. Both normal and error Avro response messages should use the 200 (OK) response code. The chunked encoding may be used for requests and responses, but, regardless the Avro request and response are the entire content of an HTTP request and response. The HTTP Content-Type of requests and responses should be specified as “avro/binary”. Requests should be made using the POST method.
HTTP is used by Avro as a stateless transport.
Message Framing
Avro messages are framed as a list of buffers.
Framing is a layer between messages and the transport. It exists to optimize certain operations.
The format of framed message data is:
- a series of buffers, where each buffer consists of:
- a four-byte, big-endian buffer length, followed by
- that many bytes of buffer data.
- a message is always terminated by a zero-length buffer.
Framing is transparent to request and response message formats (described below). Any message may be presented as a single or multiple buffers.
Framing can permit readers to more efficiently get different buffers from different sources and for writers to more efficiently store different buffers to different destinations. In particular, it can reduce the number of times large binary objects are copied. For example, if an RPC parameter consists of a megabyte of file data, that data can be copied directly to a socket from a file descriptor, and, on the other end, it could be written directly to a file descriptor, never entering user space.
A simple, recommended, framing policy is for writers to create a new segment whenever a single binary object is written that is larger than a normal output buffer. Small objects are then appended in buffers, while larger objects are written as their own buffers. When a reader then tries to read a large object the runtime can hand it an entire buffer directly, without having to copy it.
Handshake
The purpose of the handshake is to ensure that the client and the server have each other’s protocol definition, so that the client can correctly deserialize responses, and the server can correctly deserialize requests. Both clients and servers should maintain a cache of recently seen protocols, so that, in most cases, a handshake will be completed without extra round-trip network exchanges or the transmission of full protocol text.
RPC requests and responses may not be processed until a handshake has been completed. With a stateless transport, all requests and responses are prefixed by handshakes. With a stateful transport, handshakes are only attached to requests and responses until a successful handshake response has been returned over a connection. After this, request and response payloads are sent without handshakes for the lifetime of that connection.
The handshake process uses the following record schemas:
{
"type": "record",
"name": "HandshakeRequest", "namespace":"org.apache.avro.ipc",
"fields": [
{"name": "clientHash",
"type": {"type": "fixed", "name": "MD5", "size": 16}},
{"name": "clientProtocol", "type": ["null", "string"]},
{"name": "serverHash", "type": "MD5"},
{"name": "meta", "type": ["null", {"type": "map", "values": "bytes"}]}
]
}
{
"type": "record",
"name": "HandshakeResponse", "namespace": "org.apache.avro.ipc",
"fields": [
{"name": "match",
"type": {"type": "enum", "name": "HandshakeMatch",
"symbols": ["BOTH", "CLIENT", "NONE"]}},
{"name": "serverProtocol",
"type": ["null", "string"]},
{"name": "serverHash",
"type": ["null", {"type": "fixed", "name": "MD5", "size": 16}]},
{"name": "meta",
"type": ["null", {"type": "map", "values": "bytes"}]}
]
}
- A client first prefixes each request with a
HandshakeRequest containing just the hash of its protocol and of the server’s protocol (clientHash!=null, clientProtocol=null, serverHash!=null), where the hashes are 128-bit MD5 hashes of the JSON protocol text. If a client has never connected to a given server, it sends its hash as a guess of the server’s hash, otherwise it sends the hash that it previously obtained from this server.
The server responds with a HandshakeResponse containing one of:
match=BOTH, serverProtocol=null, serverHash=null if the client sent the valid hash of the server’s protocol and the server knows what protocol corresponds to the client’s hash. In this case, the request is complete and the response data immediately follows the HandshakeResponse.
match=CLIENT, serverProtocol!=null, serverHash!=null if the server has previously seen the client’s protocol, but the client sent an incorrect hash of the server’s protocol. The request is complete and the response data immediately follows the HandshakeResponse. The client must use the returned protocol to process the response and should also cache that protocol and its hash for future interactions with this server.
match=NONE if the server has not previously seen the client’s protocol. The serverHash and serverProtocol may also be non-null if the server’s protocol hash was incorrect.
In this case the client must then re-submit its request with its protocol text (clientHash!=null, clientProtocol!=null, serverHash!=null) and the server should respond with a successful match (match=BOTH, serverProtocol=null, serverHash=null) as above.
The meta field is reserved for future handshake enhancements.
A call consists of a request message paired with its resulting response or error message. Requests and responses contain extensible metadata, and both kinds of messages are framed as described above.
The format of a call request is:
- request metadata, a map with values of type bytes
- the message name, an Avro string, followed by
- the message parameters. Parameters are serialized according to the message’s request declaration.
When the empty string is used as a message name a server should ignore the parameters and return an empty response. A client may use this to ping a server or to perform a handshake without sending a protocol message.
When a message is declared one-way and a stateful connection has been established by a successful handshake response, no response data is sent. Otherwise the format of the call response is:
- response metadata, a map with values of type bytes
- a one-byte error flag boolean, followed by either:
- if the error flag is false, the message response, serialized per the message’s response schema.
- if the error flag is true, the error, serialized per the message’s effective error union schema.
Schema Resolution
A reader of Avro data, whether from an RPC or a file, can always parse that data because the original schema must be provided along with the data. However, the reader may be programmed to read data into a different schema. For example, if the data was written with a different version of the software than it is read, then fields may have been added or removed from records. This section specifies how such schema differences should be resolved.
We refer to the schema used to write the data as the writer’s schema, and the schema that the application expects the reader’s schema. Differences between these should be resolved as follows:
-
It is an error if the two schemas do not match.
To match, one of the following must hold:
- both schemas are arrays whose item types match
- both schemas are maps whose value types match
- both schemas are enums whose (unqualified) names match
- both schemas are fixed whose sizes and (unqualified) names match
- both schemas are records with the same (unqualified) name
- either schema is a union
- both schemas have same primitive type
- the writer’s schema may be promoted to the reader’s as follows:
- int is promotable to long, float, or double
- long is promotable to float or double
- float is promotable to double
- string is promotable to bytes
- bytes is promotable to string
-
if both are records:
- the ordering of fields may be different: fields are matched by name.
- schemas for fields with the same name in both records are resolved recursively.
- if the writer’s record contains a field with a name not present in the reader’s record, the writer’s value for that field is ignored.
- if the reader’s record schema has a field that contains a default value, and writer’s schema does not have a field with the same name, then the reader should use the default value from its field.
- if the reader’s record schema has a field with no default value, and writer’s schema does not have a field with the same name, an error is signalled.
-
if both are enums:
if the writer’s symbol is not present in the reader’s enum and the reader has a default value, then that value is used, otherwise an error is signalled.
-
if both are arrays:
This resolution algorithm is applied recursively to the reader’s and writer’s array item schemas.
-
if both are maps:
This resolution algorithm is applied recursively to the reader’s and writer’s value schemas.
-
if both are unions:
The first schema in the reader’s union that matches the selected writer’s union schema is recursively resolved against it. if none match, an error is signalled.
-
if reader’s is a union, but writer’s is not
The first schema in the reader’s union that matches the writer’s schema is recursively resolved against it. If none match, an error is signalled.
-
if writer’s is a union, but reader’s is not
If the reader’s schema matches the selected writer’s schema, it is recursively resolved against it. If they do not match, an error is signalled.
A schema’s doc fields are ignored for the purposes of schema resolution. Hence, the doc portion of a schema may be dropped at serialization.
One of the defining characteristics of Avro is that a reader must use the schema used by the writer of the data in order to know how to read the data. This assumption results in a data format that’s compact and also amenable to many forms of schema evolution. However, the specification so far has not defined what it means for the reader to have the “same” schema as the writer. Does the schema need to be textually identical? Well, clearly adding or removing some whitespace to a JSON expression does not change its meaning. At the same time, reordering the fields of records clearly does change the meaning. So what does it mean for a reader to have “the same” schema as a writer?
Parsing Canonical Form is a transformation of a writer’s schema that let’s us define what it means for two schemas to be “the same” for the purpose of reading data written against the schema. It is called Parsing Canonical Form because the transformations strip away parts of the schema, like “doc” attributes, that are irrelevant to readers trying to parse incoming data. It is called Canonical Form because the transformations normalize the JSON text (such as the order of attributes) in a way that eliminates unimportant differences between schemas. If the Parsing Canonical Forms of two different schemas are textually equal, then those schemas are “the same” as far as any reader is concerned, i.e., there is no serialized data that would allow a reader to distinguish data generated by a writer using one of the original schemas from data generated by a writing using the other original schema. (We sketch a proof of this property in a companion document.)
The next subsection specifies the transformations that define Parsing Canonical Form. But with a well-defined canonical form, it can be convenient to go one step further, transforming these canonical forms into simple integers (“fingerprints”) that can be used to uniquely identify schemas. The subsection after next recommends some standard practices for generating such fingerprints.
Assuming an input schema (in JSON form) that’s already UTF-8 text for a valid Avro schema (including all quotes as required by JSON), the following transformations will produce its Parsing Canonical Form:
- [PRIMITIVES] Convert primitive schemas to their simple form (e.g., int instead of
{"type":"int"}).
- [FULLNAMES] Replace short names with fullnames, using applicable namespaces to do so. Then eliminate namespace attributes, which are now redundant.
- [STRIP] Keep only attributes that are relevant to parsing data, which are: type, name, fields, symbols, items, values, size. Strip all others (e.g., doc and aliases).
- [ORDER] Order the appearance of fields of JSON objects as follows: name, type, fields, symbols, items, values, size. For example, if an object has type, name, and size fields, then the name field should appear first, followed by the type and then the size fields.
- [STRINGS] For all JSON string literals in the schema text, replace any escaped characters (e.g., \uXXXX escapes) with their UTF-8 equivalents.
- [INTEGERS] Eliminate quotes around and any leading zeros in front of JSON integer literals (which appear in the size attributes of fixed schemas).
- [WHITESPACE] Eliminate all whitespace in JSON outside of string literals.
Schema Fingerprints
“[A] fingerprinting algorithm is a procedure that maps an arbitrarily large data item (such as a computer file) to a much shorter bit string, its fingerprint, that uniquely identifies the original data for all practical purposes” (quoted from Wikipedia). In the Avro context, fingerprints of Parsing Canonical Form can be useful in a number of applications; for example, to cache encoder and decoder objects, to tag data items with a short substitute for the writer’s full schema, and to quickly negotiate common-case schemas between readers and writers.
In designing fingerprinting algorithms, there is a fundamental trade-off between the length of the fingerprint and the probability of collisions. To help application designers find appropriate points within this trade-off space, while encouraging interoperability and ease of implementation, we recommend using one of the following three algorithms when fingerprinting Avro schemas:
- When applications can tolerate longer fingerprints, we recommend using the SHA-256 digest algorithm to generate 256-bit fingerprints of Parsing Canonical Forms. Most languages today have SHA-256 implementations in their libraries.
- At the opposite extreme, the smallest fingerprint we recommend is a 64-bit Rabin fingerprint. Below, we provide pseudo-code for this algorithm that can be easily translated into any programming language. 64-bit fingerprints should guarantee uniqueness for schema caches of up to a million entries (for such a cache, the chance of a collision is 3E-8). We don’t recommend shorter fingerprints, as the chances of collisions is too great (for example, with 32-bit fingerprints, a cache with as few as 100,000 schemas has a 50% chance of having a collision).
- Between these two extremes, we recommend using the MD5 message digest to generate 128-bit fingerprints. These make sense only where very large numbers of schemas are being manipulated (tens of millions); otherwise, 64-bit fingerprints should be sufficient. As with SHA-256, MD5 implementations are found in most libraries today.
These fingerprints are not meant to provide any security guarantees, even the longer SHA-256-based ones. Most Avro applications should be surrounded by security measures that prevent attackers from writing random data and otherwise interfering with the consumers of schemas. We recommend that these surrounding mechanisms be used to prevent collision and pre-image attacks (i.e., “forgery”) on schema fingerprints, rather than relying on the security properties of the fingerprints themselves.
Rabin fingerprints are cyclic redundancy checks computed using irreducible polynomials. In the style of the Appendix of RFC 1952 (pg 10), which defines the CRC-32 algorithm, here’s our definition of the 64-bit AVRO fingerprinting algorithm:
long fingerprint64(byte[] buf) {
if (FP_TABLE == null) initFPTable();
long fp = EMPTY;
for (int i = 0; i < buf.length; i++)
fp = (fp >>> 8) ^ FP_TABLE[(int)(fp ^ buf[i]) & 0xff];
return fp;
}
static long EMPTY = 0xc15d213aa4d7a795L;
static long[] FP_TABLE = null;
void initFPTable() {
FP_TABLE = new long[256];
for (int i = 0; i < 256; i++) {
long fp = i;
for (int j = 0; j < 8; j++)
fp = (fp >>> 1) ^ (EMPTY & -(fp & 1L));
FP_TABLE[i] = fp;
}
}
Readers interested in the mathematics behind this algorithm may want to read Chapter 14 of the Second Edition of Hacker’s Delight. (Unlike RFC-1952 and the book chapter, we prepend a single one bit to messages. We do this because CRCs ignore leading zero bits, which can be problematic. Our code prepends a one-bit by initializing fingerprints using EMPTY, rather than initializing using zero as in RFC-1952 and the book chapter.)
Logical Types
A logical type is an Avro primitive or complex type with extra attributes to represent a derived type. The attribute logicalType must always be present for a logical type, and is a string with the name of one of the logical types listed later in this section. Other attributes may be defined for particular logical types.
A logical type is always serialized using its underlying Avro type so that values are encoded in exactly the same way as the equivalent Avro type that does not have a logicalType attribute. Language implementations may choose to represent logical types with an appropriate native type, although this is not required.
Language implementations must ignore unknown logical types when reading, and should use the underlying Avro type. If a logical type is invalid, for example a decimal with scale greater than its precision, then implementations should ignore the logical type and use the underlying Avro type.
Decimal
The decimal logical type represents an arbitrary-precision signed decimal number of the form unscaled × 10-scale.
A decimal logical type annotates Avro bytes or fixed types. The byte array must contain the two’s-complement representation of the unscaled integer value in big-endian byte order. The scale is fixed, and is specified using an attribute.
The following attributes are supported:
- scale, a JSON integer representing the scale (optional). If not specified the scale is 0.
- precision, a JSON integer representing the (maximum) precision of decimals stored in this type (required).
For example, the following schema represents decimal numbers with a maximum precision of 4 and a scale of 2:
{
"type": "bytes",
"logicalType": "decimal",
"precision": 4,
"scale": 2
}
Precision must be a positive integer greater than zero. If the underlying type is a fixed, then the precision is limited by its size. An array of length n can store at most floor(log10(28 × n - 1 - 1)) base-10 digits of precision.
Scale must be zero or a positive integer less than or equal to the precision.
For the purposes of schema resolution, two schemas that are decimal logical types match if their scales and precisions match.
UUID
The uuid logical type represents a random generated universally unique identifier (UUID).
A uuid logical type annotates an Avro string. The string has to conform with RFC-4122
Date
The date logical type represents a date within the calendar, with no reference to a particular time zone or time of day.
A date logical type annotates an Avro int, where the int stores the number of days from the unix epoch, 1 January 1970 (ISO calendar).
The following schema represents a date:
{
"type": "int",
"logicalType": "date"
}
Time (millisecond precision)
The time-millis logical type represents a time of day, with no reference to a particular calendar, time zone or date, with a precision of one millisecond.
A time-millis logical type annotates an Avro int, where the int stores the number of milliseconds after midnight, 00:00:00.000.
Time (microsecond precision)
The time-micros logical type represents a time of day, with no reference to a particular calendar, time zone or date, with a precision of one microsecond.
A time-micros logical type annotates an Avro long, where the long stores the number of microseconds after midnight, 00:00:00.000000.
Timestamp (millisecond precision)
The timestamp-millis logical type represents an instant on the global timeline, independent of a particular time zone or calendar, with a precision of one millisecond. Please note that time zone information gets lost in this process. Upon reading a value back, we can only reconstruct the instant, but not the original representation. In practice, such timestamps are typically displayed to users in their local time zones, therefore they may be displayed differently depending on the execution environment.
A timestamp-millis logical type annotates an Avro long, where the long stores the number of milliseconds from the unix epoch, 1 January 1970 00:00:00.000 UTC.
Timestamp (microsecond precision)
The timestamp-micros logical type represents an instant on the global timeline, independent of a particular time zone or calendar, with a precision of one microsecond. Please note that time zone information gets lost in this process. Upon reading a value back, we can only reconstruct the instant, but not the original representation. In practice, such timestamps are typically displayed to users in their local time zones, therefore they may be displayed differently depending on the execution environment.
A timestamp-micros logical type annotates an Avro long, where the long stores the number of microseconds from the unix epoch, 1 January 1970 00:00:00.000000 UTC.
Local timestamp (millisecond precision)
The local-timestamp-millis logical type represents a timestamp in a local timezone, regardless of what specific time zone is considered local, with a precision of one millisecond.
A local-timestamp-millis logical type annotates an Avro long, where the long stores the number of milliseconds, from 1 January 1970 00:00:00.000.
Local timestamp (microsecond precision)
The local-timestamp-micros logical type represents a timestamp in a local timezone, regardless of what specific time zone is considered local, with a precision of one microsecond.
A local-timestamp-micros logical type annotates an Avro long, where the long stores the number of microseconds, from 1 January 1970 00:00:00.000000.
Duration
The duration logical type represents an amount of time defined by a number of months, days and milliseconds. This is not equivalent to a number of milliseconds, because, depending on the moment in time from which the duration is measured, the number of days in the month and number of milliseconds in a day may differ. Other standard periods such as years, quarters, hours and minutes can be expressed through these basic periods.
A duration logical type annotates Avro fixed type of size 12, which stores three little-endian unsigned integers that represent durations at different granularities of time. The first stores a number in months, the second stores a number in days, and the third stores a number in milliseconds.
1.4 - MapReduce guide
Avro provides a convenient way to represent complex data structures within a Hadoop MapReduce job. Avro data can be used as both input to and output from a MapReduce job, as well as the intermediate format. The example in this guide uses Avro data for all three, but it’s possible to mix and match; for instance, MapReduce can be used to aggregate a particular field in an Avro record.
This guide assumes basic familiarity with both Hadoop MapReduce and Avro. See the Hadoop documentation and the Avro getting started guide for introductions to these projects. This guide uses the old MapReduce API (org.apache.hadoop.mapred) and the new MapReduce API (org.apache.hadoop.mapreduce).
Setup
The code from this guide is included in the Avro docs under examples/mr-example. The example is set up as a Maven project that includes the necessary Avro and MapReduce dependencies and the Avro Maven plugin for code generation, so no external jars are needed to run the example. In particular, the POM includes the following dependencies:
<dependency>
<groupId>org.apache.avro</groupId>
<artifactId>avro</artifactId>
<version>1.11.1</version>
</dependency>
<dependency>
<groupId>org.apache.avro</groupId>
<artifactId>avro-mapred</artifactId>
<version>1.11.1</version>
</dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-client</artifactId>
<version>3.1.2</version>
</dependency>
And the following plugin:
<plugin>
<groupId>org.apache.avro</groupId>
<artifactId>avro-maven-plugin</artifactId>
<version>1.11.1</version>
<executions>
<execution>
<phase>generate-sources</phase>
<goals>
<goal>schema</goal>
</goals>
<configuration>
<sourceDirectory>${project.basedir}/../</sourceDirectory>
<outputDirectory>${project.basedir}/target/generated-sources/</outputDirectory>
</configuration>
</execution>
</executions>
</plugin>
If you do not configure the sourceDirectory and outputDirectory properties, the defaults will be used. The sourceDirectory property defaults to src/main/avro. The outputDirectory property defaults to target/generated-sources. You can change the paths to match your project layout.
Alternatively, Avro jars can be downloaded directly from the Apache Avro™ Releases page. The relevant Avro jars for this guide are avro-1.11.1.jar and avro-mapred-1.11.1.jar, as well as avro-tools-1.11.1.jar for code generation and viewing Avro data files as JSON. In addition, you will need to install Hadoop in order to use MapReduce.
Example: ColorCount
Below is a simple example of a MapReduce that uses Avro. There is an example for both the old (org.apache.hadoop.mapred) and new (org.apache.hadoop.mapreduce) APIs under examples/mr-example/src/main/java/example/. MapredColorCount is the example for the older mapred API while MapReduceColorCount is the example for the newer mapreduce API. Both examples are below, but we will detail the mapred API in our subsequent examples.
MapredColorCount.java:
package example;
import java.io.IOException;
import org.apache.avro.*;
import org.apache.avro.Schema.Type;
import org.apache.avro.mapred.*;
import org.apache.hadoop.conf.*;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.mapred.*;
import org.apache.hadoop.util.*;
import example.avro.User;
public class MapredColorCount extends Configured implements Tool {
public static class ColorCountMapper extends AvroMapper<User, Pair<CharSequence, Integer>> {
@Override
public void map(User user, AvroCollector<Pair<CharSequence, Integer>> collector, Reporter reporter)
throws IOException {
CharSequence color = user.getFavoriteColor();
// We need this check because the User.favorite_color field has type ["string", "null"]
if (color == null) {
color = "none";
}
collector.collect(new Pair<CharSequence, Integer>(color, 1));
}
}
public static class ColorCountReducer extends AvroReducer<CharSequence, Integer,
Pair<CharSequence, Integer>> {
@Override
public void reduce(CharSequence key, Iterable<Integer> values,
AvroCollector<Pair<CharSequence, Integer>> collector,
Reporter reporter)
throws IOException {
int sum = 0;
for (Integer value : values) {
sum += value;
}
collector.collect(new Pair<CharSequence, Integer>(key, sum));
}
}
public int run(String[] args) throws Exception {
if (args.length != 2) {
System.err.println("Usage: MapredColorCount <input path> <output path>");
return -1;
}
JobConf conf = new JobConf(getConf(), MapredColorCount.class);
conf.setJobName("colorcount");
FileInputFormat.setInputPaths(conf, new Path(args[0]));
FileOutputFormat.setOutputPath(conf, new Path(args[1]));
AvroJob.setMapperClass(conf, ColorCountMapper.class);
AvroJob.setReducerClass(conf, ColorCountReducer.class);
// Note that AvroJob.setInputSchema and AvroJob.setOutputSchema set
// relevant config options such as input/output format, map output
// classes, and output key class.
AvroJob.setInputSchema(conf, User.getClassSchema());
AvroJob.setOutputSchema(conf, Pair.getPairSchema(Schema.create(Type.STRING),
Schema.create(Type.INT)));
JobClient.runJob(conf);
return 0;
}
public static void main(String[] args) throws Exception {
int res = ToolRunner.run(new Configuration(), new MapredColorCount(), args);
System.exit(res);
}
}
MapReduceColorCount.java:
package example;
import java.io.IOException;
import org.apache.avro.Schema;
import org.apache.avro.mapred.AvroKey;
import org.apache.avro.mapred.AvroValue;
import org.apache.avro.mapreduce.AvroJob;
import org.apache.avro.mapreduce.AvroKeyInputFormat;
import org.apache.avro.mapreduce.AvroKeyValueOutputFormat;
import org.apache.hadoop.conf.Configured;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.IntWritable;
import org.apache.hadoop.io.NullWritable;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.mapreduce.Job;
import org.apache.hadoop.mapreduce.Mapper;
import org.apache.hadoop.mapreduce.Reducer;
import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
import org.apache.hadoop.util.Tool;
import org.apache.hadoop.util.ToolRunner;
import example.avro.User;
public class MapReduceColorCount extends Configured implements Tool {
public static class ColorCountMapper extends
Mapper<AvroKey<User>, NullWritable, Text, IntWritable> {
@Override
public void map(AvroKey<User> key, NullWritable value, Context context)
throws IOException, InterruptedException {
CharSequence color = key.datum().getFavoriteColor();
if (color == null) {
color = "none";
}
context.write(new Text(color.toString()), new IntWritable(1));
}
}
public static class ColorCountReducer extends
Reducer<Text, IntWritable, AvroKey<CharSequence>, AvroValue<Integer>> {
@Override
public void reduce(Text key, Iterable<IntWritable> values,
Context context) throws IOException, InterruptedException {
int sum = 0;
for (IntWritable value : values) {
sum += value.get();
}
context.write(new AvroKey<CharSequence>(key.toString()), new AvroValue<Integer>(sum));
}
}
public int run(String[] args) throws Exception {
if (args.length != 2) {
System.err.println("Usage: MapReduceColorCount <input path> <output path>");
return -1;
}
Job job = new Job(getConf());
job.setJarByClass(MapReduceColorCount.class);
job.setJobName("Color Count");
FileInputFormat.setInputPaths(job, new Path(args[0]));
FileOutputFormat.setOutputPath(job, new Path(args[1]));
job.setInputFormatClass(AvroKeyInputFormat.class);
job.setMapperClass(ColorCountMapper.class);
AvroJob.setInputKeySchema(job, User.getClassSchema());
job.setMapOutputKeyClass(Text.class);
job.setMapOutputValueClass(IntWritable.class);
job.setOutputFormatClass(AvroKeyValueOutputFormat.class);
job.setReducerClass(ColorCountReducer.class);
AvroJob.setOutputKeySchema(job, Schema.create(Schema.Type.STRING));
AvroJob.setOutputValueSchema(job, Schema.create(Schema.Type.INT));
return (job.waitForCompletion(true) ? 0 : 1);
}
public static void main(String[] args) throws Exception {
int res = ToolRunner.run(new MapReduceColorCount(), args);
System.exit(res);
}
}
ColorCount reads in data files containing User records, defined in examples/user.avsc, and counts the number of instances of each favorite color. (This example draws inspiration from the canonical WordCount MapReduce application.) This example uses the old MapReduce API. See MapReduceAvroWordCount, found under doc/examples/mr-example/src/main/java/example/ to see the new MapReduce API example. The User schema is defined as follows:
{"namespace": "example.avro",
"type": "record",
"name": "User",
"fields": [
{"name": "name", "type": "string"},
{"name": "favorite_number", "type": ["int", "null"]},
{"name": "favorite_color", "type": ["string", "null"]}
]
}
This schema is compiled into the User class used by ColorCount via the Avro Maven plugin (see examples/mr-example/pom.xml for how this is set up).
ColorCountMapper essentially takes a User as input and extracts the User’s favorite color, emitting the key-value pair <favoriteColor, 1>. ColorCountReducer then adds up how many occurrences of a particular favorite color were emitted, and outputs the result as a Pair record. These Pairs are serialized to an Avro data file.
Running ColorCount
The ColorCount application is provided as a Maven project in the Avro docs under examples/mr-example. To build the project, including the code generation of the User schema, run:
Next, run GenerateData from examples/mr-examples to create an Avro data file, input/users.avro, containing 20 Users with favorite colors chosen randomly from a list:
mvn exec:java -q -Dexec.mainClass=example.GenerateData
Besides creating the data file, GenerateData prints the JSON representations of the Users generated to stdout, for example:
{"name": "user", "favorite_number": null, "favorite_color": "red"}
{"name": "user", "favorite_number": null, "favorite_color": "green"}
{"name": "user", "favorite_number": null, "favorite_color": "purple"}
{"name": "user", "favorite_number": null, "favorite_color": null}
...
Now we’re ready to run ColorCount. We specify our freshly-generated input folder as the input path and output as our output folder (note that MapReduce will not start a job if the output folder already exists):
mvn exec:java -q -Dexec.mainClass=example.MapredColorCount -Dexec.args="input output"
Once ColorCount completes, checking the contents of the new output directory should yield the following:
$ ls output/
part-00000.avro _SUCCESS
You can check the contents of the generated Avro file using the avro-tools jar:
$ java -jar /path/to/avro-tools-1.11.1.jar tojson output/part-00000.avro
{"value": 3, "key": "blue"}
{"value": 7, "key": "green"}
{"value": 1, "key": "none"}
{"value": 2, "key": "orange"}
{"value": 3, "key": "purple"}
{"value": 2, "key": "red"}
{"value": 2, "key": "yellow"}
Now let’s go over the ColorCount example in detail.
AvroMapper - org.apache.hadoop.mapred API
The easiest way to use Avro data files as input to a MapReduce job is to subclass AvroMapper. An AvroMapper defines a map function that takes an Avro datum as input and outputs a key/value pair represented as a Pair record. In the ColorCount example, ColorCountMapper is an AvroMapper that takes a User as input and outputs a Pair<CharSequence, Integer>>, where the CharSequence key is the user’s favorite color and the Integer value is 1.
public static class ColorCountMapper extends AvroMapper<User, Pair<CharSequence, Integer>> {
@Override
public void map(User user, AvroCollector<Pair<CharSequence, Integer>> collector, Reporter reporter)
throws IOException {
CharSequence color = user.getFavoriteColor();
// We need this check because the User.favorite_color field has type ["string", "null"]
if (color == null) {
color = "none";
}
collector.collect(new Pair<CharSequence, Integer>(color, 1));
}
}
In order to use our AvroMapper, we must call AvroJob.setMapperClass and AvroJob.setInputSchema.
AvroJob.setMapperClass(conf, ColorCountMapper.class);
AvroJob.setInputSchema(conf, User.getClassSchema());
Note that AvroMapper does not implement the Mapper interface. Under the hood, the specified Avro data files are deserialized into AvroWrappers containing the actual data, which are processed by a Mapper that calls the configured AvroMapper’s map function. AvroJob.setInputSchema sets up the relevant configuration parameters needed to make this happen, thus you should not need to call JobConf.setMapperClass, JobConf.setInputFormat, JobConf.setMapOutputKeyClass, JobConf.setMapOutputValueClass, or JobConf.setOutputKeyComparatorClass.
Mapper - org.apache.hadoop.mapreduce API
This document will not go into all the differences between the mapred and mapreduce APIs, however will describe the main differences. As you can see, ColorCountMapper is now a subclass of the Hadoop Mapper class and is passed an AvroKey as it’s key. Additionally, the AvroJob method calls were slightly changed.
public static class ColorCountMapper extends
Mapper<AvroKey<User>, NullWritable, Text, IntWritable> {
@Override
public void map(AvroKey<User> key, NullWritable value, Context context)
throws IOException, InterruptedException {
CharSequence color = key.datum().getFavoriteColor();
if (color == null) {
color = "none";
}
context.write(new Text(color.toString()), new IntWritable(1));
}
}
AvroReducer - org.apache.hadoop.mapred API
Analogously to AvroMapper, an AvroReducer defines a reducer function that takes the key/value types output by an AvroMapper (or any mapper that outputs Pairs) and outputs a key/value pair represented a Pair record. In the ColorCount example, ColorCountReducer is an AvroReducer that takes the CharSequence key representing a favorite color and the Iterable<Integer> representing the counts for that color (they should all be 1 in this example) and adds up the counts.
public static class ColorCountReducer extends AvroReducer<CharSequence, Integer,
Pair<CharSequence, Integer>> {
@Override
public void reduce(CharSequence key, Iterable<Integer> values,
AvroCollector<Pair<CharSequence, Integer>> collector,
Reporter reporter)
throws IOException {
int sum = 0;
for (Integer value : values) {
sum += value;
}
collector.collect(new Pair<CharSequence, Integer>(key, sum));
}
}
In order to use our AvroReducer, we must call AvroJob.setReducerClass and AvroJob.setOutputSchema.
AvroJob.setReducerClass(conf, ColorCountReducer.class);
AvroJob.setOutputSchema(conf, Pair.getPairSchema(Schema.create(Type.STRING),
Schema.create(Type.INT)));
Note that AvroReducer does not implement the Reducer interface. The intermediate Pairs output by the mapper are split into AvroKeys and AvroValues, which are processed by a Reducer that calls the configured AvroReducer’s reduce function. AvroJob.setOutputSchema sets up the relevant configuration parameters needed to make this happen, thus you should not need to call JobConf.setReducerClass, JobConf.setOutputFormat, JobConf.setOutputKeyClass, JobConf.setMapOutputKeyClass, JobConf.setMapOutputValueClass, or JobConf.setOutputKeyComparatorClass.
Reduce - org.apache.hadoop.mapreduce API
As before we not detail every difference between the APIs. As with the Mapper change ColorCountReducer is now a subclass of Reducer and AvroKey and AvroValue are emitted. Additionally, the AvroJob method calls were slightly changed.
public static class ColorCountReducer extends
Reducer<Text, IntWritable, AvroKey<CharSequence>, AvroValue<Integer>> {
@Override
public void reduce(Text key, Iterable<IntWritable> values,
Context context) throws IOException, InterruptedException {
int sum = 0;
for (IntWritable value : values) {
sum += value.get();
}
context.write(new AvroKey<CharSequence>(key.toString()), new AvroValue<Integer>(sum));
}
}
Learning more
The mapred API allows users to mix Avro AvroMappers and AvroReducers with non-Avro Mappers and Reducers and the mapreduce API allows users input Avro and output non-Avro or vice versa.
The mapred package has API org.apache.avro.mapred documentation as does the org.apache.avro.mapreduce package. MapReduce API (org.apache.hadoop.mapreduce). Similarily to the mapreduce package, it’s possible with the mapred API to implement your own Mappers and Reducers directly using the public classes provided in these libraries. See the AvroWordCount application, found under examples/mr-example/src/main/java/example/AvroWordCount.java in the Avro documentation, for an example of implementing a Reducer that outputs Avro data using the old MapReduce API. See the MapReduceAvroWordCount application, found under examples/mr-example/src/main/java/example/MapReduceAvroWordCount.java in the Avro documentation, for an example of implementing a Reducer that outputs Avro data using the new MapReduce API.
1.5 - IDL Language
Introduction
This document defines Avro IDL, a higher-level language for authoring Avro schemata. Before reading this document, you should have familiarity with the concepts of schemata and protocols, as well as the various primitive and complex types available in Avro.
Overview
Purpose
The aim of the Avro IDL language is to enable developers to author schemata in a way that feels more similar to common programming languages like Java, C++, or Python. Additionally, the Avro IDL language may feel more familiar for those users who have previously used the interface description languages (IDLs) in other frameworks like Thrift, Protocol Buffers, or CORBA.
Usage
Each Avro IDL file defines a single Avro Protocol, and thus generates as its output a JSON-format Avro Protocol file with extension .avpr.
To convert a .avdl file into a .avpr file, it may be processed by the idl tool. For example:
$ java -jar avro-tools.jar idl src/test/idl/input/namespaces.avdl /tmp/namespaces.avpr
$ head /tmp/namespaces.avpr
{
"protocol" : "TestNamespace",
"namespace" : "avro.test.protocol",
The idl tool can also process input to and from stdin and stdout. See idl --help for full usage information.
A Maven plugin is also provided to compile .avdl files. To use it, add something like the following to your pom.xml:
<build>
<plugins>
<plugin>
<groupId>org.apache.avro</groupId>
<artifactId>avro-maven-plugin</artifactId>
<executions>
<execution>
<goals>
<goal>idl-protocol</goal>
</goals>
</execution>
</executions>
</plugin>
</plugins>
</build>
Defining a Protocol in Avro IDL
An Avro IDL file consists of exactly one protocol definition. The minimal protocol is defined by the following code:
This is equivalent to (and generates) the following JSON protocol definition:
{
"protocol" : "MyProtocol",
"types" : [ ],
"messages" : {
}
}
The namespace of the protocol may be changed using the @namespace annotation:
@namespace("mynamespace")
protocol MyProtocol {
}
This notation is used throughout Avro IDL as a way of specifying properties for the annotated element, as will be described later in this document.
Protocols in Avro IDL can contain the following items:
- Imports of external protocol and schema files.
- Definitions of named schemata, including records, errors, enums, and fixeds.
- Definitions of RPC messages
Imports
Files may be imported in one of three formats:
-
An IDL file may be imported with a statement like:
import idl "foo.avdl";
-
A JSON protocol file may be imported with a statement like:
import protocol "foo.avpr";
-
A JSON schema file may be imported with a statement like:
import schema "foo.avsc";
Messages and types in the imported file are added to this file’s protocol.
Imported file names are resolved relative to the current IDL file.
Defining an Enumeration
Enums are defined in Avro IDL using a syntax similar to C or Java. An Avro Enum supports optional default values. In the case that a reader schema is unable to recognize a symbol written by the writer, the reader will fall back to using the defined default value. This default is only used when an incompatible symbol is read. It is not used if the enum field is missing.
Example Writer Enum Definition
enum Shapes {
SQUARE, TRIANGLE, CIRCLE, OVAL
}
Example Reader Enum Definition
enum Shapes {
SQUARE, TRIANGLE, CIRCLE
} = CIRCLE;
In the above example, the reader will use the default value of CIRCLE whenever reading data written with the OVAL symbol of the writer. Also note that, unlike the JSON format, anonymous enums cannot be defined.
Defining a Fixed Length Field
Fixed fields are defined using the following syntax:
fixed MD5(16);
This example defines a fixed-length type called MD5 which contains 16 bytes.
Defining Records and Errors
Records are defined in Avro IDL using a syntax similar to a struct definition in C:
record Employee {
string name;
boolean active = true;
long salary;
}
The above example defines a record with the name “Employee” with three fields.
To define an error, simply use the keyword error instead of record. For example:
error Kaboom {
string explanation;
int result_code = -1;
}
Each field in a record or error consists of a type and a name, optional property annotations and an optional default value.
A type reference in Avro IDL must be one of:
- A primitive type
- A logical type
- A named schema defined prior to this usage in the same Protocol
- A complex type (array, map, or union)
Primitive Types
The primitive types supported by Avro IDL are the same as those supported by Avro’s JSON format. This list includes int, long, string, boolean, float, double, null, and bytes.
Logical Types
Some of the logical types supported by Avro’s JSON format are also supported by Avro IDL. The currently supported types are:
For example:
record Job {
string jobid;
date submitDate;
time_ms submitTime;
timestamp_ms finishTime;
decimal(9,2) finishRatio;
uuid pk = "a1a2a3a4-b1b2-c1c2-d1d2-d3d4d5d6d7d8";
}
Logical types can also be specified via an annotation, which is useful for logical types for which a keyword does not exist:
record Job {
string jobid;
@logicalType("timestamp-micros")
long finishTime;
}
References to Named Schemata
If a named schema has already been defined in the same Avro IDL file, it may be referenced by name as if it were a primitive type:
record Card {
Suit suit; // refers to the enum Card defined above
int number;
}
Default Values
Default values for fields may be optionally specified by using an equals sign after the field name followed by a JSON expression indicating the default value. This JSON is interpreted as described in the spec.
Complex Types
Arrays
Array types are written in a manner that will seem familiar to C++ or Java programmers. An array of any type t is denoted array<t>. For example, an array of strings is denoted array<string>, and a multidimensional array of Foo records would be array<array<Foo>>.
Maps
Map types are written similarly to array types. An array that contains values of type t is written map<t>. As in the JSON schema format, all maps contain string-type keys.
Unions
Union types are denoted as union { typeA, typeB, typeC, ... }. For example, this record contains a string field that is optional (unioned with null), and a field containing either a precise or a imprecise number:
record RecordWithUnion {
union { null, string } optionalString;
union { decimal(12, 6), float } number;
}
Note that the same restrictions apply to Avro IDL unions as apply to unions defined in the JSON format; namely, a record may not contain multiple elements of the same type. Also, fields/parameters that use the union type and have a default parameter must specify a default value of the same type as the first union type.
Because it occurs so often, there is a special shorthand to denote a union of null with another type. In the following snippet, the first three fields have identical types:
record RecordWithUnion {
union { null, string } optionalString1 = null;
string? optionalString2 = null;
string? optionalString3; // No default value
string? optionalString4 = "something";
}
Note that unlike explicit unions, the position of the null type is fluid; it will be the first or last type depending on the default value (if any). So in the example above, all fields are valid.
Defining RPC Messages
The syntax to define an RPC message within a Avro IDL protocol is similar to the syntax for a method declaration within a C header file or a Java interface. To define an RPC message add which takes two arguments named foo and bar, returning an int, simply include the following definition within the protocol:
int add(int foo, int bar = 0);
Message arguments, like record fields, may specify default values.
To define a message with no response, you may use the alias void, equivalent to the Avro null type:
void logMessage(string message);
If you have previously defined an error type within the same protocol, you may declare that a message can throw this error using the syntax:
void goKaboom() throws Kaboom;
To define a one-way message, use the keyword oneway after the parameter list, for example:
void fireAndForget(string message) oneway;
Other Language Features
All Java-style comments are supported within a Avro IDL file. Any text following // on a line is ignored, as is any text between /* and */, possibly spanning multiple lines.
Comments that begin with /** are used as the documentation string for the type or field definition that follows the comment.
Escaping Identifiers
Occasionally, one will need to use a reserved language keyword as an identifier. In order to do so, backticks (`) may be used to escape the identifier. For example, to define a message with the literal name error, you may write:
This syntax is allowed anywhere an identifier is expected.
Annotations for Ordering and Namespaces
Java-style annotations may be used to add additional properties to types and fields throughout Avro IDL.
For example, to specify the sort order of a field within a record, one may use the @order annotation before the field name as follows:
record MyRecord {
string @order("ascending") myAscendingSortField;
string @order("descending") myDescendingField;
string @order("ignore") myIgnoredField;
}
A field’s type (with the exception of type references) may also be preceded by annotations, e.g.:
record MyRecord {
@java-class("java.util.ArrayList") array<string> myStrings;
}
This can be used to support java classes that can be serialized/deserialized via their toString/String constructor, e.g.:
record MyRecord {
@java-class("java.math.BigDecimal") string value;
@java-key-class("java.io.File") map<string> fileStates;
array<@java-class("java.math.BigDecimal") string> weights;
}
Similarly, a @namespace annotation may be used to modify the namespace when defining a named schema. For example:
@namespace("org.apache.avro.firstNamespace")
protocol MyProto {
@namespace("org.apache.avro.someOtherNamespace")
record Foo {}
record Bar {}
}
will define a protocol in the firstNamespace namespace. The record Foo will be defined in someOtherNamespace and Bar will be defined in firstNamespace as it inherits its default from its container.
Type and field aliases are specified with the @aliases annotation as follows:
@aliases(["org.old.OldRecord", "org.ancient.AncientRecord"])
record MyRecord {
string @aliases(["oldField", "ancientField"]) myNewField;
}
Some annotations like those listed above are handled specially. All other annotations are added as properties to the protocol, message, schema or field.
Complete Example
The following is an example of an Avro IDL file that shows most of the above features:
/*
* Header with license information.
*/
/**
* An example protocol in Avro IDL
*/
@namespace("org.apache.avro.test")
protocol Simple {
/** Documentation for the enum type Kind */
@aliases(["org.foo.KindOf"])
enum Kind {
FOO,
BAR, // the bar enum value
BAZ
} = FOO; // For schema evolution purposes, unmatched values do not throw an error, but are resolved to FOO.
/** MD5 hash; good enough to avoid most collisions, and smaller than (for example) SHA256. */
fixed MD5(16);
record TestRecord {
/** Record name; has no intrinsic order */
string @order("ignore") name;
Kind @order("descending") kind;
MD5 hash;
/*
Note that 'null' is the first union type. Just like .avsc / .avpr files, the default value must be of the first union type.
*/
union { null, MD5 } /** Optional field */ @aliases(["hash"]) nullableHash = null;
array<long> arrayOfLongs;
}
/** Errors are records that can be thrown from a method */
error TestError {
string message;
}
string hello(string greeting);
/** Return what was given. Demonstrates the use of backticks to name types/fields/messages/parameters after keywords */
TestRecord echo(TestRecord `record`);
int add(int arg1, int arg2);
bytes echoBytes(bytes data);
void `error`() throws TestError;
// The oneway keyword forces the method to return null.
void ping() oneway;
}
Additional examples may be found in the Avro source tree under the src/test/idl/input directory.
IDE support
There are several editors and IDEs that support Avro IDL files, usually via plugins.
JetBrains
Apache Avro IDL Schema Support 203.1.2 was released in 9 December 2021.
Features:
- Syntax Highlighting
- Code Completion
- Code Formatting
- Error Highlighting
- Inspections & quick fixes
- JSON schemas for .avpr and .avsc files
It’s available via the JetBrains Marketplace
and on GitHub.
The plugin supports almost the all JetBrains products: IntelliJ IDEA, PyCharm, WebStorm, Android Studio, AppCode, GoLand, Rider, CLion, RubyMine, PhpStorm, DataGrip, DataSpell, MPS, Code With Me Guest and JetBrains Client.
Only JetBrains Gateway does not support this plugin directly. But the backend (JetBrains) IDE that it connects to does.
Eclipse
Avroclipse 0.0.11 was released on 4 December 2019.
Features:
- Syntax Highlighting
- Error Highlighting
- Code Completion
It is available on the Eclipse Marketplace
and GitHub.
Visual Studio Code
avro-idl 0.5.0 was released on 16 June 2021. It provides syntax highlighting.
It is available on the VisualStudio Marketplace
and GitHub
Atom.io
atom-language-avro 0.0.13 was released on 14 August 2015. It provides syntax highlighting.
It is available as Atom.io package
and GitHub
Vim
A .avdl detecting plugin by Gurpreet Atwal on GitHub (Last change in December 2016)
avro-idl.vim in the Avro repository share/editors directory (last change in September 2010)
Both provide syntax highlighting.
1.6 - SASL profile
Introduction
SASL (RFC 2222) provides a framework for authentication and security of network protocols. Each protocol that uses SASL is meant to define a SASL profile. This document provides a SASL profile for connection-based Avro RPC.
Overview
SASL negotiation proceeds as a series of message interactions over a connection between a client and server using a selected SASL mechanism. The client starts this negotiation by sending its chosen mechanism name with an initial (possibly empty) message. Negotiation proceeds with the exchange of messages until either side indicates success or failure. The content of the messages is mechanism-specific. If the negotiation succeeds, then the session can proceed over the connection, otherwise it must be abandoned.
Some mechanisms continue to process session data after negotiation (e.g., encrypting it), while some specify that further session data is transmitted unmodified.
Negotiation
Commands
Avro SASL negotiation uses four one-byte commands.
- 0: START Used in a client’s initial message.
- 1: CONTINUE Used while negotiation is ongoing.
- 2: FAIL Terminates negotiation unsuccessfully.
- 3: COMPLETE Terminates negotiation successfully.
The format of a START message is:
| 0 | 4-byte mechanism name length | mechanism name | 4-byte payload length | payload data |
The format of a CONTINUE message is:
| 1 | 4-byte payload length | payload data |
The format of a FAIL message is:
| 2 | 4-byte message length | UTF-8 message |
The format of a COMPLETE message is:
| 3 | 4-byte payload length | payload data |
Process
Negotiation is initiated by a client sending a START command containing the client’s chosen mechanism name and any mechanism-specific payload data.
The server and client then interchange some number (possibly zero) of CONTINUE messages. Each message contains payload data that is processed by the security mechanism to generate the next message.
Once either the client or server send a FAIL message then negotiation has failed. UTF-8-encoded text is included in the failure message. Once either a FAIL message has been sent or received, or any other error occurs in the negotiation, further communication on this connection must cease.
Once either the client or server send a COMPLETE message then negotiation has completed successfully. Session data may now be transmitted over the connection until it is closed by either side.
Session Data
If no SASL QOP (quality of protection) is negotiated, then all subsequent writes to/reads over this connection are written/read unmodified. In particular, messages use Avro framing, and are of the form:
| 4-byte frame length | frame data | ... | 4 zero bytes |
If a SASL QOP is negotiated, then it must be used by the connection for all subsequent messages. This is done by wrapping each non-empty frame written using the security mechanism and unwrapping each non-empty frame read. The length written in each non-empty frame is the length of the wrapped data. Complete frames must be passed to the security mechanism for unwrapping. Unwrapped data is then passed to the application as the content of the frame.
If at any point processing fails due to wrapping, unwrapping or framing errors, then all further communication on this connection must cease.
Anonymous Mechanism
The SASL anonymous mechanism (RFC 2245) is quite simple to implement. In particular, an initial anonymous request may be prefixed by the following static sequence:
| 0 | 0009 | ANONYMOUS | 0000 |
If a server uses the anonymous mechanism, it should check that the mechanism name in the start message prefixing the first request received is ‘ANONYMOUS’, then simply prefix its initial response with a COMPLETE message of:
| 3 | 0000 |
If an anonymous server recieves some other mechanism name, then it may respond with a FAIL message as simple as:
| 2 | 0000 |
Note that the anonymous mechanism need add no additional round-trip messages between client and server. The START message can be piggybacked on the initial request and the COMPLETE or FAIL message can be piggybacked on the initial response.
2 - Apache Avro™ 1.11.0 Documentation
Introduction
Apache Avro™ is a data serialization system.
Avro provides:
- Rich data structures.
- A compact, fast, binary data format.
- A container file, to store persistent data.
- Remote procedure call (RPC).
- Simple integration with dynamic languages. Code generation is not required to read or write data files nor to use or implement RPC protocols. Code generation as an optional optimization, only worth implementing for statically typed languages.
Schemas
Avro relies on schemas. When Avro data is read, the schema used when writing it is always present. This permits each datum to be written with no per-value overheads, making serialization both fast and small. This also facilitates use with dynamic, scripting languages, since data, together with its schema, is fully self-describing.
When Avro data is stored in a file, its schema is stored with it, so that files may be processed later by any program. If the program reading the data expects a different schema this can be easily resolved, since both schemas are present.
When Avro is used in RPC, the client and server exchange schemas in the connection handshake. (This can be optimized so that, for most calls, no schemas are actually transmitted.) Since both client and server both have the other’s full schema, correspondence between same named fields, missing fields, extra fields, etc. can all be easily resolved.
Avro schemas are defined with JSON . This facilitates implementation in languages that already have JSON libraries.
Comparison with other systems
Avro provides functionality similar to systems such as Thrift, Protocol Buffers, etc. Avro differs from these systems in the following fundamental aspects.
- Dynamic typing: Avro does not require that code be generated. Data is always accompanied by a schema that permits full processing of that data without code generation, static datatypes, etc. This facilitates construction of generic data-processing systems and languages.
- Untagged data: Since the schema is present when data is read, considerably less type information need be encoded with data, resulting in smaller serialization size.
- No manually-assigned field IDs: When a schema changes, both the old and new schema are always present when processing data, so differences may be resolved symbolically, using field names.
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
2.1 - Getting Started (Java)
This is a short guide for getting started with Apache Avro™ using Java. This guide only covers using Avro for data serialization; see Patrick Hunt’s Avro RPC Quick Start for a good introduction to using Avro for RPC.
Download
Avro implementations for C, C++, C#, Java, PHP, Python, and Ruby can be downloaded from the Apache Avro™ Releases page. This guide uses Avro 1.11.0, the latest version at the time of writing. For the examples in this guide, download avro-1.11.0.jar and avro-tools-1.11.0.jar.
Alternatively, if you are using Maven, add the following dependency to your POM:
<dependency>
<groupId>org.apache.avro</groupId>
<artifactId>avro</artifactId>
<version>1.11.0</version>
</dependency>
As well as the Avro Maven plugin (for performing code generation):
<plugin>
<groupId>org.apache.avro</groupId>
<artifactId>avro-maven-plugin</artifactId>
<version>1.11.0</version>
<executions>
<execution>
<phase>generate-sources</phase>
<goals>
<goal>schema</goal>
</goals>
<configuration>
<sourceDirectory>${project.basedir}/src/main/avro/</sourceDirectory>
<outputDirectory>${project.basedir}/src/main/java/</outputDirectory>
</configuration>
</execution>
</executions>
</plugin>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-compiler-plugin</artifactId>
<configuration>
<source>1.8</source>
<target>1.8</target>
</configuration>
</plugin>
You may also build the required Avro jars from source. Building Avro is beyond the scope of this guide; see the Build Documentation page in the wiki for more information.
Defining a schema
Avro schemas are defined using JSON. Schemas are composed of primitive types (null, boolean, int, long, float, double, bytes, and string) and complex types (record, enum, array, map, union, and fixed). You can learn more about Avro schemas and types from the specification, but for now let’s start with a simple schema example, user.avsc:
{"namespace": "example.avro",
"type": "record",
"name": "User",
"fields": [
{"name": "name", "type": "string"},
{"name": "favorite_number", "type": ["int", "null"]},
{"name": "favorite_color", "type": ["string", "null"]}
]
}
This schema defines a record representing a hypothetical user. (Note that a schema file can only contain a single schema definition.) At minimum, a record definition must include its type (“type”: “record”), a name (“name”: “User”), and fields, in this case name, favorite_number, and favorite_color. We also define a namespace (“namespace”: “example.avro”), which together with the name attribute defines the “full name” of the schema (example.avro.User in this case).
Fields are defined via an array of objects, each of which defines a name and type (other attributes are optional, see the record specification for more details). The type attribute of a field is another schema object, which can be either a primitive or complex type. For example, the name field of our User schema is the primitive type string, whereas the favorite_number and favorite_color fields are both unions, represented by JSON arrays. unions are a complex type that can be any of the types listed in the array; e.g., favorite_number can either be an int or null, essentially making it an optional field.
Serializing and deserializing with code generation
Compiling the schema
Code generation allows us to automatically create classes based on our previously-defined schema. Once we have defined the relevant classes, there is no need to use the schema directly in our programs. We use the avro-tools jar to generate code as follows:
java -jar /path/to/avro-tools-1.11.0.jar compile schema <schema file> <destination>
This will generate the appropriate source files in a package based on the schema’s namespace in the provided destination folder. For instance, to generate a User class in package example.avro from the schema defined above, run
java -jar /path/to/avro-tools-1.11.0.jar compile schema user.avsc .
Note that if you using the Avro Maven plugin, there is no need to manually invoke the schema compiler; the plugin automatically performs code generation on any .avsc files present in the configured source directory.
Creating Users
Now that we’ve completed the code generation, let’s create some Users, serialize them to a data file on disk, and then read back the file and deserialize the User objects.
First let’s create some Users and set their fields.
User user1 = new User();
user1.setName("Alyssa");
user1.setFavoriteNumber(256);
// Leave favorite color null
// Alternate constructor
User user2 = new User("Ben", 7, "red");
// Construct via builder
User user3 = User.newBuilder()
.setName("Charlie")
.setFavoriteColor("blue")
.setFavoriteNumber(null)
.build();
As shown in this example, Avro objects can be created either by invoking a constructor directly or by using a builder. Unlike constructors, builders will automatically set any default values specified in the schema. Additionally, builders validate the data as it set, whereas objects constructed directly will not cause an error until the object is serialized. However, using constructors directly generally offers better performance, as builders create a copy of the datastructure before it is written.
Note that we do not set user1’s favorite color. Since that record is of type [“string”, “null”], we can either set it to a string or leave it null; it is essentially optional. Similarly, we set user3’s favorite number to null (using a builder requires setting all fields, even if they are null).
Serializing
Now let’s serialize our Users to disk.
/// Serialize user1, user2 and user3 to disk
DatumWriter<User> userDatumWriter = new SpecificDatumWriter<User>(User.class);
DataFileWriter<User> dataFileWriter = new DataFileWriter<User>(userDatumWriter);
dataFileWriter.create(user1.getSchema(), new File("users.avro"));
dataFileWriter.append(user1);
dataFileWriter.append(user2);
dataFileWriter.append(user3);
dataFileWriter.close();
We create a DatumWriter, which converts Java objects into an in-memory serialized format. The SpecificDatumWriter class is used with generated classes and extracts the schema from the specified generated type.
Next we create a DataFileWriter, which writes the serialized records, as well as the schema, to the file specified in the dataFileWriter.create call. We write our users to the file via calls to the dataFileWriter.append method. When we are done writing, we close the data file.
Deserializing
Finally, let’s deserialize the data file we just created.
// Deserialize Users from disk
DatumReader<User> userDatumReader = new SpecificDatumReader<User>(User.class);
DataFileReader<User> dataFileReader = new DataFileReader<User>(file, userDatumReader);
User user = null;
while (dataFileReader.hasNext()) {
// Reuse user object by passing it to next(). This saves us from
// allocating and garbage collecting many objects for files with
// many items.
user = dataFileReader.next(user);
System.out.println(user);
}
This snippet will output:
{"name": "Alyssa", "favorite_number": 256, "favorite_color": null}
{"name": "Ben", "favorite_number": 7, "favorite_color": "red"}
{"name": "Charlie", "favorite_number": null, "favorite_color": "blue"}
Deserializing is very similar to serializing. We create a SpecificDatumReader, analogous to the SpecificDatumWriter we used in serialization, which converts in-memory serialized items into instances of our generated class, in this case User. We pass the DatumReader and the previously created File to a DataFileReader, analogous to the DataFileWriter, which reads both the schema used by the writer as well as the data from the file on disk. The data will be read using the writer’s schema included in the file and the schema provided by the reader, in this case the User class. The writer’s schema is needed to know the order in which fields were written, while the reader’s schema is needed to know what fields are expected and how to fill in default values for fields added since the file was written. If there are differences between the two schemas, they are resolved according to the Schema Resolution specification.
Next we use the DataFileReader to iterate through the serialized Users and print the deserialized object to stdout. Note how we perform the iteration: we create a single User object which we store the current deserialized user in, and pass this record object to every call of dataFileReader.next. This is a performance optimization that allows the DataFileReader to reuse the same User object rather than allocating a new User for every iteration, which can be very expensive in terms of object allocation and garbage collection if we deserialize a large data file. While this technique is the standard way to iterate through a data file, it’s also possible to use for (User user : dataFileReader) if performance is not a concern.
Compiling and running the example code
This example code is included as a Maven project in the examples/java-example directory in the Avro docs. From this directory, execute the following commands to build and run the example:
$ mvn compile # includes code generation via Avro Maven plugin
$ mvn -q exec:java -Dexec.mainClass=example.SpecificMain
Beta feature: Generating faster code
In this release we have introduced a new approach to generating code that speeds up decoding of objects by more than 10% and encoding by more than 30% (future performance enhancements are underway). To ensure a smooth introduction of this change into production systems, this feature is controlled by a feature flag, the system property org.apache.avro.specific.use_custom_coders. In this first release, this feature is off by default. To turn it on, set the system flag to true at runtime. In the sample above, for example, you could enable the fater coders as follows:
$ mvn -q exec:java -Dexec.mainClass=example.SpecificMain \
-Dorg.apache.avro.specific.use_custom_coders=true
Note that you do not have to recompile your Avro schema to have access to this feature. The feature is compiled and built into your code, and you turn it on and off at runtime using the feature flag. As a result, you can turn it on during testing, for example, and then off in production. Or you can turn it on in production, and quickly turn it off if something breaks.
We encourage the Avro community to exercise this new feature early to help build confidence. (For those paying one-demand for compute resources in the cloud, it can lead to meaningful cost savings.) As confidence builds, we will turn this feature on by default, and eventually eliminate the feature flag (and the old code).
Serializing and deserializing without code generation
Data in Avro is always stored with its corresponding schema, meaning we can always read a serialized item regardless of whether we know the schema ahead of time. This allows us to perform serialization and deserialization without code generation.
Let’s go over the same example as in the previous section, but without using code generation: we’ll create some users, serialize them to a data file on disk, and then read back the file and deserialize the users objects.
Creating users
First, we use a Parser to read our schema definition and create a Schema object.
Schema schema = new Schema.Parser().parse(new File("user.avsc"));
Using this schema, let’s create some users.
GenericRecord user1 = new GenericData.Record(schema);
user1.put("name", "Alyssa");
user1.put("favorite_number", 256);
// Leave favorite color null
GenericRecord user2 = new GenericData.Record(schema);
user2.put("name", "Ben");
user2.put("favorite_number", 7);
user2.put("favorite_color", "red");
Since we’re not using code generation, we use GenericRecords to represent users. GenericRecord uses the schema to verify that we only specify valid fields. If we try to set a non-existent field (e.g., user1.put(“favorite_animal”, “cat”)), we’ll get an AvroRuntimeException when we run the program.
Note that we do not set user1’s favorite color. Since that record is of type [“string”, “null”], we can either set it to a string or leave it null; it is essentially optional.
Serializing
Now that we’ve created our user objects, serializing and deserializing them is almost identical to the example above which uses code generation. The main difference is that we use generic instead of specific readers and writers.
First we’ll serialize our users to a data file on disk.
// Serialize user1 and user2 to disk
File file = new File("users.avro");
DatumWriter<GenericRecord> datumWriter = new GenericDatumWriter<GenericRecord>(schema);
DataFileWriter<GenericRecord> dataFileWriter = new DataFileWriter<GenericRecord>(datumWriter);
dataFileWriter.create(schema, file);
dataFileWriter.append(user1);
dataFileWriter.append(user2);
dataFileWriter.close();
We create a DatumWriter, which converts Java objects into an in-memory serialized format. Since we are not using code generation, we create a GenericDatumWriter. It requires the schema both to determine how to write the GenericRecords and to verify that all non-nullable fields are present.
As in the code generation example, we also create a DataFileWriter, which writes the serialized records, as well as the schema, to the file specified in the dataFileWriter.create call. We write our users to the file via calls to the dataFileWriter.append method. When we are done writing, we close the data file.
Deserializing
Finally, we’ll deserialize the data file we just created.
// Deserialize users from disk
DatumReader<GenericRecord> datumReader = new GenericDatumReader<GenericRecord>(schema);
DataFileReader<GenericRecord> dataFileReader = new DataFileReader<GenericRecord>(file, datumReader);
GenericRecord user = null;
while (dataFileReader.hasNext()) {
// Reuse user object by passing it to next(). This saves us from
// allocating and garbage collecting many objects for files with
// many items.
user = dataFileReader.next(user);
System.out.println(user);
This outputs:
{"name": "Alyssa", "favorite_number": 256, "favorite_color": null}
{"name": "Ben", "favorite_number": 7, "favorite_color": "red"}
Deserializing is very similar to serializing. We create a GenericDatumReader, analogous to the GenericDatumWriter we used in serialization, which converts in-memory serialized items into GenericRecords. We pass the DatumReader and the previously created File to a DataFileReader, analogous to the DataFileWriter, which reads both the schema used by the writer as well as the data from the file on disk. The data will be read using the writer’s schema included in the file, and the reader’s schema provided to the GenericDatumReader. The writer’s schema is needed to know the order in which fields were written, while the reader’s schema is needed to know what fields are expected and how to fill in default values for fields added since the file was written. If there are differences between the two schemas, they are resolved according to the Schema Resolution specification.
Next, we use the DataFileReader to iterate through the serialized users and print the deserialized object to stdout. Note how we perform the iteration: we create a single GenericRecord object which we store the current deserialized user in, and pass this record object to every call of dataFileReader.next. This is a performance optimization that allows the DataFileReader to reuse the same record object rather than allocating a new GenericRecord for every iteration, which can be very expensive in terms of object allocation and garbage collection if we deserialize a large data file. While this technique is the standard way to iterate through a data file, it’s also possible to use for (GenericRecord user : dataFileReader) if performance is not a concern.
Compiling and running the example code
This example code is included as a Maven project in the examples/java-example directory in the Avro docs. From this directory, execute the following commands to build and run the example:
$ mvn compile
$ mvn -q exec:java -Dexec.mainClass=example.GenericMain
2.2 - Getting Started (Python)
This is a short guide for getting started with Apache Avro™ using Python. This guide only covers using Avro for data serialization; see Patrick Hunt’s Avro RPC Quick Start for a good introduction to using Avro for RPC.
Notice for Python 3 users
A package called “avro-python3” had been provided to support Python 3 previously, but the codebase was consolidated into the “avro” package that supports Python 3 now. The avro-python3 package will be removed in the near future, so users should use the “avro” package instead. They are mostly API compatible, but there’s a few minor difference (e.g., function name capitalization, such as avro.schema.Parse vs avro.schema.parse).
Download and Install
The easiest way to get started in Python is to install avro from PyPI using pip, the Python Package Installer.
$ python3 -m pip install avro
Consider doing a local install or using a virtualenv to avoid permissions problems and interfering with system packages:
$ python3 -m pip install --user install avro
or
$ python3 -m venv avro-venv
$ avro-venv/bin/pip install avro
The official releases of the Avro implementations for C, C++, C#, Java, PHP, Python, and Ruby can be downloaded from the Apache Avro™ Releases page. This guide uses Avro 1.11.0, the latest version at the time of writing. Download and install avro-1.11.0-py2.py3-none-any.whl or avro-1.11.0.tar.gz via python -m pip avro-1.11.0-py2.py3-none-any.whl or python -m pip avro-1.11.0.tar.gz. (As above, consider using a virtualenv or user-local install.)
Check that you can import avro from a Python prompt.
$ python3 -c 'import avro; print(avro.__version__)'
The above should print 1.11.0. It should not raise an ImportError.
Alternatively, you may build the Avro Python library from source. From your the root Avro directory, run the commands
$ cd lang/py/
$ python3 -m pip install -e .
$ python3
Defining a schema
Avro schemas are defined using JSON. Schemas are composed of primitive types (null, boolean, int, long, float, double, bytes, and string) and complex types (record, enum, array, map, union, and fixed). You can learn more about Avro schemas and types from the specification, but for now let’s start with a simple schema example, user.avsc:
{"namespace": "example.avro",
"type": "record",
"name": "User",
"fields": [
{"name": "name", "type": "string"},
{"name": "favorite_number", "type": ["int", "null"]},
{"name": "favorite_color", "type": ["string", "null"]}
]
}
This schema defines a record representing a hypothetical user. (Note that a schema file can only contain a single schema definition.) At minimum, a record definition must include its type (“type”: “record”), a name (“name”: “User”), and fields, in this case name, favorite_number, and favorite_color. We also define a namespace (“namespace”: “example.avro”), which together with the name attribute defines the “full name” of the schema (example.avro.User in this case).
Fields are defined via an array of objects, each of which defines a name and type (other attributes are optional, see the record specification for more details). The type attribute of a field is another schema object, which can be either a primitive or complex type. For example, the name field of our User schema is the primitive type string, whereas the favorite_number and favorite_color fields are both unions, represented by JSON arrays. unions are a complex type that can be any of the types listed in the array; e.g., favorite_number can either be an int or null, essentially making it an optional field.
Serializing and deserializing without code generation
Data in Avro is always stored with its corresponding schema, meaning we can always read a serialized item, regardless of whether we know the schema ahead of time. This allows us to perform serialization and deserialization without code generation. Note that the Avro Python library does not support code generation.
Try running the following code snippet, which serializes two users to a data file on disk, and then reads back and deserializes the data file:
import avro.schema
from avro.datafile import DataFileReader, DataFileWriter
from avro.io import DatumReader, DatumWriter
schema = avro.schema.parse(open("user.avsc", "rb").read())
writer = DataFileWriter(open("users.avro", "wb"), DatumWriter(), schema)
writer.append({"name": "Alyssa", "favorite_number": 256})
writer.append({"name": "Ben", "favorite_number": 7, "favorite_color": "red"})
writer.close()
reader = DataFileReader(open("users.avro", "rb"), DatumReader())
for user in reader:
print user
reader.close()
This outputs:
{u'favorite_color': None, u'favorite_number': 256, u'name': u'Alyssa'}
{u'favorite_color': u'red', u'favorite_number': 7, u'name': u'Ben'}
Do make sure that you open your files in binary mode (i.e. using the modes wb or rb respectively). Otherwise you might generate corrupt files due to automatic replacement of newline characters with the platform-specific representations.
Let’s take a closer look at what’s going on here.
schema = avro.schema.parse(open("user.avsc", "rb").read())
avro.schema.parse takes a string containing a JSON schema definition as input and outputs a avro.schema.Schema object (specifically a subclass of Schema, in this case RecordSchema). We’re passing in the contents of our user.avsc schema file here.
writer = DataFileWriter(open("users.avro", "wb"), DatumWriter(), schema)
We create a DataFileWriter, which we’ll use to write serialized items to a data file on disk. The DataFileWriter constructor takes three arguments:
- The file we’ll serialize to
- A DatumWriter, which is responsible for actually serializing the items to Avro’s binary format (DatumWriters can be used separately from DataFileWriters, e.g., to perform IPC with Avro).
- The schema we’re using. The DataFileWriter needs the schema both to write the schema to the data file, and to verify that the items we write are valid items and write the appropriate fields.
writer.append({"name": "Alyssa", "favorite_number": 256})
writer.append({"name": "Ben", "favorite_number": 7, "favorite_color": "red"})
We use DataFileWriter.append to add items to our data file. Avro records are represented as Python dicts. Since the field favorite_color has type [“int”, “null”], we are not required to specify this field, as shown in the first append. Were we to omit the required name field, an exception would be raised. Any extra entries not corresponding to a field are present in the dict are ignored.
reader = DataFileReader(open("users.avro", "rb"), DatumReader())
We open the file again, this time for reading back from disk. We use a DataFileReader and DatumReader analagous to the DataFileWriter and DatumWriter above.
for user in reader:
print user
The DataFileReader is an iterator that returns dicts corresponding to the serialized items.
2.3 - Specification
Introduction
This document defines Apache Avro. It is intended to be the authoritative specification. Implementations of Avro must adhere to this document.
Schema Declaration
A Schema is represented in JSON by one of:
- A JSON string, naming a defined type.
- A JSON object, of the form:
{"type": "typeName" ...attributes...}
where typeName is either a primitive or derived type name, as defined below. Attributes not defined in this document are permitted as metadata, but must not affect the format of serialized data.
- A JSON array, representing a union of embedded types.
Primitive Types
The set of primitive type names is:
- null: no value
- boolean: a binary value
- int: 32-bit signed integer
- long: 64-bit signed integer
- float: single precision (32-bit) IEEE 754 floating-point number
- double: double precision (64-bit) IEEE 754 floating-point number
- bytes: sequence of 8-bit unsigned bytes
- string: unicode character sequence
Primitive types have no specified attributes.
Primitive type names are also defined type names. Thus, for example, the schema “string” is equivalent to:
Complex Types
Avro supports six kinds of complex types: records, enums, arrays, maps, unions and fixed.
Records
Records use the type name “record” and support the following attributes:
- name: a JSON string providing the name of the record (required).
- namespace, a JSON string that qualifies the name;
- doc: a JSON string providing documentation to the user of this schema (optional).
- aliases: a JSON array of strings, providing alternate names for this record (optional).
- fields: a JSON array, listing fields (required). Each field is a JSON object with the following attributes:
- name: a JSON string providing the name of the field (required), and
- doc: a JSON string describing this field for users (optional).
- type: a schema, as defined above
- default: A default value for this field, only used when reading instances that lack the field for schema evolution purposes. The presence of a default value does not make the field optional at encoding time. 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. Default values for bytes and fixed fields are JSON strings, where Unicode code points 0-255 are mapped to unsigned 8-bit byte values 0-255. Avro encodes a field even if its value is equal to its default.
field default values
| avro type |
json type |
example |
| null |
null |
null |
| boolean |
boolean |
true |
| int,long |
integer |
1 |
| float,double |
number |
1.1 |
| bytes |
string |
"\u00FF" |
| string |
string |
"foo" |
| record |
object |
{"a": 1} |
| enum |
string |
"FOO" |
| array |
array |
[1] |
| map |
object |
{"a": 1} |
| fixed |
string |
"\u00ff" |
- order: specifies how this field impacts sort ordering of this record (optional). Valid values are “ascending” (the default), “descending”, or “ignore”. For more details on how this is used, see the sort order section below.
- aliases: a JSON array of strings, providing alternate names for this field (optional).
For example, a linked-list of 64-bit values may be defined with:
{
"type": "record",
"name": "LongList",
"aliases": ["LinkedLongs"], // old name for this
"fields" : [
{"name": "value", "type": "long"}, // each element has a long
{"name": "next", "type": ["null", "LongList"]} // optional next element
]
}
Enums
Enums use the type name “enum” and support the following attributes:
- name: a JSON string providing the name of the enum (required).
- namespace, a JSON string that qualifies the name ;
- aliases: a JSON array of strings, providing alternate names for this enum (optional).
- doc: a JSON string providing documentation to the user of this schema (optional).
- symbols: a JSON array, listing symbols, as JSON strings (required). All symbols in an enum must be unique; duplicates are prohibited. Every symbol must match the regular expression [A-Za-z_][A-Za-z0-9_]* (the same requirement as for names).
- default: A default value for this enumeration, used during resolution when the reader encounters a symbol from the writer that isn’t defined in the reader’s schema (optional). The value provided here must be a JSON string that’s a member of the symbols array. See documentation on schema resolution for how this gets used.
For example, playing card suits might be defined with:
{
"type": "enum",
"name": "Suit",
"symbols" : ["SPADES", "HEARTS", "DIAMONDS", "CLUBS"]
}
Arrays
Arrays use the type name “array” and support a single attribute:
- items: the schema of the array’s items.
For example, an array of strings is declared with:
{
"type": "array",
"items" : "string",
"default": []
}
Maps
Maps use the type name “map” and support one attribute:
- values: the schema of the map’s values.
Map keys are assumed to be strings.
For example, a map from string to long is declared with:
{
"type": "map",
"values" : "long",
"default": {}
}
Unions
Unions, as mentioned above, are represented using JSON arrays. For example, ["null", "string"] declares a schema which may be either a null or string.
(Note that when a default value is specified for a record field whose type is a union, the type of the default value must match the first element of the union. Thus, for unions containing “null”, the “null” is usually listed first, since the default value of such unions is typically null.)
Unions may not contain more than one schema with the same type, except for the named types record, fixed and enum. For example, unions containing two array types or two map types are not permitted, but two types with different names are permitted. (Names permit efficient resolution when reading and writing unions.)
Unions may not immediately contain other unions.
Fixed
Fixed uses the type name “fixed” and supports the following attributes:
- name: a string naming this fixed (required).
- namespace, a string that qualifies the name ;
- aliases: a JSON array of strings, providing alternate names for this enum (optional).
- doc: a JSON string providing documentation to the user of this schema (optional).
- size: an integer, specifying the number of bytes per value (required).
For example, 16-byte quantity may be declared with:
{"type": "fixed", "size": 16, "name": "md5"}
Names
Record, enums and fixed are named types. Each has a fullname that is composed of two parts; a name and a namespace. Equality of names is defined on the fullname.
The name portion of a fullname, record field names, and enum symbols must:
- start with [A-Za-z_]
- subsequently contain only [A-Za-z0-9_]
A namespace is a dot-separated sequence of such names. The empty string may also be used as a namespace to indicate the null namespace. Equality of names (including field names and enum symbols) as well as fullnames is case-sensitive.
The null namespace may not be used in a dot-separated sequence of names. So the grammar for a namespace is:
<empty> | <name>[(<dot><name>)*]
In record, enum and fixed definitions, the fullname is determined in one of the following ways:
A name and namespace are both specified. For example, one might use “name”: “X”, “namespace”: “org.foo” to indicate the fullname org.foo.X.
- A fullname is specified. If the name specified contains a dot, then it is assumed to be a fullname, and any namespace also specified is ignored. For example, use “name”: “org.foo.X” to indicate the fullname org.foo.X.
- A name only is specified, i.e., a name that contains no dots. In this case the namespace is taken from the most tightly enclosing schema or protocol. For example, if “name”: “X” is specified, and this occurs within a field of the record definition of org.foo.Y, then the fullname is org.foo.X. If there is no enclosing namespace then the null namespace is used.
References to previously defined names are as in the latter two cases above: if they contain a dot they are a fullname, if they do not contain a dot, the namespace is the namespace of the enclosing definition.
Primitive type names have no namespace and their names may not be defined in any namespace.
A schema or protocol may not contain multiple definitions of a fullname. Further, a name must be defined before it is used (“before” in the depth-first, left-to-right traversal of the JSON parse tree, where the types attribute of a protocol is always deemed to come “before” the messages attribute.)
Aliases
Named types and fields may have aliases. An implementation may optionally use aliases to map a writer’s schema to the reader’s. This facilitates both schema evolution as well as processing disparate datasets.
Aliases function by re-writing the writer’s schema using aliases from the reader’s schema. For example, if the writer’s schema was named “Foo” and the reader’s schema is named “Bar” and has an alias of “Foo”, then the implementation would act as though “Foo” were named “Bar” when reading. Similarly, if data was written as a record with a field named “x” and is read as a record with a field named “y” with alias “x”, then the implementation would act as though “x” were named “y” when reading.
A type alias may be specified either as a fully namespace-qualified, or relative to the namespace of the name it is an alias for. For example, if a type named “a.b” has aliases of “c” and “x.y”, then the fully qualified names of its aliases are “a.c” and “x.y”.
Data Serialization and Deserialization
Binary encoded Avro data does not include type information or field names. The benefit is that the serialized data is small, but as a result a schema must always be used in order to read Avro data correctly. The best way to ensure that the schema is structurally identical to the one used to write the data is to use the exact same schema.
Therefore, files or systems that store Avro data should always include the writer’s schema for that data. Avro-based remote procedure call (RPC) systems must also guarantee that remote recipients of data have a copy of the schema used to write that data. In general, it is advisable that any reader of Avro data should use a schema that is the same (as defined more fully in Parsing Canonical Form for Schemas) as the schema that was used to write the data in order to deserialize it correctly. Deserializing data into a newer schema is accomplished by specifying an additional schema, the results of which are described in Schema Resolution.
In general, both serialization and deserialization proceed as a depth-first, left-to-right traversal of the schema, serializing or deserializing primitive types as they are encountered. Therefore, it is possible, though not advisable, to read Avro data with a schema that does not have the same Parsing Canonical Form as the schema with which the data was written. In order for this to work, the serialized primitive values must be compatible, in order value by value, with the items in the deserialization schema. For example, int and long are always serialized the same way, so an int could be deserialized as a long. Since the compatibility of two schemas depends on both the data and the serialization format (eg. binary is more permissive than JSON because JSON includes field names, eg. a long that is too large will overflow an int), it is simpler and more reliable to use schemas with identical Parsing Canonical Form.
Encodings
Avro specifies two serialization encodings: binary and JSON. Most applications will use the binary encoding, as it is smaller and faster. But, for debugging and web-based applications, the JSON encoding may sometimes be appropriate.
Binary Encoding
Binary encoding does not include field names, self-contained information about the types of individual bytes, nor field or record separators. Therefore readers are wholly reliant on the schema used when the data was encoded.
Primitive Types
Primitive types are encoded in binary as follows:
- null is written as zero bytes.
- a boolean is written as a single byte whose value is either 0 (false) or 1 (true).
- int and long values are written using variable-length zig-zag coding. Some examples:
| value |
hex |
| 0 |
00 |
| -1 |
01 |
| 1 |
02 |
| -2 |
03 |
| 2 |
04 |
| … |
|
| -64 |
7f |
| 64 |
80 01 |
| … |
|
- a float is written as 4 bytes. The float is converted into a 32-bit integer using a method equivalent to Java’s floatToIntBits and then encoded in little-endian format.
- a double is written as 8 bytes. The double is converted into a 64-bit integer using a method equivalent to Java’s doubleToLongBits and then encoded in little-endian format.
- bytes are encoded as a long followed by that many bytes of data.
- a string is encoded as a long followed by that many bytes of UTF-8 encoded character data.
For example, the three-character string “foo” would be encoded as the long value 3 (encoded as hex 06) followed by the UTF-8 encoding of ‘f’, ‘o’, and ‘o’ (the hex bytes 66 6f 6f):
06 66 6f 6f
Complex Types
Complex types are encoded in binary as follows:
Records
A record is encoded by encoding the values of its fields in the order that they are declared. In other words, a record is encoded as just the concatenation of the encodings of its fields. Field values are encoded per their schema.
For example, the record schema
{
"type": "record",
"name": "test",
"fields" : [
{"name": "a", "type": "long"},
{"name": "b", "type": "string"}
]
}
An instance of this record whose a field has value 27 (encoded as hex 36) and whose b field has value “foo” (encoded as hex bytes 06 66 6f 6f), would be encoded simply as the concatenation of these, namely the hex byte sequence:
36 06 66 6f 6f
Enums
An enum is encoded by a int, representing the zero-based position of the symbol in the schema.
For example, consider the enum:
{"type": "enum", "name": "Foo", "symbols": ["A", "B", "C", "D"] }
This would be encoded by an int between zero and three, with zero indicating “A”, and 3 indicating “D”.
Arrays
Arrays are encoded as a series of blocks. Each block consists of a long count value, followed by that many array items. A block with count zero indicates the end of the array. Each item is encoded per the array’s item schema.
If a block’s count is negative, its absolute value is used, and the count is followed immediately by a long block size indicating the number of bytes in the block. This block size permits fast skipping through data, e.g., when projecting a record to a subset of its fields.
For example, the array schema
{"type": "array", "items": "long"}
an array containing the items 3 and 27 could be encoded as the long value 2 (encoded as hex 04) followed by long values 3 and 27 (encoded as hex 06 36) terminated by zero:
04 06 36 00
The blocked representation permits one to read and write arrays larger than can be buffered in memory, since one can start writing items without knowing the full length of the array.
Maps
Maps are encoded as a series of blocks. Each block consists of a long count value, followed by that many key/value pairs. A block with count zero indicates the end of the map. Each item is encoded per the map’s value schema.
If a block’s count is negative, its absolute value is used, and the count is followed immediately by a long block size indicating the number of bytes in the block. This block size permits fast skipping through data, e.g., when projecting a record to a subset of its fields.
The blocked representation permits one to read and write maps larger than can be buffered in memory, since one can start writing items without knowing the full length of the map.
Unions
A union is encoded by first writing an int value indicating the zero-based position within the union of the schema of its value. The value is then encoded per the indicated schema within the union.
For example, the union schema ["null","string"] would encode:
-
null as zero (the index of “null” in the union):
00
-
the string “a” as one (the index of “string” in the union, 1, encoded as hex 02), followed by the serialized string:
02 02 61
NOTE: Currently for C/C++ implementations, the positions are practically an int, but theoretically a long. In reality, we don’t expect unions with 215M members
Fixed
Fixed instances are encoded using the number of bytes declared in the schema.
JSON Encoding
Except for unions, the JSON encoding is the same as is used to encode field default values.
The value of a union is encoded in JSON as follows:
- if its type is null, then it is encoded as a JSON null;
- otherwise it is encoded as a JSON object with one name/value pair whose name is the type’s name and whose value is the recursively encoded value. For Avro’s named types (record, fixed or enum) the user-specified name is used, for other types the type name is used.
For example, the union schema ["null","string","Foo"], where Foo is a record name, would encode:
- null as null;
- the string “a” as
{"string": "a"} and
- a Foo instance as
{"Foo": {...}}, where {...} indicates the JSON encoding of a Foo instance.
Note that the original schema is still required to correctly process JSON-encoded data. For example, the JSON encoding does not distinguish between int and long, float and double, records and maps, enums and strings, etc.
Single-object encoding
In some situations a single Avro serialized object is to be stored for a longer period of time. One very common example is storing Avro records for several weeks in an Apache Kafka topic.
In the period after a schema change this persistence system will contain records that have been written with different schemas. So the need arises to know which schema was used to write a record to support schema evolution correctly. In most cases the schema itself is too large to include in the message, so this binary wrapper format supports the use case more effectively.
Single object encoding specification
Single Avro objects are encoded as follows:
- A two-byte marker,
C3 01, to show that the message is Avro and uses this single-record format (version 1).
- The 8-byte little-endian CRC-64-AVRO fingerprint of the object’s schema.
- The Avro object encoded using Avro’s binary encoding.
Implementations use the 2-byte marker to determine whether a payload is Avro. This check helps avoid expensive lookups that resolve the schema from a fingerprint, when the message is not an encoded Avro payload.
Sort Order
Avro defines a standard sort order for data. This permits data written by one system to be efficiently sorted by another system. This can be an important optimization, as sort order comparisons are sometimes the most frequent per-object operation. Note also that Avro binary-encoded data can be efficiently ordered without deserializing it to objects.
Data items may only be compared if they have identical schemas. Pairwise comparisons are implemented recursively with a depth-first, left-to-right traversal of the schema. The first mismatch encountered determines the order of the items.
Two items with the same schema are compared according to the following rules.
- null data is always equal.
- boolean data is ordered with false before true.
- int, long, float and double data is ordered by ascending numeric value.
- bytes and fixed data are compared lexicographically by unsigned 8-bit values.
- string data is compared lexicographically by Unicode code point. Note that since UTF-8 is used as the binary encoding for strings, sorting of bytes and string binary data is identical.
- array data is compared lexicographically by element.
- enum data is ordered by the symbol’s position in the enum schema. For example, an enum whose symbols are
["z", "a"] would sort “z” values before “a” values.
- union data is first ordered by the branch within the union, and, within that, by the type of the branch. For example, an
["int", "string"] union would order all int values before all string values, with the ints and strings themselves ordered as defined above.
- record data is ordered lexicographically by field. If a field specifies that its order is:
- “ascending”, then the order of its values is unaltered.
- “descending”, then the order of its values is reversed.
- “ignore”, then its values are ignored when sorting.
- map data may not be compared. It is an error to attempt to compare data containing maps unless those maps are in an
"order":"ignore" record field.
Object Container Files
Avro includes a simple object container file format. A file has a schema, and all objects stored in the file must be written according to that schema, using binary encoding. Objects are stored in blocks that may be compressed. Syncronization markers are used between blocks to permit efficient splitting of files for MapReduce processing.
Files may include arbitrary user-specified metadata.
A file consists of:
- A file header, followed by
- one or more file data blocks.
A file header consists of:
- Four bytes, ASCII ‘O’, ‘b’, ‘j’, followed by 1.
- file metadata, including the schema.
- The 16-byte, randomly-generated sync marker for this file.
File metadata is written as if defined by the following map schema:
{"type": "map", "values": "bytes"}
All metadata properties that start with “avro.” are reserved. The following file metadata properties are currently used:
- avro.schema contains the schema of objects stored in the file, as JSON data (required).
- avro.codec the name of the compression codec used to compress blocks, as a string. Implementations are required to support the following codecs: “null” and “deflate”. If codec is absent, it is assumed to be “null”. The codecs are described with more detail below.
A file header is thus described by the following schema:
{"type": "record", "name": "org.apache.avro.file.Header",
"fields" : [
{"name": "magic", "type": {"type": "fixed", "name": "Magic", "size": 4}},
{"name": "meta", "type": {"type": "map", "values": "bytes"}},
{"name": "sync", "type": {"type": "fixed", "name": "Sync", "size": 16}}
]
}
A file data block consists of:
- A long indicating the count of objects in this block.
- A long indicating the size in bytes of the serialized objects in the current block, after any codec is applied
- The serialized objects. If a codec is specified, this is compressed by that codec.
- The file’s 16-byte sync marker.
Thus, each block’s binary data can be efficiently extracted or skipped without deserializing the contents. The combination of block size, object counts, and sync markers enable detection of corrupt blocks and help ensure data integrity.
Required Codecs
null
The “null” codec simply passes through data uncompressed.
deflate
The “deflate” codec writes the data block using the deflate algorithm as specified in RFC 1951, and typically implemented using the zlib library. Note that this format (unlike the “zlib format” in RFC 1950) does not have a checksum.
Optional Codecs
bzip2
The “bzip2” codec uses the bzip2 compression library.
snappy
The “snappy” codec uses Google’s Snappy compression library. Each compressed block is followed by the 4-byte, big-endian CRC32 checksum of the uncompressed data in the block.
xz
The “xz” codec uses the XZ compression library.
zstandard
The “zstandard” codec uses Facebook’s Zstandard compression library.
Protocol Declaration
Avro protocols describe RPC interfaces. Like schemas, they are defined with JSON text.
A protocol is a JSON object with the following attributes:
- protocol, a string, the name of the protocol (required);
- namespace, an optional string that qualifies the name;
- doc, an optional string describing this protocol;
- types, an optional list of definitions of named types (records, enums, fixed and errors). An error definition is just like a record definition except it uses “error” instead of “record”. Note that forward references to named types are not permitted.
- messages, an optional JSON object whose keys are message names and whose values are objects whose attributes are described below. No two messages may have the same name.
The name and namespace qualification rules defined for schema objects apply to protocols as well.
Messages
A message has attributes:
- a doc, an optional description of the message,
- a request, a list of named, typed parameter schemas (this has the same form as the fields of a record declaration);
- a response schema;
- an optional union of declared error schemas. The effective union has “string” prepended to the declared union, to permit transmission of undeclared “system” errors. For example, if the declared error union is
["AccessError"], then the effective union is ["string", "AccessError"]. When no errors are declared, the effective error union is ["string"]. Errors are serialized using the effective union; however, a protocol’s JSON declaration contains only the declared union.
- an optional one-way boolean parameter.
A request parameter list is processed equivalently to an anonymous record. Since record field lists may vary between reader and writer, request parameters may also differ between the caller and responder, and such differences are resolved in the same manner as record field differences.
The one-way parameter may only be true when the response type is "null" and no errors are listed.
Sample Protocol
For example, one may define a simple HelloWorld protocol with:
{
"namespace": "com.acme",
"protocol": "HelloWorld",
"doc": "Protocol Greetings",
"types": [
{"name": "Greeting", "type": "record", "fields": [
{"name": "message", "type": "string"}]},
{"name": "Curse", "type": "error", "fields": [
{"name": "message", "type": "string"}]}
],
"messages": {
"hello": {
"doc": "Say hello.",
"request": [{"name": "greeting", "type": "Greeting" }],
"response": "Greeting",
"errors": ["Curse"]
}
}
}
Message Transport
Messages may be transmitted via different transport mechanisms.
To the transport, a message is an opaque byte sequence.
A transport is a system that supports:
- transmission of request messages
- receipt of corresponding response messages
Servers may send a response message back to the client corresponding to a request message. The mechanism of correspondence is transport-specific. For example, in HTTP it is implicit, since HTTP directly supports requests and responses. But a transport that multiplexes many client threads over a single socket would need to tag messages with unique identifiers.
Transports may be either stateless or stateful. In a stateless transport, messaging assumes no established connection state, while stateful transports establish connections that may be used for multiple messages. This distinction is discussed further in the handshake section below.
HTTP as Transport
When HTTP is used as a transport, each Avro message exchange is an HTTP request/response pair. All messages of an Avro protocol should share a single URL at an HTTP server. Other protocols may also use that URL. Both normal and error Avro response messages should use the 200 (OK) response code. The chunked encoding may be used for requests and responses, but, regardless the Avro request and response are the entire content of an HTTP request and response. The HTTP Content-Type of requests and responses should be specified as “avro/binary”. Requests should be made using the POST method.
HTTP is used by Avro as a stateless transport.
Message Framing
Avro messages are framed as a list of buffers.
Framing is a layer between messages and the transport. It exists to optimize certain operations.
The format of framed message data is:
- a series of buffers, where each buffer consists of:
- a four-byte, big-endian buffer length, followed by
- that many bytes of buffer data.
- a message is always terminated by a zero-length buffer.
Framing is transparent to request and response message formats (described below). Any message may be presented as a single or multiple buffers.
Framing can permit readers to more efficiently get different buffers from different sources and for writers to more efficiently store different buffers to different destinations. In particular, it can reduce the number of times large binary objects are copied. For example, if an RPC parameter consists of a megabyte of file data, that data can be copied directly to a socket from a file descriptor, and, on the other end, it could be written directly to a file descriptor, never entering user space.
A simple, recommended, framing policy is for writers to create a new segment whenever a single binary object is written that is larger than a normal output buffer. Small objects are then appended in buffers, while larger objects are written as their own buffers. When a reader then tries to read a large object the runtime can hand it an entire buffer directly, without having to copy it.
Handshake
The purpose of the handshake is to ensure that the client and the server have each other’s protocol definition, so that the client can correctly deserialize responses, and the server can correctly deserialize requests. Both clients and servers should maintain a cache of recently seen protocols, so that, in most cases, a handshake will be completed without extra round-trip network exchanges or the transmission of full protocol text.
RPC requests and responses may not be processed until a handshake has been completed. With a stateless transport, all requests and responses are prefixed by handshakes. With a stateful transport, handshakes are only attached to requests and responses until a successful handshake response has been returned over a connection. After this, request and response payloads are sent without handshakes for the lifetime of that connection.
The handshake process uses the following record schemas:
{
"type": "record",
"name": "HandshakeRequest", "namespace":"org.apache.avro.ipc",
"fields": [
{"name": "clientHash",
"type": {"type": "fixed", "name": "MD5", "size": 16}},
{"name": "clientProtocol", "type": ["null", "string"]},
{"name": "serverHash", "type": "MD5"},
{"name": "meta", "type": ["null", {"type": "map", "values": "bytes"}]}
]
}
{
"type": "record",
"name": "HandshakeResponse", "namespace": "org.apache.avro.ipc",
"fields": [
{"name": "match",
"type": {"type": "enum", "name": "HandshakeMatch",
"symbols": ["BOTH", "CLIENT", "NONE"]}},
{"name": "serverProtocol",
"type": ["null", "string"]},
{"name": "serverHash",
"type": ["null", {"type": "fixed", "name": "MD5", "size": 16}]},
{"name": "meta",
"type": ["null", {"type": "map", "values": "bytes"}]}
]
}
- A client first prefixes each request with a
HandshakeRequest containing just the hash of its protocol and of the server’s protocol (clientHash!=null, clientProtocol=null, serverHash!=null), where the hashes are 128-bit MD5 hashes of the JSON protocol text. If a client has never connected to a given server, it sends its hash as a guess of the server’s hash, otherwise it sends the hash that it previously obtained from this server.
The server responds with a HandshakeResponse containing one of:
-
match=BOTH, serverProtocol=null, serverHash=null if the client sent the valid hash of the server’s protocol and the server knows what protocol corresponds to the client’s hash. In this case, the request is complete and the response data immediately follows the HandshakeResponse.
-
match=CLIENT, serverProtocol!=null, serverHash!=null if the server has previously seen the client’s protocol, but the client sent an incorrect hash of the server’s protocol. The request is complete and the response data immediately follows the HandshakeResponse. The client must use the returned protocol to process the response and should also cache that protocol and its hash for future interactions with this server.
-
match=NONE if the server has not previously seen the client’s protocol. The serverHash and serverProtocol may also be non-null if the server’s protocol hash was incorrect.
In this case the client must then re-submit its request with its protocol text (clientHash!=null, clientProtocol!=null, serverHash!=null) and the server should respond with a successful match (match=BOTH, serverProtocol=null, serverHash=null) as above.
The meta field is reserved for future handshake enhancements.
A call consists of a request message paired with its resulting response or error message. Requests and responses contain extensible metadata, and both kinds of messages are framed as described above.
The format of a call request is:
- request metadata, a map with values of type bytes
- the message name, an Avro string, followed by
- the message parameters. Parameters are serialized according to the message’s request declaration.
When the empty string is used as a message name a server should ignore the parameters and return an empty response. A client may use this to ping a server or to perform a handshake without sending a protocol message.
When a message is declared one-way and a stateful connection has been established by a successful handshake response, no response data is sent. Otherwise the format of the call response is:
- response metadata, a map with values of type bytes
- a one-byte error flag boolean, followed by either:
- if the error flag is false, the message response, serialized per the message’s response schema.
- if the error flag is true, the error, serialized per the message’s effective error union schema.
Schema Resolution
A reader of Avro data, whether from an RPC or a file, can always parse that data because the original schema must be provided along with the data. However, the reader may be programmed to read data into a different schema. For example, if the data was written with a different version of the software than it is read, then fields may have been added or removed from records. This section specifies how such schema differences should be resolved.
We refer to the schema used to write the data as the writer’s schema, and the schema that the application expects the reader’s schema. Differences between these should be resolved as follows:
- It is an error if the two schemas do not match.
To match, one of the following must hold:
-
both schemas are arrays whose item types match
-
both schemas are maps whose value types match
-
both schemas are enums whose (unqualified) names match
-
both schemas are fixed whose sizes and (unqualified) names match
-
both schemas are records with the same (unqualified) name
-
either schema is a union
-
both schemas have same primitive type
-
the writer’s schema may be promoted to the reader’s as follows:
- int is promotable to long, float, or double
- long is promotable to float or double
- float is promotable to double
- string is promotable to bytes
- bytes is promotable to string
-
if both are records:
- the ordering of fields may be different: fields are matched by name.
- schemas for fields with the same name in both records are resolved recursively.
- if the writer’s record contains a field with a name not present in the reader’s record, the writer’s value for that field is ignored.
- if the reader’s record schema has a field that contains a default value, and writer’s schema does not have a field with the same name, then the reader should use the default value from its field.
- if the reader’s record schema has a field with no default value, and writer’s schema does not have a field with the same name, an error is signalled.
-
if both are enums:
if the writer’s symbol is not present in the reader’s enum and the reader has a default value, then that value is used, otherwise an error is signalled.
-
if both are arrays:
This resolution algorithm is applied recursively to the reader’s and writer’s array item schemas.
-
if both are maps:
This resolution algorithm is applied recursively to the reader’s and writer’s value schemas.
-
if both are unions:
The first schema in the reader’s union that matches the selected writer’s union schema is recursively resolved against it. if none match, an error is signalled.
-
if reader’s is a union, but writer’s is not
The first schema in the reader’s union that matches the writer’s schema is recursively resolved against it. If none match, an error is signalled.
-
if writer’s is a union, but reader’s is not
If the reader’s schema matches the selected writer’s schema, it is recursively resolved against it. If they do not match, an error is signalled.
A schema’s doc fields are ignored for the purposes of schema resolution. Hence, the doc portion of a schema may be dropped at serialization.
One of the defining characteristics of Avro is that a reader must use the schema used by the writer of the data in order to know how to read the data. This assumption results in a data format that’s compact and also amenable to many forms of schema evolution. However, the specification so far has not defined what it means for the reader to have the “same” schema as the writer. Does the schema need to be textually identical? Well, clearly adding or removing some whitespace to a JSON expression does not change its meaning. At the same time, reordering the fields of records clearly does change the meaning. So what does it mean for a reader to have “the same” schema as a writer?
Parsing Canonical Form is a transformation of a writer’s schema that let’s us define what it means for two schemas to be “the same” for the purpose of reading data written against the schema. It is called Parsing Canonical Form because the transformations strip away parts of the schema, like “doc” attributes, that are irrelevant to readers trying to parse incoming data. It is called Canonical Form because the transformations normalize the JSON text (such as the order of attributes) in a way that eliminates unimportant differences between schemas. If the Parsing Canonical Forms of two different schemas are textually equal, then those schemas are “the same” as far as any reader is concerned, i.e., there is no serialized data that would allow a reader to distinguish data generated by a writer using one of the original schemas from data generated by a writing using the other original schema. (We sketch a proof of this property in a companion document.)
The next subsection specifies the transformations that define Parsing Canonical Form. But with a well-defined canonical form, it can be convenient to go one step further, transforming these canonical forms into simple integers (“fingerprints”) that can be used to uniquely identify schemas. The subsection after next recommends some standard practices for generating such fingerprints.
Assuming an input schema (in JSON form) that’s already UTF-8 text for a valid Avro schema (including all quotes as required by JSON), the following transformations will produce its Parsing Canonical Form:
- [PRIMITIVES] Convert primitive schemas to their simple form (e.g., int instead of
{"type":"int"}).
- [FULLNAMES] Replace short names with fullnames, using applicable namespaces to do so. Then eliminate namespace attributes, which are now redundant.
- [STRIP] Keep only attributes that are relevant to parsing data, which are: type, name, fields, symbols, items, values, size. Strip all others (e.g., doc and aliases).
- [ORDER] Order the appearance of fields of JSON objects as follows: name, type, fields, symbols, items, values, size. For example, if an object has type, name, and size fields, then the name field should appear first, followed by the type and then the size fields.
- [STRINGS] For all JSON string literals in the schema text, replace any escaped characters (e.g., \uXXXX escapes) with their UTF-8 equivalents.
- [INTEGERS] Eliminate quotes around and any leading zeros in front of JSON integer literals (which appear in the size attributes of fixed schemas).
- [WHITESPACE] Eliminate all whitespace in JSON outside of string literals.
Schema Fingerprints
“[A] fingerprinting algorithm is a procedure that maps an arbitrarily large data item (such as a computer file) to a much shorter bit string, its fingerprint, that uniquely identifies the original data for all practical purposes” (quoted from Wikipedia). In the Avro context, fingerprints of Parsing Canonical Form can be useful in a number of applications; for example, to cache encoder and decoder objects, to tag data items with a short substitute for the writer’s full schema, and to quickly negotiate common-case schemas between readers and writers.
In designing fingerprinting algorithms, there is a fundamental trade-off between the length of the fingerprint and the probability of collisions. To help application designers find appropriate points within this trade-off space, while encouraging interoperability and ease of implementation, we recommend using one of the following three algorithms when fingerprinting Avro schemas:
- When applications can tolerate longer fingerprints, we recommend using the SHA-256 digest algorithm to generate 256-bit fingerprints of Parsing Canonical Forms. Most languages today have SHA-256 implementations in their libraries.
- At the opposite extreme, the smallest fingerprint we recommend is a 64-bit Rabin fingerprint. Below, we provide pseudo-code for this algorithm that can be easily translated into any programming language. 64-bit fingerprints should guarantee uniqueness for schema caches of up to a million entries (for such a cache, the chance of a collision is 3E-8). We don’t recommend shorter fingerprints, as the chances of collisions is too great (for example, with 32-bit fingerprints, a cache with as few as 100,000 schemas has a 50% chance of having a collision).
- Between these two extremes, we recommend using the MD5 message digest to generate 128-bit fingerprints. These make sense only where very large numbers of schemas are being manipulated (tens of millions); otherwise, 64-bit fingerprints should be sufficient. As with SHA-256, MD5 implementations are found in most libraries today.
These fingerprints are not meant to provide any security guarantees, even the longer SHA-256-based ones. Most Avro applications should be surrounded by security measures that prevent attackers from writing random data and otherwise interfering with the consumers of schemas. We recommend that these surrounding mechanisms be used to prevent collision and pre-image attacks (i.e., “forgery”) on schema fingerprints, rather than relying on the security properties of the fingerprints themselves.
Rabin fingerprints are cyclic redundancy checks computed using irreducible polynomials. In the style of the Appendix of RFC 1952 (pg 10), which defines the CRC-32 algorithm, here’s our definition of the 64-bit AVRO fingerprinting algorithm:
long fingerprint64(byte[] buf) {
if (FP_TABLE == null) initFPTable();
long fp = EMPTY;
for (int i = 0; i < buf.length; i++)
fp = (fp >>> 8) ^ FP_TABLE[(int)(fp ^ buf[i]) & 0xff];
return fp;
}
static long EMPTY = 0xc15d213aa4d7a795L;
static long[] FP_TABLE = null;
void initFPTable() {
FP_TABLE = new long[256];
for (int i = 0; i < 256; i++) {
long fp = i;
for (int j = 0; j < 8; j++)
fp = (fp >>> 1) ^ (EMPTY & -(fp & 1L));
FP_TABLE[i] = fp;
}
}
Readers interested in the mathematics behind this algorithm may want to read Chapter 14 of the Second Edition of Hacker’s Delight. (Unlike RFC-1952 and the book chapter, we prepend a single one bit to messages. We do this because CRCs ignore leading zero bits, which can be problematic. Our code prepends a one-bit by initializing fingerprints using EMPTY, rather than initializing using zero as in RFC-1952 and the book chapter.)
Logical Types
A logical type is an Avro primitive or complex type with extra attributes to represent a derived type. The attribute logicalType must always be present for a logical type, and is a string with the name of one of the logical types listed later in this section. Other attributes may be defined for particular logical types.
A logical type is always serialized using its underlying Avro type so that values are encoded in exactly the same way as the equivalent Avro type that does not have a logicalType attribute. Language implementations may choose to represent logical types with an appropriate native type, although this is not required.
Language implementations must ignore unknown logical types when reading, and should use the underlying Avro type. If a logical type is invalid, for example a decimal with scale greater than its precision, then implementations should ignore the logical type and use the underlying Avro type.
Decimal
The decimal logical type represents an arbitrary-precision signed decimal number of the form unscaled × 10-scale.
A decimal logical type annotates Avro bytes or fixed types. The byte array must contain the two’s-complement representation of the unscaled integer value in big-endian byte order. The scale is fixed, and is specified using an attribute.
The following attributes are supported:
- scale, a JSON integer representing the scale (optional). If not specified the scale is 0.
- precision, a JSON integer representing the (maximum) precision of decimals stored in this type (required).
For example, the following schema represents decimal numbers with a maximum precision of 4 and a scale of 2:
{
"type": "bytes",
"logicalType": "decimal",
"precision": 4,
"scale": 2
}
Precision must be a positive integer greater than zero. If the underlying type is a fixed, then the precision is limited by its size. An array of length n can store at most floor(log10(28 × n - 1 - 1)) base-10 digits of precision.
Scale must be zero or a positive integer less than or equal to the precision.
For the purposes of schema resolution, two schemas that are decimal logical types match if their scales and precisions match.
UUID
The uuid logical type represents a random generated universally unique identifier (UUID).
A uuid logical type annotates an Avro string. The string has to conform with RFC-4122
Date
The date logical type represents a date within the calendar, with no reference to a particular time zone or time of day.
A date logical type annotates an Avro int, where the int stores the number of days from the unix epoch, 1 January 1970 (ISO calendar).
The following schema represents a date:
{
"type": "int",
"logicalType": "date"
}
Time (millisecond precision)
The time-millis logical type represents a time of day, with no reference to a particular calendar, time zone or date, with a precision of one millisecond.
A time-millis logical type annotates an Avro int, where the int stores the number of milliseconds after midnight, 00:00:00.000.
Time (microsecond precision)
The time-micros logical type represents a time of day, with no reference to a particular calendar, time zone or date, with a precision of one microsecond.
A time-micros logical type annotates an Avro long, where the long stores the number of microseconds after midnight, 00:00:00.000000.
Timestamp (millisecond precision)
The timestamp-millis logical type represents an instant on the global timeline, independent of a particular time zone or calendar, with a precision of one millisecond. Please note that time zone information gets lost in this process. Upon reading a value back, we can only reconstruct the instant, but not the original representation. In practice, such timestamps are typically displayed to users in their local time zones, therefore they may be displayed differently depending on the execution environment.
A timestamp-millis logical type annotates an Avro long, where the long stores the number of milliseconds from the unix epoch, 1 January 1970 00:00:00.000 UTC.
Timestamp (microsecond precision)
The timestamp-micros logical type represents an instant on the global timeline, independent of a particular time zone or calendar, with a precision of one microsecond. Please note that time zone information gets lost in this process. Upon reading a value back, we can only reconstruct the instant, but not the original representation. In practice, such timestamps are typically displayed to users in their local time zones, therefore they may be displayed differently depending on the execution environment.
A timestamp-micros logical type annotates an Avro long, where the long stores the number of microseconds from the unix epoch, 1 January 1970 00:00:00.000000 UTC.
Local timestamp (millisecond precision)
The local-timestamp-millis logical type represents a timestamp in a local timezone, regardless of what specific time zone is considered local, with a precision of one millisecond.
A local-timestamp-millis logical type annotates an Avro long, where the long stores the number of milliseconds, from 1 January 1970 00:00:00.000.
Local timestamp (microsecond precision)
The local-timestamp-micros logical type represents a timestamp in a local timezone, regardless of what specific time zone is considered local, with a precision of one microsecond.
A local-timestamp-micros logical type annotates an Avro long, where the long stores the number of microseconds, from 1 January 1970 00:00:00.000000.
Duration
The duration logical type represents an amount of time defined by a number of months, days and milliseconds. This is not equivalent to a number of milliseconds, because, depending on the moment in time from which the duration is measured, the number of days in the month and number of milliseconds in a day may differ. Other standard periods such as years, quarters, hours and minutes can be expressed through these basic periods.
A duration logical type annotates Avro fixed type of size 12, which stores three little-endian unsigned integers that represent durations at different granularities of time. The first stores a number in months, the second stores a number in days, and the third stores a number in milliseconds.
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
2.4 - Trevni
Version 0.1
DRAFT
This document is the authoritative specification of a file format. Its intent is to permit compatible, independent implementations that read and/or write files in this format.
Introduction
Data sets are often described as a table composed of rows and columns. Each record in the dataset is considered a row, with each field of the record occupying a different column. Writing records to a file one-by-one as they are created results in a row-major format, like Hadoop’s SequenceFile or Avro data files.
In many cases higher query performance may be achieved if the data is instead organized in a column-major format, where multiple values of a given column are stored adjacently. This document defines such a column-major file format for datasets.
To permit scalable, distributed query evaluation, datasets are partitioned into row groups, containing distinct collections of rows. Each row group is organized in column-major order, while row groups form a row-major partitioning of the entire dataset.
Rationale
Goals
The format is meant satisfy the following goals:
Maximize the size of row groups. Disc drives are used most efficiently when sequentially accessing data. Consider a drive that takes 10ms to seek and transfers at 100MB/second. If a 10-column dataset whose values are all the same size is split into 10MB row groups, then accessing a single column will require a sequence of seek+1MB reads, for a cost of 20ms/MB processed. If the same dataset is split into 100MB row groups then this drops to 11ms/MB processed. This effect is exaggerated for datasets with larger numbers of columns and with columns whose values are smaller than average. So we’d prefer row groups that are 100MB or greater.
Permit random access within a row group. Some queries will first examine one column, and, only when certain relatively rare criteria are met, examine other columns. Rather than iterating through selected columns of the row-group in parallel, one might iterate through one column and randomly access another. This is called support for WHERE clauses, after the SQL operator of that name.
Minimize the number of files per dataset. HDFS is a primary intended deployment platform for these files. The HDFS Namenode requires memory for each file in the filesystem, thus for a format to be HDFS-friendly it should strive to require the minimum number of distinct files.
Support co-location of columns within row-groups. Row groups are the unit of parallel operation on a column dataset. For efficient file i/o, the entirety of a row-group should ideally reside on the host that is evaluating the query in order to avoid network latencies and bottlenecks.
Data integrity. The format should permit applications to detect data corruption. Many file systems may prevent corruption, but files may be moved between filesystems and be subject to corruption at points in that process. It is best if the data in a file can be validated independently.
Extensibility. The format should permit applications to store additional annotations about a datasets in the files, such as type information, origin, etc. Some environments may have metadata stores for such information, but not all do, and files might be moved among systems with different metadata systems. The ability to keep such information within the file simplifies the coordination of such information.
Minimal overhead. The column format should not make datasets appreciably larger. Storage is a primary cost and a choice to use this format should not require additional storage.
Primary format. The column format should be usable as a primary format for datasets, not as an auxiliary, accelerated format. Applications that process a dataset in row-major order should be able to easily consume column files and applications that produce datasets in row-major order should be able to easily generate column files.
Design
To meet these goals we propose the following design.
Each row group is a separate file. All values of a column in a file are written contiguously. This maximizes the row group size, optimizing performance when querying few and small columns.
Each file occupies a single HDFS block. A larger than normal block size may be specified, e.g., ~1GB instead of the typical ~100MB. This guarantees co-location and eliminates network use when query processing can be co-located with the file. This also moderates the memory impact on the HDFS Namenode since no small files are written.
Each column in a file is written as a sequence of ~64kB compressed blocks. The sequence is prefixed by a table describing all of the blocks in the column to permit random access within the column.
Application-specific metadata may be added at the file, column, and block levels.
Checksums are included with each block, providing data integrity.
Discussion
The use of a single block per file achieves the same effect as the custom block placement policy described in the CIF paper, but while still permitting HDFS rebalancing and not increasing the number of files in the namespace.
This section formally describes the proposed column file format.
Data Model
We assume a simple data model, where a record is a set of named fields, and the value of each field is a sequence of untyped bytes. A type system may be layered on top of this, as specified in the Type Mapping section below.
Primitive Values
We define the following primitive value types:
- Signed 64-bit long values are written using a variable-length zig-zag coding, where the high-order bit in each byte determines whether subsequent bytes are present. For example:
decimal value hex bytes
| decimal value |
hex bytes |
|
| 0 |
00 |
|
| -1 |
01 |
|
| 1 |
02 |
|
| … |
|
|
| -64 |
7f |
|
| 64 |
80 01 |
|
| … |
|
|
- bytes are encoded as a long followed by that many bytes of data.
- a string is encoded as a long followed by that many bytes of UTF-8 encoded character data.
For example, the three-character string “foo” would be encoded as the long value 3 (encoded as hex 06) followed by the UTF-8 encoding of ‘f’, ‘o’, and ‘o’ (the hex bytes 66 6f 6f): 06 66 6f 6f
Type Names
The following type names are used to describe column values:
- null, requires zero bytes. Sometimes used in array columns.
- boolean, one bit, packed into bytes, little-endian;
- int, like long, but restricted to 32-bit signed values
- long 64-bit signed values, represented as above
- fixed32 32-bit values stored as four bytes, little-endian.
- fixed64 64-bit values stored as eight bytes, little-endian.
- float 32-bit IEEE floating point value, little-endian
- double 64-bit IEEE floating point value, little-endian
- string as above
- bytes as above, may be used to encapsulate more complex objects
Type names are represented as strings (UTF-8 encoded, length-prefixed).
Metadata consists of:
- A long indicating the number of metadata key/value pairs.
- For each pair, a string key and bytes value.
All metadata properties that start with “trevni.” are reserved.
The following file metadata properties are defined:
- trevni.codec the name of the default compression codec used to compress blocks, as a string. Implementations are required to support the “null” codec. Optional. If absent, it is assumed to be “null”. Codecs are described in more detail below.
- trevni.checksum the name of the checksum algorithm used in this file, as a string. Implementations are required to support the “crc-32” checksum. Optional. If absent, it is assumed to be “null”. Checksums are described in more detail below.
The following column metadata properties are defined:
- trevni.codec the name of the compression codec used to compress the blocks of this column, as a string. Implementations are required to support the “null” codec. Optional. If absent, it is assumed to be “null”. Codecs are described in more detail below.
- trevni.name the name of the column, as a string. Required.
- trevni.type the type of data in the column. One of the type names above. Required.
- trevni.values if present, indicates that the initial value of each block in this column will be stored in the block’s descriptor. Not permitted for array columns or columns that specify a parent.
- trevni.array if present, indicates that each row in this column contains a sequence of values of the named type rather than just a single value. An integer length precedes each sequence of values indicating the count of values in the sequence. If the length is negative then it indicates a sequence of zero or one lengths, where -1 indicates two zeros, -2 two ones, -3 three zeros, -4 three ones, etc.
- trevni.parent if present, the name of an array column whose lengths are also used by this column. Thus values of this column are sequences but no lengths are stored in this column.
For example, consider the following row, as JSON, where all values are primitive types, but one has multiple values.
{"id"=566, "date"=23423234234
"from"="foo@bar.com",
"to"=["bar@baz.com", "bang@foo.com"],
"content"="Hi!"}
The columns for this might be specified as:
name=id type=int
name=date type=long
name=from type=string
name=to type=string array=true
name=content type=string
If a row contains an array of records, e.g. “received” in the following:
{"id"=566, "date"=23423234234
"from"="foo@bar.com",
"to"=["bar@baz.com", "bang@foo.com"],
"content"="Hi!"
"received"=[{"date"=234234234234, "host"="192.168.0.0.1"},
{"date"=234234545645, "host"="192.168.0.0.2"}]
}
Then one can define a parent column followed by a column for each field in the record, adding the following columns:
name=received type=null array=true
name=date type=long parent=received
name=host type=string parent=received
If an array value itself contains an array, e.g. the “sigs” below:
{"id"=566, "date"=23423234234
"from"="foo@bar.com",
"to"=["bar@baz.com", "bang@foo.com"],
"content"="Hi!"
"received"=[{"date"=234234234234, "host"="192.168.0.0.1",
"sigs"=[{"algo"="weak", "value"="0af345de"}]},
{"date"=234234545645, "host"="192.168.0.0.2",
"sigs"=[]}]
}
Then a parent column may be defined that itself has a parent column.
name=sigs type=null array=true parent=received
name=algo type=string parent=sigs
name=value type=string parent=sigs
No block metadata properties are currently defined.
A file consists of:
- A file header, followed by
- one or more columns.
A file header consists of:
- Four bytes, ASCII ‘T’, ‘r’, ‘v’, followed by 0x02.
- a fixed64 indicating the number of rows in the file
- a fixed32 indicating the number of columns in the file
- file metadata.
- for each column, its column metadata
- for each column, its starting position in the file as a fixed64.
A column consists of:
- A fixed32 indicating the number of blocks in this column.
- For each block, a block descriptor
- One or more blocks.
A block descriptor consists of:
- A fixed32 indicating the number of rows in the block
- A fixed32 indicating the size in bytes of the block before the codec is applied (excluding checksum).
- A fixed32 indicating the size in bytes of the block after the codec is applied (excluding checksum).
- If this column’s metadata declares it to include values, the first value in the column, serialized according to this column’s type.
A block consists of:
- The serialized column values. If a column is an array column then value sequences are preceded by their length, as an int. If a codec is specified, the values and lengths are compressed by that codec.
- The checksum, as determined by the file metadata.
Codecs
null
The “null” codec simply passes data through uncompressed.
deflate
The “deflate” codec writes the data block using the deflate algorithm as specified in RFC 1951.
snappy
The “snappy” codec uses Google’s Snappy compression library.
Checksum algorithms
null
The “null” checksum contains zero bytes.
crc-32
Each “crc-32” checksum contains the four bytes of an ISO 3309 CRC-32 checksum of the uncompressed block data as a fixed32.
Type Mappings
We define a standard mapping for how types defined in various serialization systems are represented in a column file. Records from these systems are shredded into columns. When records are nested, a depth-first recursive walk can assign a separate column for each primitive value.
Avro
Protocol Buffers
Thrift
Implementation Notes
Some possible techniques for writing column files include:
- Use a standard ~100MB block, buffer in memory up to the block size, then flush the file directly to HDFS. A single reduce task might create multiple output files. The namenode requires memory proportional to the number of names and blocks*replication. This would increase the number of names but not blocks, so this should still be much better than a file per column.
- Spill each column to a separate local, temporary file then, when the file is closed, append these files, writing a single file to HDFS whose block size is set to be that of the entire file. This would be a bit slower than and may have trouble when the local disk is full, but it would better use HDFS namespace and further reduce seeks when processing columns whose values are small.
- Use a separate mapreduce job to convert row-major files to column-major. The map output would output a by (row#, column#, value) tuple, partitioned by row# but sorted by column# then row#. The reducer could directly write the column file. But the column file format would need to be changed to write counts, descriptors, etc. at the end of files rather than at the front.
(1) is the simplest to implement and most implementations should start with it.
References
CIF Column-Oriented Storage Techniques for MapReduce, Floratou, Patel, Shekita, & Tata, VLDB 2011.
DREMEL Dremel: Interactive Analysis of Web-Scale Datasets, Melnik, Gubarev, Long, Romer, Shivakumar, & Tolton, VLDB 2010.
2.5 - MapReduce guide
Avro provides a convenient way to represent complex data structures within a Hadoop MapReduce job. Avro data can be used as both input to and output from a MapReduce job, as well as the intermediate format. The example in this guide uses Avro data for all three, but it’s possible to mix and match; for instance, MapReduce can be used to aggregate a particular field in an Avro record.
This guide assumes basic familiarity with both Hadoop MapReduce and Avro. See the Hadoop documentation and the Avro getting started guide for introductions to these projects. This guide uses the old MapReduce API (org.apache.hadoop.mapred) and the new MapReduce API (org.apache.hadoop.mapreduce).
Setup
The code from this guide is included in the Avro docs under examples/mr-example. The example is set up as a Maven project that includes the necessary Avro and MapReduce dependencies and the Avro Maven plugin for code generation, so no external jars are needed to run the example. In particular, the POM includes the following dependencies:
<dependency>
<groupId>org.apache.avro</groupId>
<artifactId>avro</artifactId>
<version>1.11.0</version>
</dependency>
<dependency>
<groupId>org.apache.avro</groupId>
<artifactId>avro-mapred</artifactId>
<version>1.11.0</version>
</dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-client</artifactId>
<version>3.1.2</version>
</dependency>
And the following plugin:
<plugin>
<groupId>org.apache.avro</groupId>
<artifactId>avro-maven-plugin</artifactId>
<version>1.11.0</version>
<executions>
<execution>
<phase>generate-sources</phase>
<goals>
<goal>schema</goal>
</goals>
<configuration>
<sourceDirectory>${project.basedir}/../</sourceDirectory>
<outputDirectory>${project.basedir}/target/generated-sources/</outputDirectory>
</configuration>
</execution>
</executions>
</plugin>
If you do not configure the sourceDirectory and outputDirectory properties, the defaults will be used. The sourceDirectory property defaults to src/main/avro. The outputDirectory property defaults to target/generated-sources. You can change the paths to match your project layout.
Alternatively, Avro jars can be downloaded directly from the Apache Avro™ Releases page. The relevant Avro jars for this guide are avro-1.11.0.jar and avro-mapred-1.11.0.jar, as well as avro-tools-1.11.0.jar for code generation and viewing Avro data files as JSON. In addition, you will need to install Hadoop in order to use MapReduce.
Example: ColorCount
Below is a simple example of a MapReduce that uses Avro. There is an example for both the old (org.apache.hadoop.mapred) and new (org.apache.hadoop.mapreduce) APIs under examples/mr-example/src/main/java/example/. MapredColorCount is the example for the older mapred API while MapReduceColorCount is the example for the newer mapreduce API. Both examples are below, but we will detail the mapred API in our subsequent examples.
MapredColorCount:
package example;
import java.io.IOException;
import org.apache.avro.*;
import org.apache.avro.Schema.Type;
import org.apache.avro.mapred.*;
import org.apache.hadoop.conf.*;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.mapred.*;
import org.apache.hadoop.util.*;
import example.avro.User;
public class MapredColorCount extends Configured implements Tool {
public static class ColorCountMapper extends AvroMapper<User, Pair<CharSequence, Integer>> {
@Override
public void map(User user, AvroCollector<Pair<CharSequence, Integer>> collector, Reporter reporter)
throws IOException {
CharSequence color = user.getFavoriteColor();
// We need this check because the User.favorite_color field has type ["string", "null"]
if (color == null) {
color = "none";
}
collector.collect(new Pair<CharSequence, Integer>(color, 1));
}
}
public static class ColorCountReducer extends AvroReducer<CharSequence, Integer,
Pair<CharSequence, Integer>> {
@Override
public void reduce(CharSequence key, Iterable<Integer> values,
AvroCollector<Pair<CharSequence, Integer>> collector,
Reporter reporter)
throws IOException {
int sum = 0;
for (Integer value : values) {
sum += value;
}
collector.collect(new Pair<CharSequence, Integer>(key, sum));
}
}
public int run(String[] args) throws Exception {
if (args.length != 2) {
System.err.println("Usage: MapredColorCount <input path> <output path>");
return -1;
}
JobConf conf = new JobConf(getConf(), MapredColorCount.class);
conf.setJobName("colorcount");
FileInputFormat.setInputPaths(conf, new Path(args[0]));
FileOutputFormat.setOutputPath(conf, new Path(args[1]));
AvroJob.setMapperClass(conf, ColorCountMapper.class);
AvroJob.setReducerClass(conf, ColorCountReducer.class);
// Note that AvroJob.setInputSchema and AvroJob.setOutputSchema set
// relevant config options such as input/output format, map output
// classes, and output key class.
AvroJob.setInputSchema(conf, User.getClassSchema());
AvroJob.setOutputSchema(conf, Pair.getPairSchema(Schema.create(Type.STRING),
Schema.create(Type.INT)));
JobClient.runJob(conf);
return 0;
}
public static void main(String[] args) throws Exception {
int res = ToolRunner.run(new Configuration(), new MapredColorCount(), args);
System.exit(res);
}
}
MapReduceColorCount:
package example;
import java.io.IOException;
import org.apache.avro.Schema;
import org.apache.avro.mapred.AvroKey;
import org.apache.avro.mapred.AvroValue;
import org.apache.avro.mapreduce.AvroJob;
import org.apache.avro.mapreduce.AvroKeyInputFormat;
import org.apache.avro.mapreduce.AvroKeyValueOutputFormat;
import org.apache.hadoop.conf.Configured;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.IntWritable;
import org.apache.hadoop.io.NullWritable;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.mapreduce.Job;
import org.apache.hadoop.mapreduce.Mapper;
import org.apache.hadoop.mapreduce.Reducer;
import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
import org.apache.hadoop.util.Tool;
import org.apache.hadoop.util.ToolRunner;
import example.avro.User;
public class MapReduceColorCount extends Configured implements Tool {
public static class ColorCountMapper extends
Mapper<AvroKey<User>, NullWritable, Text, IntWritable> {
@Override
public void map(AvroKey<User> key, NullWritable value, Context context)
throws IOException, InterruptedException {
CharSequence color = key.datum().getFavoriteColor();
if (color == null) {
color = "none";
}
context.write(new Text(color.toString()), new IntWritable(1));
}
}
public static class ColorCountReducer extends
Reducer<Text, IntWritable, AvroKey<CharSequence>, AvroValue<Integer>> {
@Override
public void reduce(Text key, Iterable<IntWritable> values,
Context context) throws IOException, InterruptedException {
int sum = 0;
for (IntWritable value : values) {
sum += value.get();
}
context.write(new AvroKey<CharSequence>(key.toString()), new AvroValue<Integer>(sum));
}
}
public int run(String[] args) throws Exception {
if (args.length != 2) {
System.err.println("Usage: MapReduceColorCount <input path> <output path>");
return -1;
}
Job job = new Job(getConf());
job.setJarByClass(MapReduceColorCount.class);
job.setJobName("Color Count");
FileInputFormat.setInputPaths(job, new Path(args[0]));
FileOutputFormat.setOutputPath(job, new Path(args[1]));
job.setInputFormatClass(AvroKeyInputFormat.class);
job.setMapperClass(ColorCountMapper.class);
AvroJob.setInputKeySchema(job, User.getClassSchema());
job.setMapOutputKeyClass(Text.class);
job.setMapOutputValueClass(IntWritable.class);
job.setOutputFormatClass(AvroKeyValueOutputFormat.class);
job.setReducerClass(ColorCountReducer.class);
AvroJob.setOutputKeySchema(job, Schema.create(Schema.Type.STRING));
AvroJob.setOutputValueSchema(job, Schema.create(Schema.Type.INT));
return (job.waitForCompletion(true) ? 0 : 1);
}
public static void main(String[] args) throws Exception {
int res = ToolRunner.run(new MapReduceColorCount(), args);
System.exit(res);
}
}
ColorCount reads in data files containing User records, defined in examples/user.avsc, and counts the number of instances of each favorite color. (This example draws inspiration from the canonical WordCount MapReduce application.) This example uses the old MapReduce API. See MapReduceAvroWordCount, found under doc/examples/mr-example/src/main/java/example/ to see the new MapReduce API example. The User schema is defined as follows:
{"namespace": "example.avro",
"type": "record",
"name": "User",
"fields": [
{"name": "name", "type": "string"},
{"name": "favorite_number", "type": ["int", "null"]},
{"name": "favorite_color", "type": ["string", "null"]}
]
}
This schema is compiled into the User class used by ColorCount via the Avro Maven plugin (see examples/mr-example/pom.xml for how this is set up).
ColorCountMapper essentially takes a User as input and extracts the User’s favorite color, emitting the key-value pair <favoriteColor, 1>. ColorCountReducer then adds up how many occurrences of a particular favorite color were emitted, and outputs the result as a Pair record. These Pairs are serialized to an Avro data file.
Running ColorCount
The ColorCount application is provided as a Maven project in the Avro docs under examples/mr-example. To build the project, including the code generation of the User schema, run:
Next, run GenerateData from examples/mr-examples to create an Avro data file, input/users.avro, containing 20 Users with favorite colors chosen randomly from a list:
mvn exec:java -q -Dexec.mainClass=example.GenerateData
Besides creating the data file, GenerateData prints the JSON representations of the Users generated to stdout, for example:
{"name": "user", "favorite_number": null, "favorite_color": "red"}
{"name": "user", "favorite_number": null, "favorite_color": "green"}
{"name": "user", "favorite_number": null, "favorite_color": "purple"}
{"name": "user", "favorite_number": null, "favorite_color": null}
...
Now we’re ready to run ColorCount. We specify our freshly-generated input folder as the input path and output as our output folder (note that MapReduce will not start a job if the output folder already exists):
mvn exec:java -q -Dexec.mainClass=example.MapredColorCount -Dexec.args="input output"
Once ColorCount completes, checking the contents of the new output directory should yield the following:
$ ls output/
part-00000.avro _SUCCESS
You can check the contents of the generated Avro file using the avro-tools jar:
$ java -jar /path/to/avro-tools-1.11.0.jar tojson output/part-00000.avro
{"value": 3, "key": "blue"}
{"value": 7, "key": "green"}
{"value": 1, "key": "none"}
{"value": 2, "key": "orange"}
{"value": 3, "key": "purple"}
{"value": 2, "key": "red"}
{"value": 2, "key": "yellow"}
Now let’s go over the ColorCount example in detail.
Mapper - org.apache.hadoop.mapred API
The easiest way to use Avro data files as input to a MapReduce job is to subclass AvroMapper. An AvroMapper defines a map function that takes an Avro datum as input and outputs a key/value pair represented as a Pair record. In the ColorCount example, ColorCountMapper is an AvroMapper that takes a User as input and outputs a Pair<CharSequence, Integer>>, where the CharSequence key is the user’s favorite color and the Integer value is 1.
public static class ColorCountMapper extends AvroMapper<User, Pair<CharSequence, Integer>> {
@Override
public void map(User user, AvroCollector<Pair<CharSequence, Integer>> collector, Reporter reporter)
throws IOException {
CharSequence color = user.getFavoriteColor();
// We need this check because the User.favorite_color field has type ["string", "null"]
if (color == null) {
color = "none";
}
collector.collect(new Pair<CharSequence, Integer>(color, 1));
}
}
In order to use our AvroMapper, we must call AvroJob.setMapperClass and AvroJob.setInputSchema.
AvroJob.setMapperClass(conf, ColorCountMapper.class);
AvroJob.setInputSchema(conf, User.getClassSchema());
Note that AvroMapper does not implement the Mapper interface. Under the hood, the specified Avro data files are deserialized into AvroWrappers containing the actual data, which are processed by a Mapper that calls the configured AvroMapper’s map function. AvroJob.setInputSchema sets up the relevant configuration parameters needed to make this happen, thus you should not need to call JobConf.setMapperClass, JobConf.setInputFormat, JobConf.setMapOutputKeyClass, JobConf.setMapOutputValueClass, or JobConf.setOutputKeyComparatorClass.
Mapper - org.apache.hadoop.mapreduce API
This document will not go into all the differences between the mapred and mapreduce APIs, however will describe the main differences. As you can see, ColorCountMapper is now a subclass of the Hadoop Mapper class and is passed an AvroKey as it’s key. Additionally, the AvroJob method calls were slightly changed.
public static class ColorCountMapper extends
Mapper<AvroKey<User>, NullWritable, Text, IntWritable> {
@Override
public void map(AvroKey<User> key, NullWritable value, Context context)
throws IOException, InterruptedException {
CharSequence color = key.datum().getFavoriteColor();
if (color == null) {
color = "none";
}
context.write(new Text(color.toString()), new IntWritable(1));
}
}
Reducer - org.apache.hadoop.mapred API
Analogously to AvroMapper, an AvroReducer defines a reducer function that takes the key/value types output by an AvroMapper (or any mapper that outputs Pairs) and outputs a key/value pair represented a Pair record. In the ColorCount example, ColorCountReducer is an AvroReducer that takes the CharSequence key representing a favorite color and the Iterable<Integer> representing the counts for that color (they should all be 1 in this example) and adds up the counts.
public static class ColorCountReducer extends AvroReducer<CharSequence, Integer,
Pair<CharSequence, Integer>> {
@Override
public void reduce(CharSequence key, Iterable<Integer> values,
AvroCollector<Pair<CharSequence, Integer>> collector,
Reporter reporter)
throws IOException {
int sum = 0;
for (Integer value : values) {
sum += value;
}
collector.collect(new Pair<CharSequence, Integer>(key, sum));
}
}
In order to use our AvroReducer, we must call AvroJob.setReducerClass and AvroJob.setOutputSchema.
AvroJob.setReducerClass(conf, ColorCountReducer.class);
AvroJob.setOutputSchema(conf, Pair.getPairSchema(Schema.create(Type.STRING),
Schema.create(Type.INT)));
Note that AvroReducer does not implement the Reducer interface. The intermediate Pairs output by the mapper are split into AvroKeys and AvroValues, which are processed by a Reducer that calls the configured AvroReducer’s reduce function. AvroJob.setOutputSchema sets up the relevant configuration parameters needed to make this happen, thus you should not need to call JobConf.setReducerClass, JobConf.setOutputFormat, JobConf.setOutputKeyClass, JobConf.setMapOutputKeyClass, JobConf.setMapOutputValueClass, or JobConf.setOutputKeyComparatorClass.
Reduce - org.apache.hadoop.mapreduce API
As before we not detail every difference between the APIs. As with the Mapper change ColorCountReducer is now a subclass of Reducer and AvroKey and AvroValue are emitted. Additionally, the AvroJob method calls were slightly changed.
public static class ColorCountReducer extends
Reducer<Text, IntWritable, AvroKey<CharSequence>, AvroValue<Integer>> {
@Override
public void reduce(Text key, Iterable<IntWritable> values,
Context context) throws IOException, InterruptedException {
int sum = 0;
for (IntWritable value : values) {
sum += value.get();
}
context.write(new AvroKey<CharSequence>(key.toString()), new AvroValue<Integer>(sum));
}
}
Learning more
The mapred API allows users to mix Avro AvroMappers and AvroReducers with non-Avro Mappers and Reducers and the mapreduce API allows users input Avro and output non-Avro or vice versa.
The mapred package has API org.apache.avro.mapred documentation as does the org.apache.avro.mapreduce package. MapReduce API (org.apache.hadoop.mapreduce). Similarily to the mapreduce package, it’s possible with the mapred API to implement your own Mappers and Reducers directly using the public classes provided in these libraries. See the AvroWordCount application, found under examples/mr-example/src/main/java/example/AvroWordCount.java in the Avro documentation, for an example of implementing a Reducer that outputs Avro data using the old MapReduce API. See the MapReduceAvroWordCount application, found under examples/mr-example/src/main/java/example/MapReduceAvroWordCount.java in the Avro documentation, for an example of implementing a Reducer that outputs Avro data using the new MapReduce API.
2.6 - IDL Language
Introduction
This document defines Avro IDL, a higher-level language for authoring Avro schemata. Before reading this document, you should have familiarity with the concepts of schemata and protocols, as well as the various primitive and complex types available in Avro.
Overview
Purpose
The aim of the Avro IDL language is to enable developers to author schemata in a way that feels more similar to common programming languages like Java, C++, or Python. Additionally, the Avro IDL language may feel more familiar for those users who have previously used the interface description languages (IDLs) in other frameworks like Thrift, Protocol Buffers, or CORBA.
Usage
Each Avro IDL file defines a single Avro Protocol, and thus generates as its output a JSON-format Avro Protocol file with extension .avpr.
To convert a .avdl file into a .avpr file, it may be processed by the idl tool. For example:
$ java -jar avro-tools.jar idl src/test/idl/input/namespaces.avdl /tmp/namespaces.avpr
$ head /tmp/namespaces.avpr
{
"protocol" : "TestNamespace",
"namespace" : "avro.test.protocol",
The idl tool can also process input to and from stdin and stdout. See idl --help for full usage information.
A Maven plugin is also provided to compile .avdl files. To use it, add something like the following to your pom.xml:
<build>
<plugins>
<plugin>
<groupId>org.apache.avro</groupId>
<artifactId>avro-maven-plugin</artifactId>
<executions>
<execution>
<goals>
<goal>idl-protocol</goal>
</goals>
</execution>
</executions>
</plugin>
</plugins>
</build>
Defining a Protocol in Avro IDL
An Avro IDL file consists of exactly one protocol definition. The minimal protocol is defined by the following code:
This is equivalent to (and generates) the following JSON protocol definition:
{
"protocol" : "MyProtocol",
"types" : [ ],
"messages" : {
}
}
The namespace of the protocol may be changed using the @namespace annotation:
@namespace("mynamespace")
protocol MyProtocol {
}
This notation is used throughout Avro IDL as a way of specifying properties for the annotated element, as will be described later in this document.
Protocols in Avro IDL can contain the following items:
- Imports of external protocol and schema files.
- Definitions of named schemata, including records, errors, enums, and fixeds.
- Definitions of RPC messages
Imports
Files may be imported in one of three formats:
-
An IDL file may be imported with a statement like:
import idl "foo.avdl";
-
A JSON protocol file may be imported with a statement like:
import protocol "foo.avpr";
-
A JSON schema file may be imported with a statement like:
import schema "foo.avsc";
Messages and types in the imported file are added to this file’s protocol.
Imported file names are resolved relative to the current IDL file.
Defining an Enumeration
Enums are defined in Avro IDL using a syntax similar to C or Java. An Avro Enum supports optional default values. In the case that a reader schema is unable to recognize a symbol written by the writer, the reader will fall back to using the defined default value. This default is only used when an incompatible symbol is read. It is not used if the enum field is missing.
Example Writer Enum Definition
enum Shapes {
SQUARE, TRIANGLE, CIRCLE, OVAL
}
Example Reader Enum Definition
enum Shapes {
SQUARE, TRIANGLE, CIRCLE
} = CIRCLE;
In the above example, the reader will use the default value of CIRCLE whenever reading data written with the OVAL symbol of the writer. Also note that, unlike the JSON format, anonymous enums cannot be defined.
Defining a Fixed Length Field
Fixed fields are defined using the following syntax:
fixed MD5(16);
This example defines a fixed-length type called MD5 which contains 16 bytes.
Defining Records and Errors
Records are defined in Avro IDL using a syntax similar to a struct definition in C:
record Employee {
string name;
boolean active = true;
long salary;
}
The above example defines a record with the name “Employee” with three fields.
To define an error, simply use the keyword error instead of record. For example:
error Kaboom {
string explanation;
int result_code = -1;
}
Each field in a record or error consists of a type and a name, optional property annotations and an optional default value.
A type reference in Avro IDL must be one of:
- A primitive type
- A logical type
- A named schema defined prior to this usage in the same Protocol
- A complex type (array, map, or union)
Primitive Types
The primitive types supported by Avro IDL are the same as those supported by Avro’s JSON format. This list includes int, long, string, boolean, float, double, null, and bytes.
Logical Types
Some of the logical types supported by Avro’s JSON format are also supported by Avro IDL. The currently supported types are:
For example:
record Job {
string jobid;
date submitDate;
time_ms submitTime;
timestamp_ms finishTime;
decimal(9,2) finishRatio;
}
References to Named Schemata
If a named schema has already been defined in the same Avro IDL file, it may be referenced by name as if it were a primitive type:
record Card {
Suit suit; // refers to the enum Card defined above
int number;
}
Default Values
Default values for fields may be optionally specified by using an equals sign after the field name followed by a JSON expression indicating the default value. This JSON is interpreted as described in the spec.
Complex Types
Arrays
Array types are written in a manner that will seem familiar to C++ or Java programmers. An array of any type t is denoted array<t>. For example, an array of strings is denoted array<string>, and a multidimensional array of Foo records would be array<array<Foo>>.
Maps
Map types are written similarly to array types. An array that contains values of type t is written map<t>. As in the JSON schema format, all maps contain string-type keys.
Unions
Union types are denoted as union { typeA, typeB, typeC, … }. For example, this record contains a string field that is optional (unioned with null):
record RecordWithUnion {
union { null, string } optionalString;
union { decimal(12, 6), float } number;
}
Note that the same restrictions apply to Avro IDL unions as apply to unions defined in the JSON format; namely, a record may not contain multiple elements of the same type.
Defining RPC Messages
The syntax to define an RPC message within a Avro IDL protocol is similar to the syntax for a method declaration within a C header file or a Java interface. To define an RPC message add which takes two arguments named foo and bar, returning an int, simply include the following definition within the protocol:
int add(int foo, int bar = 0);
Message arguments, like record fields, may specify default values.
To define a message with no response, you may use the alias void, equivalent to the Avro null type:
void logMessage(string message);
If you have previously defined an error type within the same protocol, you may declare that a message can throw this error using the syntax:
void goKaboom() throws Kaboom;
To define a one-way message, use the keyword oneway after the parameter list, for example:
void fireAndForget(string message) oneway;
Other Language Features
All Java-style comments are supported within a Avro IDL file. Any text following // on a line is ignored, as is any text between /* and */, possibly spanning multiple lines.
Comments that begin with /** are used as the documentation string for the type or field definition that follows the comment.
Escaping Identifiers
Occasionally, one will need to use a reserved language keyword as an identifier. In order to do so, backticks (`) may be used to escape the identifier. For example, to define a message with the literal name error, you may write:
This syntax is allowed anywhere an identifier is expected.
Annotations for Ordering and Namespaces
Java-style annotations may be used to add additional properties to types and fields throughout Avro IDL.
For example, to specify the sort order of a field within a record, one may use the @order annotation before the field name as follows:
record MyRecord {
string @order("ascending") myAscendingSortField;
string @order("descending") myDescendingField;
string @order("ignore") myIgnoredField;
}
A field’s type may also be preceded by annotations, e.g.:
record MyRecord {
@java-class("java.util.ArrayList") array<string> myStrings;
}
This can be used to support java classes that can be serialized/deserialized via their toString/String constructor, e.g.:
record MyRecord {
@java-class("java.math.BigDecimal") string value;
@java-key-class("java.io.File") map<string> fileStates;
array<@java-class("java.math.BigDecimal") string> weights;
}
Similarly, a @namespace annotation may be used to modify the namespace when defining a named schema. For example:
@namespace("org.apache.avro.firstNamespace")
protocol MyProto {
@namespace("org.apache.avro.someOtherNamespace")
record Foo {}
record Bar {}
}
will define a protocol in the firstNamespace namespace. The record Foo will be defined in someOtherNamespace and Bar will be defined in firstNamespace as it inherits its default from its container.
Type and field aliases are specified with the @aliases annotation as follows:
@aliases(["org.old.OldRecord", "org.ancient.AncientRecord"])
record MyRecord {
string @aliases(["oldField", "ancientField"]) myNewField;
}
Some annotations like those listed above are handled specially. All other annotations are added as properties to the protocol, message, schema or field.
Complete Example
The following is a complete example of a Avro IDL file that shows most of the above features:
/**
* An example protocol in Avro IDL
*/
@namespace("org.apache.avro.test")
protocol Simple {
@aliases(["org.foo.KindOf"])
enum Kind {
FOO,
BAR, // the bar enum value
BAZ
} = FOO; // For schema evolution purposes, unmatched values do not throw an error, but are resolved to FOO.
/** MD5 hash; good enough to avoid most collisions, and smaller than (for example) SHA256. */
fixed MD5(16);
record TestRecord {
@order("ignore")
string name;
@order("descending")
Kind kind;
MD5 hash;
union { MD5, null} @aliases(["hash"]) nullableHash;
array<long> arrayOfLongs;
}
/** Errors are records that can be thrown from a method */
error TestError {
string message;
}
string hello(string greeting);
TestRecord echo(TestRecord `record`);
int add(int arg1, int arg2);
bytes echoBytes(bytes data);
void `error`() throws TestError;
void ping() oneway;
}
Additional examples may be found in the Avro source tree under the src/test/idl/input directory.
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
2.7 - SASL profile
Introduction
SASL (RFC 2222) provides a framework for authentication and security of network protocols. Each protocol that uses SASL is meant to define a SASL profile. This document provides a SASL profile for connection-based Avro RPC.
Overview
SASL negotiation proceeds as a series of message interactions over a connection between a client and server using a selected SASL mechanism. The client starts this negotiation by sending its chosen mechanism name with an initial (possibly empty) message. Negotiation proceeds with the exchange of messages until either side indicates success or failure. The content of the messages is mechanism-specific. If the negotiation succeeds, then the session can proceed over the connection, otherwise it must be abandoned.
Some mechanisms continue to process session data after negotiation (e.g., encrypting it), while some specify that further session data is transmitted unmodified.
Negotiation
Commands
Avro SASL negotiation uses four one-byte commands.
- 0: START Used in a client’s initial message.
- 1: CONTINUE Used while negotiation is ongoing.
- 2: FAIL Terminates negotiation unsuccessfully.
- 3: COMPLETE Terminates negotiation successfully.
The format of a START message is:
| 0 | 4-byte mechanism name length | mechanism name | 4-byte payload length | payload data |
The format of a CONTINUE message is:
| 1 | 4-byte payload length | payload data |
The format of a FAIL message is:
| 2 | 4-byte message length | UTF-8 message |
The format of a COMPLETE message is:
| 3 | 4-byte payload length | payload data |
Process
Negotiation is initiated by a client sending a START command containing the client’s chosen mechanism name and any mechanism-specific payload data.
The server and client then interchange some number (possibly zero) of CONTINUE messages. Each message contains payload data that is processed by the security mechanism to generate the next message.
Once either the client or server send a FAIL message then negotiation has failed. UTF-8-encoded text is included in the failure message. Once either a FAIL message has been sent or received, or any other error occurs in the negotiation, further communication on this connection must cease.
Once either the client or server send a COMPLETE message then negotiation has completed successfully. Session data may now be transmitted over the connection until it is closed by either side.
Session Data
If no SASL QOP (quality of protection) is negotiated, then all subsequent writes to/reads over this connection are written/read unmodified. In particular, messages use Avro framing, and are of the form:
| 4-byte frame length | frame data | ... | 4 zero bytes |
If a SASL QOP is negotiated, then it must be used by the connection for all subsequent messages. This is done by wrapping each non-empty frame written using the security mechanism and unwrapping each non-empty frame read. The length written in each non-empty frame is the length of the wrapped data. Complete frames must be passed to the security mechanism for unwrapping. Unwrapped data is then passed to the application as the content of the frame.
If at any point processing fails due to wrapping, unwrapping or framing errors, then all further communication on this connection must cease.
Anonymous Mechanism
The SASL anonymous mechanism (RFC 2245) is quite simple to implement. In particular, an initial anonymous request may be prefixed by the following static sequence:
| 0 | 0009 | ANONYMOUS | 0000 |
If a server uses the anonymous mechanism, it should check that the mechanism name in the start message prefixing the first request received is ‘ANONYMOUS’, then simply prefix its initial response with a COMPLETE message of:
| 3 | 0000 |
If an anonymous server recieves some other mechanism name, then it may respond with a FAIL message as simple as:
| 2 | 0000 |
Note that the anonymous mechanism need add no additional round-trip messages between client and server. The START message can be piggybacked on the initial request and the COMPLETE or FAIL message can be piggybacked on the initial response.
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
2.8 - Wiki
The Wiki page can be found here.
2.9 - FAQ
What is Avro?
Avro is a data serialization system.
Avro provides:
- Rich data structures.
- A compact, fast, binary data format.
- A container file, to store persistent data.
- Remote procedure call (RPC).
- Simple integration with dynamic languages. Code generation is not required to read or write data files nor to use or implement RPC protocols. Code generation as an optional optimization, only worth implementing for statically typed languages.
How can I get started quickly with Avro?
Check out the Quick Start Guide.
How do I statically compile a schema or protocol into generated code?
In Java
- Add the avro jar, the jackson-mapper-asl.jar and jackson-core-asl.jar to your CLASSPATH.
- Run java org.apache.avro.specific.SpecificCompiler .
This appears to be out of date, the SpecificCompiler requires two arguments, presumably an input and and output file, but it isn’t clear that this does.
Or use the Schema or Protocol Ant tasks. Avro’s build.xml provides examples of how these are used.
Lastly, you can also use the “avro-tools” jar which ships with an Avro release. Just use the “compile (schema|protocol)” command.
How are Strings represented in Java?
They use org.apache.avro.util.Utf8, not java.lang.String.
More generally, how do Avro types map to Java types?
The mappings are documented in the package javadoc for generic, specific and reflect API.
From Doug Cutting:
HDFS splits files into blocks, and mapreduce runs a map task for each block. When the task starts, it needs to be able to seek into the file to the start of the block process through the block’s end. If the file were, e.g., a gzip file, this would not be possible, since gzip files must be decompressed from the start. One cannot seek into the middle of a gzip file and start decompressing. So Hadoop’s SequenceFile places a marker periodically (~64k) in the file at record and compression boundaries, where processing can be sensibly started. Then, when a map task starts processing an HDFS block, it finds the first marker after the block’s start and continues through the first marker in the next block of the file. This requires a bit of non-local access (~0.1%). Avro’s data file uses the same method as SequenceFile.
Why isn’t every value in Avro nullable?
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 of every value stored and
corresponding computational costs to create this bit on write and
interpret it on read. These costs are wasted when values may not in
fact be null, as is the case in many datasets. In Avro such costs are
only paid when values may actually be null.
Also, allowing values to be null is a well-known source of errors. In
Avro, a value declared as non-null will always be non-null and programs
need not test for null values when processing it nor will they ever fail
for lack of such tests.
Tony Hoare calls his invention of null references his “Billion Dollar
Mistake”.
http://qconlondon.com/london-2009/presentation/Null+References:+The+Billion+Dollar+Mistake
Also note that in some programming languages not all values are permitted to be null. For example, in Java, values of type boolean, byte, short, char, int, float, long, and double may not be null.
How can I serialize directly to/from a byte array?
As pointed out in the specification, Avro data should always be stored with its schema. The Avro provided classes DataFileWriter, DataFileReader, and DataFileStream all ensure this by serializing the Schema in a container header. In some special cases, such as when implementing a new storage system or writing unit tests, you may need to write and read directly with the bare Avro serialized values. The following examples use code generated for Java from an Avro Schema, specifically the User example from the Quickstart guide.
Serializing to a byte array
This example takes a User object and returns a newly allocated byte array with the Avro serialization of that user.
ByteArrayOutputStream out = new ByteArrayOutputStream();
BinaryEncoder encoder = EncoderFactory.get().binaryEncoder(out, null);
DatumWriter<User> writer = new SpecificDatumWriter<User>(User.getClassSchema());
writer.write(user, encoder);
encoder.flush();
out.close();
byte[] serializedBytes = out.toByteArray();
Deserializing from a byte array
This example takes a byte array containing the Avro serialization of a user and returns a User object.SpecificDatumReader reader = new SpecificDatumReader(User.getClassSchema());
Decoder decoder = DecoderFactory.get().binaryDecoder(bytes, null);
User user = reader.read(null, decoder);
Note: If you are serializing or deserializing in a loop or as a method, you should be reusing objects, readers and/or writers. Check the JavaDoc to see how to reuse the objects.
3 - Apache Avro™ 1.10.2 Documentation
Introduction
Apache Avro™ is a data serialization system.
Avro provides:
- Rich data structures.
- A compact, fast, binary data format.
- A container file, to store persistent data.
- Remote procedure call (RPC).
- Simple integration with dynamic languages. Code generation is not required to read or write data files nor to use or implement RPC protocols. Code generation as an optional optimization, only worth implementing for statically typed languages.
Schemas
Avro relies on schemas. When Avro data is read, the schema used when writing it is always present. This permits each datum to be written with no per-value overheads, making serialization both fast and small. This also facilitates use with dynamic, scripting languages, since data, together with its schema, is fully self-describing.
When Avro data is stored in a file, its schema is stored with it, so that files may be processed later by any program. If the program reading the data expects a different schema this can be easily resolved, since both schemas are present.
When Avro is used in RPC, the client and server exchange schemas in the connection handshake. (This can be optimized so that, for most calls, no schemas are actually transmitted.) Since both client and server both have the other’s full schema, correspondence between same named fields, missing fields, extra fields, etc. can all be easily resolved.
Avro schemas are defined with JSON . This facilitates implementation in languages that already have JSON libraries.
Comparison with other systems
Avro provides functionality similar to systems such as Thrift, Protocol Buffers, etc. Avro differs from these systems in the following fundamental aspects.
- Dynamic typing: Avro does not require that code be generated. Data is always accompanied by a schema that permits full processing of that data without code generation, static datatypes, etc. This facilitates construction of generic data-processing systems and languages.
- Untagged data: Since the schema is present when data is read, considerably less type information need be encoded with data, resulting in smaller serialization size.
- No manually-assigned field IDs: When a schema changes, both the old and new schema are always present when processing data, so differences may be resolved symbolically, using field names.
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
3.1 - Getting Started (Java)
This is a short guide for getting started with Apache Avro™ using Java. This guide only covers using Avro for data serialization; see Patrick Hunt’s Avro RPC Quick Start for a good introduction to using Avro for RPC.
Download
Avro implementations for C, C++, C#, Java, PHP, Python, and Ruby can be downloaded from the Apache Avro™ Releases page. This guide uses Avro 1.10.2, the latest version at the time of writing. For the examples in this guide, download avro-1.10.2.jar and avro-tools-1.10.2.jar.
Alternatively, if you are using Maven, add the following dependency to your POM:
<dependency>
<groupId>org.apache.avro</groupId>
<artifactId>avro</artifactId>
<version>1.10.2</version>
</dependency>
As well as the Avro Maven plugin (for performing code generation):
<plugin>
<groupId>org.apache.avro</groupId>
<artifactId>avro-maven-plugin</artifactId>
<version>1.10.2</version>
<executions>
<execution>
<phase>generate-sources</phase>
<goals>
<goal>schema</goal>
</goals>
<configuration>
<sourceDirectory>${project.basedir}/src/main/avro/</sourceDirectory>
<outputDirectory>${project.basedir}/src/main/java/</outputDirectory>
</configuration>
</execution>
</executions>
</plugin>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-compiler-plugin</artifactId>
<configuration>
<source>1.8</source>
<target>1.8</target>
</configuration>
</plugin>
You may also build the required Avro jars from source. Building Avro is beyond the scope of this guide; see the Build Documentation page in the wiki for more information.
Defining a schema
Avro schemas are defined using JSON. Schemas are composed of primitive types (null, boolean, int, long, float, double, bytes, and string) and complex types (record, enum, array, map, union, and fixed). You can learn more about Avro schemas and types from the specification, but for now let’s start with a simple schema example, user.avsc:
{"namespace": "example.avro",
"type": "record",
"name": "User",
"fields": [
{"name": "name", "type": "string"},
{"name": "favorite_number", "type": ["int", "null"]},
{"name": "favorite_color", "type": ["string", "null"]}
]
}
This schema defines a record representing a hypothetical user. (Note that a schema file can only contain a single schema definition.) At minimum, a record definition must include its type (“type”: “record”), a name (“name”: “User”), and fields, in this case name, favorite_number, and favorite_color. We also define a namespace (“namespace”: “example.avro”), which together with the name attribute defines the “full name” of the schema (example.avro.User in this case).
Fields are defined via an array of objects, each of which defines a name and type (other attributes are optional, see the record specification for more details). The type attribute of a field is another schema object, which can be either a primitive or complex type. For example, the name field of our User schema is the primitive type string, whereas the favorite_number and favorite_color fields are both unions, represented by JSON arrays. unions are a complex type that can be any of the types listed in the array; e.g., favorite_number can either be an int or null, essentially making it an optional field.
Serializing and deserializing with code generation
Compiling the schema
Code generation allows us to automatically create classes based on our previously-defined schema. Once we have defined the relevant classes, there is no need to use the schema directly in our programs. We use the avro-tools jar to generate code as follows:
java -jar /path/to/avro-tools-1.10.2.jar compile schema <schema file> <destination>
This will generate the appropriate source files in a package based on the schema’s namespace in the provided destination folder. For instance, to generate a User class in package example.avro from the schema defined above, run
java -jar /path/to/avro-tools-1.10.2.jar compile schema user.avsc .
Note that if you using the Avro Maven plugin, there is no need to manually invoke the schema compiler; the plugin automatically performs code generation on any .avsc files present in the configured source directory.
Creating Users
Now that we’ve completed the code generation, let’s create some Users, serialize them to a data file on disk, and then read back the file and deserialize the User objects.
First let’s create some Users and set their fields.
User user1 = new User();
user1.setName("Alyssa");
user1.setFavoriteNumber(256);
// Leave favorite color null
// Alternate constructor
User user2 = new User("Ben", 7, "red");
// Construct via builder
User user3 = User.newBuilder()
.setName("Charlie")
.setFavoriteColor("blue")
.setFavoriteNumber(null)
.build();
As shown in this example, Avro objects can be created either by invoking a constructor directly or by using a builder. Unlike constructors, builders will automatically set any default values specified in the schema. Additionally, builders validate the data as it set, whereas objects constructed directly will not cause an error until the object is serialized. However, using constructors directly generally offers better performance, as builders create a copy of the datastructure before it is written.
Note that we do not set user1’s favorite color. Since that record is of type [“string”, “null”], we can either set it to a string or leave it null; it is essentially optional. Similarly, we set user3’s favorite number to null (using a builder requires setting all fields, even if they are null).
Serializing
Now let’s serialize our Users to disk.
/// Serialize user1, user2 and user3 to disk
DatumWriter<User> userDatumWriter = new SpecificDatumWriter<User>(User.class);
DataFileWriter<User> dataFileWriter = new DataFileWriter<User>(userDatumWriter);
dataFileWriter.create(user1.getSchema(), new File("users.avro"));
dataFileWriter.append(user1);
dataFileWriter.append(user2);
dataFileWriter.append(user3);
dataFileWriter.close();
We create a DatumWriter, which converts Java objects into an in-memory serialized format. The SpecificDatumWriter class is used with generated classes and extracts the schema from the specified generated type.
Next we create a DataFileWriter, which writes the serialized records, as well as the schema, to the file specified in the dataFileWriter.create call. We write our users to the file via calls to the dataFileWriter.append method. When we are done writing, we close the data file.
Deserializing
Finally, let’s deserialize the data file we just created.
// Deserialize Users from disk
DatumReader<User> userDatumReader = new SpecificDatumReader<User>(User.class);
DataFileReader<User> dataFileReader = new DataFileReader<User>(file, userDatumReader);
User user = null;
while (dataFileReader.hasNext()) {
// Reuse user object by passing it to next(). This saves us from
// allocating and garbage collecting many objects for files with
// many items.
user = dataFileReader.next(user);
System.out.println(user);
}
This snippet will output:
{"name": "Alyssa", "favorite_number": 256, "favorite_color": null}
{"name": "Ben", "favorite_number": 7, "favorite_color": "red"}
{"name": "Charlie", "favorite_number": null, "favorite_color": "blue"}
Deserializing is very similar to serializing. We create a SpecificDatumReader, analogous to the SpecificDatumWriter we used in serialization, which converts in-memory serialized items into instances of our generated class, in this case User. We pass the DatumReader and the previously created File to a DataFileReader, analogous to the DataFileWriter, which reads both the schema used by the writer as well as the data from the file on disk. The data will be read using the writer’s schema included in the file and the schema provided by the reader, in this case the User class. The writer’s schema is needed to know the order in which fields were written, while the reader’s schema is needed to know what fields are expected and how to fill in default values for fields added since the file was written. If there are differences between the two schemas, they are resolved according to the Schema Resolution specification.
Next we use the DataFileReader to iterate through the serialized Users and print the deserialized object to stdout. Note how we perform the iteration: we create a single User object which we store the current deserialized user in, and pass this record object to every call of dataFileReader.next. This is a performance optimization that allows the DataFileReader to reuse the same User object rather than allocating a new User for every iteration, which can be very expensive in terms of object allocation and garbage collection if we deserialize a large data file. While this technique is the standard way to iterate through a data file, it’s also possible to use for (User user : dataFileReader) if performance is not a concern.
Compiling and running the example code
This example code is included as a Maven project in the examples/java-example directory in the Avro docs. From this directory, execute the following commands to build and run the example:
$ mvn compile # includes code generation via Avro Maven plugin
$ mvn -q exec:java -Dexec.mainClass=example.SpecificMain
Beta feature: Generating faster code
In this release we have introduced a new approach to generating code that speeds up decoding of objects by more than 10% and encoding by more than 30% (future performance enhancements are underway). To ensure a smooth introduction of this change into production systems, this feature is controlled by a feature flag, the system property org.apache.avro.specific.use_custom_coders. In this first release, this feature is off by default. To turn it on, set the system flag to true at runtime. In the sample above, for example, you could enable the fater coders as follows:
$ mvn -q exec:java -Dexec.mainClass=example.SpecificMain \
-Dorg.apache.avro.specific.use_custom_coders=true
Note that you do not have to recompile your Avro schema to have access to this feature. The feature is compiled and built into your code, and you turn it on and off at runtime using the feature flag. As a result, you can turn it on during testing, for example, and then off in production. Or you can turn it on in production, and quickly turn it off if something breaks.
We encourage the Avro community to exercise this new feature early to help build confidence. (For those paying one-demand for compute resources in the cloud, it can lead to meaningful cost savings.) As confidence builds, we will turn this feature on by default, and eventually eliminate the feature flag (and the old code).
Serializing and deserializing without code generation
Data in Avro is always stored with its corresponding schema, meaning we can always read a serialized item regardless of whether we know the schema ahead of time. This allows us to perform serialization and deserialization without code generation.
Let’s go over the same example as in the previous section, but without using code generation: we’ll create some users, serialize them to a data file on disk, and then read back the file and deserialize the users objects.
Creating users
First, we use a Parser to read our schema definition and create a Schema object.
Schema schema = new Schema.Parser().parse(new File("user.avsc"));
Using this schema, let’s create some users.
GenericRecord user1 = new GenericData.Record(schema);
user1.put("name", "Alyssa");
user1.put("favorite_number", 256);
// Leave favorite color null
GenericRecord user2 = new GenericData.Record(schema);
user2.put("name", "Ben");
user2.put("favorite_number", 7);
user2.put("favorite_color", "red");
Since we’re not using code generation, we use GenericRecords to represent users. GenericRecord uses the schema to verify that we only specify valid fields. If we try to set a non-existent field (e.g., user1.put(“favorite_animal”, “cat”)), we’ll get an AvroRuntimeException when we run the program.
Note that we do not set user1’s favorite color. Since that record is of type [“string”, “null”], we can either set it to a string or leave it null; it is essentially optional.
Serializing
Now that we’ve created our user objects, serializing and deserializing them is almost identical to the example above which uses code generation. The main difference is that we use generic instead of specific readers and writers.
First we’ll serialize our users to a data file on disk.
// Serialize user1 and user2 to disk
File file = new File("users.avro");
DatumWriter<GenericRecord> datumWriter = new GenericDatumWriter<GenericRecord>(schema);
DataFileWriter<GenericRecord> dataFileWriter = new DataFileWriter<GenericRecord>(datumWriter);
dataFileWriter.create(schema, file);
dataFileWriter.append(user1);
dataFileWriter.append(user2);
dataFileWriter.close();
We create a DatumWriter, which converts Java objects into an in-memory serialized format. Since we are not using code generation, we create a GenericDatumWriter. It requires the schema both to determine how to write the GenericRecords and to verify that all non-nullable fields are present.
As in the code generation example, we also create a DataFileWriter, which writes the serialized records, as well as the schema, to the file specified in the dataFileWriter.create call. We write our users to the file via calls to the dataFileWriter.append method. When we are done writing, we close the data file.
Deserializing
Finally, we’ll deserialize the data file we just created.
// Deserialize users from disk
DatumReader<GenericRecord> datumReader = new GenericDatumReader<GenericRecord>(schema);
DataFileReader<GenericRecord> dataFileReader = new DataFileReader<GenericRecord>(file, datumReader);
GenericRecord user = null;
while (dataFileReader.hasNext()) {
// Reuse user object by passing it to next(). This saves us from
// allocating and garbage collecting many objects for files with
// many items.
user = dataFileReader.next(user);
System.out.println(user);
This outputs:
{"name": "Alyssa", "favorite_number": 256, "favorite_color": null}
{"name": "Ben", "favorite_number": 7, "favorite_color": "red"}
Deserializing is very similar to serializing. We create a GenericDatumReader, analogous to the GenericDatumWriter we used in serialization, which converts in-memory serialized items into GenericRecords. We pass the DatumReader and the previously created File to a DataFileReader, analogous to the DataFileWriter, which reads both the schema used by the writer as well as the data from the file on disk. The data will be read using the writer’s schema included in the file, and the reader’s schema provided to the GenericDatumReader. The writer’s schema is needed to know the order in which fields were written, while the reader’s schema is needed to know what fields are expected and how to fill in default values for fields added since the file was written. If there are differences between the two schemas, they are resolved according to the Schema Resolution specification.
Next, we use the DataFileReader to iterate through the serialized users and print the deserialized object to stdout. Note how we perform the iteration: we create a single GenericRecord object which we store the current deserialized user in, and pass this record object to every call of dataFileReader.next. This is a performance optimization that allows the DataFileReader to reuse the same record object rather than allocating a new GenericRecord for every iteration, which can be very expensive in terms of object allocation and garbage collection if we deserialize a large data file. While this technique is the standard way to iterate through a data file, it’s also possible to use for (GenericRecord user : dataFileReader) if performance is not a concern.
Compiling and running the example code
This example code is included as a Maven project in the examples/java-example directory in the Avro docs. From this directory, execute the following commands to build and run the example:
$ mvn compile
$ mvn -q exec:java -Dexec.mainClass=example.GenericMain
3.2 - Getting Started (Python)
This is a short guide for getting started with Apache Avro™ using Python. This guide only covers using Avro for data serialization; see Patrick Hunt’s Avro RPC Quick Start for a good introduction to using Avro for RPC.
Notice for Python 3 users
A package called “avro-python3” had been provided to support Python 3 previously, but the codebase was consolidated into the “avro” package and that supports both Python 2 and 3 now. The avro-python3 package will be removed in the near future, so users should use the “avro” package instead. They are mostly API compatible, but there’s a few minor difference (e.g., function name capitalization, such as avro.schema.Parse vs avro.schema.parse).
Download
For Python, the easiest way to get started is to install it from PyPI. Python’s Avro API is available over PyPi.
$ python3 -m pip install avro
The official releases of the Avro implementations for C, C++, C#, Java, PHP, Python, and Ruby can be downloaded from the Apache Avro™ Releases page. This guide uses Avro 1.10.2, the latest version at the time of writing. Download and unzip avro-1.10.2.tar.gz, and install via python setup.py (this will probably require root privileges). Ensure that you can import avro from a Python prompt.
$ tar xvf avro-1.10.2.tar.gz
$ cd avro-1.10.2
$ python setup.py install
$ python
>>> import avro # should not raise ImportError
Alternatively, you may build the Avro Python library from source. From your the root Avro directory, run the commands
$ cd lang/py/
$ python3 -m pip install -e .
$ python
Defining a schema
Avro schemas are defined using JSON. Schemas are composed of primitive types (null, boolean, int, long, float, double, bytes, and string) and complex types (record, enum, array, map, union, and fixed). You can learn more about Avro schemas and types from the specification, but for now let’s start with a simple schema example, user.avsc:
{"namespace": "example.avro",
"type": "record",
"name": "User",
"fields": [
{"name": "name", "type": "string"},
{"name": "favorite_number", "type": ["int", "null"]},
{"name": "favorite_color", "type": ["string", "null"]}
]
}
This schema defines a record representing a hypothetical user. (Note that a schema file can only contain a single schema definition.) At minimum, a record definition must include its type (“type”: “record”), a name (“name”: “User”), and fields, in this case name, favorite_number, and favorite_color. We also define a namespace (“namespace”: “example.avro”), which together with the name attribute defines the “full name” of the schema (example.avro.User in this case).
Fields are defined via an array of objects, each of which defines a name and type (other attributes are optional, see the record specification for more details). The type attribute of a field is another schema object, which can be either a primitive or complex type. For example, the name field of our User schema is the primitive type string, whereas the favorite_number and favorite_color fields are both unions, represented by JSON arrays. unions are a complex type that can be any of the types listed in the array; e.g., favorite_number can either be an int or null, essentially making it an optional field.
Serializing and deserializing without code generation
Data in Avro is always stored with its corresponding schema, meaning we can always read a serialized item, regardless of whether we know the schema ahead of time. This allows us to perform serialization and deserialization without code generation. Note that the Avro Python library does not support code generation.
Try running the following code snippet, which serializes two users to a data file on disk, and then reads back and deserializes the data file:
import avro.schema
from avro.datafile import DataFileReader, DataFileWriter
from avro.io import DatumReader, DatumWriter
schema = avro.schema.parse(open("user.avsc", "rb").read())
writer = DataFileWriter(open("users.avro", "wb"), DatumWriter(), schema)
writer.append({"name": "Alyssa", "favorite_number": 256})
writer.append({"name": "Ben", "favorite_number": 7, "favorite_color": "red"})
writer.close()
reader = DataFileReader(open("users.avro", "rb"), DatumReader())
for user in reader:
print user
reader.close()
This outputs:
{u'favorite_color': None, u'favorite_number': 256, u'name': u'Alyssa'}
{u'favorite_color': u'red', u'favorite_number': 7, u'name': u'Ben'}
Do make sure that you open your files in binary mode (i.e. using the modes wb or rb respectively). Otherwise you might generate corrupt files due to automatic replacement of newline characters with the platform-specific representations.
Let’s take a closer look at what’s going on here.
schema = avro.schema.parse(open("user.avsc", "rb").read())
avro.schema.parse takes a string containing a JSON schema definition as input and outputs a avro.schema.Schema object (specifically a subclass of Schema, in this case RecordSchema). We’re passing in the contents of our user.avsc schema file here.
writer = DataFileWriter(open("users.avro", "wb"), DatumWriter(), schema)
We create a DataFileWriter, which we’ll use to write serialized items to a data file on disk. The DataFileWriter constructor takes three arguments:
- The file we’ll serialize to
- A DatumWriter, which is responsible for actually serializing the items to Avro’s binary format (DatumWriters can be used separately from DataFileWriters, e.g., to perform IPC with Avro).
- The schema we’re using. The DataFileWriter needs the schema both to write the schema to the data file, and to verify that the items we write are valid items and write the appropriate fields.
writer.append({"name": "Alyssa", "favorite_number": 256})
writer.append({"name": "Ben", "favorite_number": 7, "favorite_color": "red"})
We use DataFileWriter.append to add items to our data file. Avro records are represented as Python dicts. Since the field favorite_color has type [“int”, “null”], we are not required to specify this field, as shown in the first append. Were we to omit the required name field, an exception would be raised. Any extra entries not corresponding to a field are present in the dict are ignored.
reader = DataFileReader(open("users.avro", "rb"), DatumReader())
We open the file again, this time for reading back from disk. We use a DataFileReader and DatumReader analagous to the DataFileWriter and DatumWriter above.
for user in reader:
print user
The DataFileReader is an iterator that returns dicts corresponding to the serialized items.
3.3 - Specification
Introduction
This document defines Apache Avro. It is intended to be the authoritative specification. Implementations of Avro must adhere to this document.
Schema Declaration
A Schema is represented in JSON by one of:
- A JSON string, naming a defined type.
- A JSON object, of the form:
{"type": "typeName" ...attributes...}
where typeName is either a primitive or derived type name, as defined below. Attributes not defined in this document are permitted as metadata, but must not affect the format of serialized data.
- A JSON array, representing a union of embedded types.
Primitive Types
The set of primitive type names is:
- null: no value
- boolean: a binary value
- int: 32-bit signed integer
- long: 64-bit signed integer
- float: single precision (32-bit) IEEE 754 floating-point number
- double: double precision (64-bit) IEEE 754 floating-point number
- bytes: sequence of 8-bit unsigned bytes
- string: unicode character sequence
Primitive types have no specified attributes.
Primitive type names are also defined type names. Thus, for example, the schema “string” is equivalent to:
Complex Types
Avro supports six kinds of complex types: records, enums, arrays, maps, unions and fixed.
Records
Records use the type name “record” and support the following attributes:
- name: a JSON string providing the name of the record (required).
- namespace, a JSON string that qualifies the name;
- doc: a JSON string providing documentation to the user of this schema (optional).
- aliases: a JSON array of strings, providing alternate names for this record (optional).
- fields: a JSON array, listing fields (required). Each field is a JSON object with the following attributes:
- name: a JSON string providing the name of the field (required), and
- doc: a JSON string describing this field for users (optional).
- type: a schema, as defined above
- default: A default value for this field, only used when reading instances that lack the field for schema evolution purposes. The presence of a default value does not make the field optional at encoding time. 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. Default values for bytes and fixed fields are JSON strings, where Unicode code points 0-255 are mapped to unsigned 8-bit byte values 0-255. Avro encodes a field even if its value is equal to its default.
field default values
| avro type |
json type |
example |
| null |
null |
null |
| boolean |
boolean |
true |
| int,long |
integer |
1 |
| float,double |
number |
1.1 |
| bytes |
string |
"\u00FF" |
| string |
string |
"foo" |
| record |
object |
{"a": 1} |
| enum |
string |
"FOO" |
| array |
array |
[1] |
| map |
object |
{"a": 1} |
| fixed |
string |
"\u00ff" |
- order: specifies how this field impacts sort ordering of this record (optional). Valid values are “ascending” (the default), “descending”, or “ignore”. For more details on how this is used, see the sort order section below.
- aliases: a JSON array of strings, providing alternate names for this field (optional).
For example, a linked-list of 64-bit values may be defined with:
{
"type": "record",
"name": "LongList",
"aliases": ["LinkedLongs"], // old name for this
"fields" : [
{"name": "value", "type": "long"}, // each element has a long
{"name": "next", "type": ["null", "LongList"]} // optional next element
]
}
Enums
Enums use the type name “enum” and support the following attributes:
- name: a JSON string providing the name of the enum (required).
- namespace, a JSON string that qualifies the name ;
- aliases: a JSON array of strings, providing alternate names for this enum (optional).
- doc: a JSON string providing documentation to the user of this schema (optional).
- symbols: a JSON array, listing symbols, as JSON strings (required). All symbols in an enum must be unique; duplicates are prohibited. Every symbol must match the regular expression [A-Za-z_][A-Za-z0-9_]* (the same requirement as for names).
- default: A default value for this enumeration, used during resolution when the reader encounters a symbol from the writer that isn’t defined in the reader’s schema (optional). The value provided here must be a JSON string that’s a member of the symbols array. See documentation on schema resolution for how this gets used.
For example, playing card suits might be defined with:
{
"type": "enum",
"name": "Suit",
"symbols" : ["SPADES", "HEARTS", "DIAMONDS", "CLUBS"]
}
Arrays
Arrays use the type name “array” and support a single attribute:
- items: the schema of the array’s items.
For example, an array of strings is declared with:
{
"type": "array",
"items" : "string",
"default": []
}
Maps
Maps use the type name “map” and support one attribute:
- values: the schema of the map’s values.
Map keys are assumed to be strings.
For example, a map from string to long is declared with:
{
"type": "map",
"values" : "long",
"default": {}
}
Unions
Unions, as mentioned above, are represented using JSON arrays. For example, ["null", "string"] declares a schema which may be either a null or string.
(Note that when a default value is specified for a record field whose type is a union, the type of the default value must match the first element of the union. Thus, for unions containing “null”, the “null” is usually listed first, since the default value of such unions is typically null.)
Unions may not contain more than one schema with the same type, except for the named types record, fixed and enum. For example, unions containing two array types or two map types are not permitted, but two types with different names are permitted. (Names permit efficient resolution when reading and writing unions.)
Unions may not immediately contain other unions.
Fixed
Fixed uses the type name “fixed” and supports the following attributes:
- name: a string naming this fixed (required).
- namespace, a string that qualifies the name ;
- aliases: a JSON array of strings, providing alternate names for this enum (optional).
- doc: a JSON string providing documentation to the user of this schema (optional).
- size: an integer, specifying the number of bytes per value (required).
For example, 16-byte quantity may be declared with:
{"type": "fixed", "size": 16, "name": "md5"}
Names
Record, enums and fixed are named types. Each has a fullname that is composed of two parts; a name and a namespace. Equality of names is defined on the fullname.
The name portion of a fullname, record field names, and enum symbols must:
- start with [A-Za-z_]
- subsequently contain only [A-Za-z0-9_]
A namespace is a dot-separated sequence of such names. The empty string may also be used as a namespace to indicate the null namespace. Equality of names (including field names and enum symbols) as well as fullnames is case-sensitive.
The null namespace may not be used in a dot-separated sequence of names. So the grammar for a namespace is:
<empty> | <name>[(<dot><name>)*]
In record, enum and fixed definitions, the fullname is determined in one of the following ways:
A name and namespace are both specified. For example, one might use “name”: “X”, “namespace”: “org.foo” to indicate the fullname org.foo.X.
- A fullname is specified. If the name specified contains a dot, then it is assumed to be a fullname, and any namespace also specified is ignored. For example, use “name”: “org.foo.X” to indicate the fullname org.foo.X.
- A name only is specified, i.e., a name that contains no dots. In this case the namespace is taken from the most tightly enclosing schema or protocol. For example, if “name”: “X” is specified, and this occurs within a field of the record definition of org.foo.Y, then the fullname is org.foo.X. If there is no enclosing namespace then the null namespace is used.
References to previously defined names are as in the latter two cases above: if they contain a dot they are a fullname, if they do not contain a dot, the namespace is the namespace of the enclosing definition.
Primitive type names have no namespace and their names may not be defined in any namespace.
A schema or protocol may not contain multiple definitions of a fullname. Further, a name must be defined before it is used (“before” in the depth-first, left-to-right traversal of the JSON parse tree, where the types attribute of a protocol is always deemed to come “before” the messages attribute.)
Aliases
Named types and fields may have aliases. An implementation may optionally use aliases to map a writer’s schema to the reader’s. This facilitates both schema evolution as well as processing disparate datasets.
Aliases function by re-writing the writer’s schema using aliases from the reader’s schema. For example, if the writer’s schema was named “Foo” and the reader’s schema is named “Bar” and has an alias of “Foo”, then the implementation would act as though “Foo” were named “Bar” when reading. Similarly, if data was written as a record with a field named “x” and is read as a record with a field named “y” with alias “x”, then the implementation would act as though “x” were named “y” when reading.
A type alias may be specified either as a fully namespace-qualified, or relative to the namespace of the name it is an alias for. For example, if a type named “a.b” has aliases of “c” and “x.y”, then the fully qualified names of its aliases are “a.c” and “x.y”.
Data Serialization and Deserialization
Binary encoded Avro data does not include type information or field names. The benefit is that the serialized data is small, but as a result a schema must always be used in order to read Avro data correctly. The best way to ensure that the schema is structurally identical to the one used to write the data is to use the exact same schema.
Therefore, files or systems that store Avro data should always include the writer’s schema for that data. Avro-based remote procedure call (RPC) systems must also guarantee that remote recipients of data have a copy of the schema used to write that data. In general, it is advisable that any reader of Avro data should use a schema that is the same (as defined more fully in Parsing Canonical Form for Schemas) as the schema that was used to write the data in order to deserialize it correctly. Deserializing data into a newer schema is accomplished by specifying an additional schema, the results of which are described in Schema Resolution.
In general, both serialization and deserialization proceed as a depth-first, left-to-right traversal of the schema, serializing or deserializing primitive types as they are encountered. Therefore, it is possible, though not advisable, to read Avro data with a schema that does not have the same Parsing Canonical Form as the schema with which the data was written. In order for this to work, the serialized primitive values must be compatible, in order value by value, with the items in the deserialization schema. For example, int and long are always serialized the same way, so an int could be deserialized as a long. Since the compatibility of two schemas depends on both the data and the serialization format (eg. binary is more permissive than JSON because JSON includes field names, eg. a long that is too large will overflow an int), it is simpler and more reliable to use schemas with identical Parsing Canonical Form.
Encodings
Avro specifies two serialization encodings: binary and JSON. Most applications will use the binary encoding, as it is smaller and faster. But, for debugging and web-based applications, the JSON encoding may sometimes be appropriate.
Binary Encoding
Binary encoding does not include field names, self-contained information about the types of individual bytes, nor field or record separators. Therefore readers are wholly reliant on the schema used when the data was encoded.
Primitive Types
Primitive types are encoded in binary as follows:
- null is written as zero bytes.
- a boolean is written as a single byte whose value is either 0 (false) or 1 (true).
- int and long values are written using variable-length zig-zag coding. Some examples:
| value |
hex |
| 0 |
00 |
| -1 |
01 |
| 1 |
02 |
| -2 |
03 |
| 2 |
04 |
| … |
|
| -64 |
7f |
| 64 |
80 01 |
| … |
|
- a float is written as 4 bytes. The float is converted into a 32-bit integer using a method equivalent to Java’s floatToIntBits and then encoded in little-endian format.
- a double is written as 8 bytes. The double is converted into a 64-bit integer using a method equivalent to Java’s doubleToLongBits and then encoded in little-endian format.
- bytes are encoded as a long followed by that many bytes of data.
- a string is encoded as a long followed by that many bytes of UTF-8 encoded character data.
For example, the three-character string “foo” would be encoded as the long value 3 (encoded as hex 06) followed by the UTF-8 encoding of ‘f’, ‘o’, and ‘o’ (the hex bytes 66 6f 6f):
06 66 6f 6f
Complex Types
Complex types are encoded in binary as follows:
Records
A record is encoded by encoding the values of its fields in the order that they are declared. In other words, a record is encoded as just the concatenation of the encodings of its fields. Field values are encoded per their schema.
For example, the record schema
{
"type": "record",
"name": "test",
"fields" : [
{"name": "a", "type": "long"},
{"name": "b", "type": "string"}
]
}
An instance of this record whose a field has value 27 (encoded as hex 36) and whose b field has value “foo” (encoded as hex bytes 06 66 6f 6f), would be encoded simply as the concatenation of these, namely the hex byte sequence:
36 06 66 6f 6f
Enums
An enum is encoded by a int, representing the zero-based position of the symbol in the schema.
For example, consider the enum:
{"type": "enum", "name": "Foo", "symbols": ["A", "B", "C", "D"] }
This would be encoded by an int between zero and three, with zero indicating “A”, and 3 indicating “D”.
Arrays
Arrays are encoded as a series of blocks. Each block consists of a long count value, followed by that many array items. A block with count zero indicates the end of the array. Each item is encoded per the array’s item schema.
If a block’s count is negative, its absolute value is used, and the count is followed immediately by a long block size indicating the number of bytes in the block. This block size permits fast skipping through data, e.g., when projecting a record to a subset of its fields.
For example, the array schema
{"type": "array", "items": "long"}
an array containing the items 3 and 27 could be encoded as the long value 2 (encoded as hex 04) followed by long values 3 and 27 (encoded as hex 06 36) terminated by zero:
04 06 36 00
The blocked representation permits one to read and write arrays larger than can be buffered in memory, since one can start writing items without knowing the full length of the array.
Maps
Maps are encoded as a series of blocks. Each block consists of a long count value, followed by that many key/value pairs. A block with count zero indicates the end of the map. Each item is encoded per the map’s value schema.
If a block’s count is negative, its absolute value is used, and the count is followed immediately by a long block size indicating the number of bytes in the block. This block size permits fast skipping through data, e.g., when projecting a record to a subset of its fields.
The blocked representation permits one to read and write maps larger than can be buffered in memory, since one can start writing items without knowing the full length of the map.
Unions
A union is encoded by first writing an int value indicating the zero-based position within the union of the schema of its value. The value is then encoded per the indicated schema within the union.
For example, the union schema ["null","string"] would encode:
-
null as zero (the index of “null” in the union):
00
-
the string “a” as one (the index of “string” in the union, 1, encoded as hex 02), followed by the serialized string:
02 02 61
NOTE: Currently for C/C++ implementations, the positions are practically an int, but theoretically a long. In reality, we don’t expect unions with 215M members
Fixed
Fixed instances are encoded using the number of bytes declared in the schema.
JSON Encoding
Except for unions, the JSON encoding is the same as is used to encode field default values.
The value of a union is encoded in JSON as follows:
- if its type is null, then it is encoded as a JSON null;
- otherwise it is encoded as a JSON object with one name/value pair whose name is the type’s name and whose value is the recursively encoded value. For Avro’s named types (record, fixed or enum) the user-specified name is used, for other types the type name is used.
For example, the union schema ["null","string","Foo"], where Foo is a record name, would encode:
- null as null;
- the string “a” as
{"string": "a"} and
- a Foo instance as
{"Foo": {...}}, where {...} indicates the JSON encoding of a Foo instance.
Note that the original schema is still required to correctly process JSON-encoded data. For example, the JSON encoding does not distinguish between int and long, float and double, records and maps, enums and strings, etc.
Single-object encoding
In some situations a single Avro serialized object is to be stored for a longer period of time. One very common example is storing Avro records for several weeks in an Apache Kafka topic.
In the period after a schema change this persistence system will contain records that have been written with different schemas. So the need arises to know which schema was used to write a record to support schema evolution correctly. In most cases the schema itself is too large to include in the message, so this binary wrapper format supports the use case more effectively.
Single object encoding specification
Single Avro objects are encoded as follows:
- A two-byte marker,
C3 01, to show that the message is Avro and uses this single-record format (version 1).
- The 8-byte little-endian CRC-64-AVRO fingerprint of the object’s schema.
- The Avro object encoded using Avro’s binary encoding.
Implementations use the 2-byte marker to determine whether a payload is Avro. This check helps avoid expensive lookups that resolve the schema from a fingerprint, when the message is not an encoded Avro payload.
Sort Order
Avro defines a standard sort order for data. This permits data written by one system to be efficiently sorted by another system. This can be an important optimization, as sort order comparisons are sometimes the most frequent per-object operation. Note also that Avro binary-encoded data can be efficiently ordered without deserializing it to objects.
Data items may only be compared if they have identical schemas. Pairwise comparisons are implemented recursively with a depth-first, left-to-right traversal of the schema. The first mismatch encountered determines the order of the items.
Two items with the same schema are compared according to the following rules.
- null data is always equal.
- boolean data is ordered with false before true.
- int, long, float and double data is ordered by ascending numeric value.
- bytes and fixed data are compared lexicographically by unsigned 8-bit values.
- string data is compared lexicographically by Unicode code point. Note that since UTF-8 is used as the binary encoding for strings, sorting of bytes and string binary data is identical.
- array data is compared lexicographically by element.
- enum data is ordered by the symbol’s position in the enum schema. For example, an enum whose symbols are
["z", "a"] would sort “z” values before “a” values.
- union data is first ordered by the branch within the union, and, within that, by the type of the branch. For example, an
["int", "string"] union would order all int values before all string values, with the ints and strings themselves ordered as defined above.
- record data is ordered lexicographically by field. If a field specifies that its order is:
- “ascending”, then the order of its values is unaltered.
- “descending”, then the order of its values is reversed.
- “ignore”, then its values are ignored when sorting.
- map data may not be compared. It is an error to attempt to compare data containing maps unless those maps are in an
"order":"ignore" record field.
Object Container Files
Avro includes a simple object container file format. A file has a schema, and all objects stored in the file must be written according to that schema, using binary encoding. Objects are stored in blocks that may be compressed. Syncronization markers are used between blocks to permit efficient splitting of files for MapReduce processing.
Files may include arbitrary user-specified metadata.
A file consists of:
- A file header, followed by
- one or more file data blocks.
A file header consists of:
- Four bytes, ASCII ‘O’, ‘b’, ‘j’, followed by 1.
- file metadata, including the schema.
- The 16-byte, randomly-generated sync marker for this file.
File metadata is written as if defined by the following map schema:
{"type": "map", "values": "bytes"}
All metadata properties that start with “avro.” are reserved. The following file metadata properties are currently used:
- avro.schema contains the schema of objects stored in the file, as JSON data (required).
- avro.codec the name of the compression codec used to compress blocks, as a string. Implementations are required to support the following codecs: “null” and “deflate”. If codec is absent, it is assumed to be “null”. The codecs are described with more detail below.
A file header is thus described by the following schema:
{"type": "record", "name": "org.apache.avro.file.Header",
"fields" : [
{"name": "magic", "type": {"type": "fixed", "name": "Magic", "size": 4}},
{"name": "meta", "type": {"type": "map", "values": "bytes"}},
{"name": "sync", "type": {"type": "fixed", "name": "Sync", "size": 16}}
]
}
A file data block consists of:
- A long indicating the count of objects in this block.
- A long indicating the size in bytes of the serialized objects in the current block, after any codec is applied
- The serialized objects. If a codec is specified, this is compressed by that codec.
- The file’s 16-byte sync marker.
Thus, each block’s binary data can be efficiently extracted or skipped without deserializing the contents. The combination of block size, object counts, and sync markers enable detection of corrupt blocks and help ensure data integrity.
Required Codecs
null
The “null” codec simply passes through data uncompressed.
deflate
The “deflate” codec writes the data block using the deflate algorithm as specified in RFC 1951, and typically implemented using the zlib library. Note that this format (unlike the “zlib format” in RFC 1950) does not have a checksum.
Optional Codecs
bzip2
The “bzip2” codec uses the bzip2 compression library.
snappy
The “snappy” codec uses Google’s Snappy compression library. Each compressed block is followed by the 4-byte, big-endian CRC32 checksum of the uncompressed data in the block.
xz
The “xz” codec uses the XZ compression library.
zstandard
The “zstandard” codec uses Facebook’s Zstandard compression library.
Protocol Declaration
Avro protocols describe RPC interfaces. Like schemas, they are defined with JSON text.
A protocol is a JSON object with the following attributes:
- protocol, a string, the name of the protocol (required);
- namespace, an optional string that qualifies the name;
- doc, an optional string describing this protocol;
- types, an optional list of definitions of named types (records, enums, fixed and errors). An error definition is just like a record definition except it uses “error” instead of “record”. Note that forward references to named types are not permitted.
- messages, an optional JSON object whose keys are message names and whose values are objects whose attributes are described below. No two messages may have the same name.
The name and namespace qualification rules defined for schema objects apply to protocols as well.
Messages
A message has attributes:
- a doc, an optional description of the message,
- a request, a list of named, typed parameter schemas (this has the same form as the fields of a record declaration);
- a response schema;
- an optional union of declared error schemas. The effective union has “string” prepended to the declared union, to permit transmission of undeclared “system” errors. For example, if the declared error union is
["AccessError"], then the effective union is ["string", "AccessError"]. When no errors are declared, the effective error union is ["string"]. Errors are serialized using the effective union; however, a protocol’s JSON declaration contains only the declared union.
- an optional one-way boolean parameter.
A request parameter list is processed equivalently to an anonymous record. Since record field lists may vary between reader and writer, request parameters may also differ between the caller and responder, and such differences are resolved in the same manner as record field differences.
The one-way parameter may only be true when the response type is "null" and no errors are listed.
Sample Protocol
For example, one may define a simple HelloWorld protocol with:
{
"namespace": "com.acme",
"protocol": "HelloWorld",
"doc": "Protocol Greetings",
"types": [
{"name": "Greeting", "type": "record", "fields": [
{"name": "message", "type": "string"}]},
{"name": "Curse", "type": "error", "fields": [
{"name": "message", "type": "string"}]}
],
"messages": {
"hello": {
"doc": "Say hello.",
"request": [{"name": "greeting", "type": "Greeting" }],
"response": "Greeting",
"errors": ["Curse"]
}
}
}
Message Transport
Messages may be transmitted via different transport mechanisms.
To the transport, a message is an opaque byte sequence.
A transport is a system that supports:
- transmission of request messages
- receipt of corresponding response messages
Servers may send a response message back to the client corresponding to a request message. The mechanism of correspondence is transport-specific. For example, in HTTP it is implicit, since HTTP directly supports requests and responses. But a transport that multiplexes many client threads over a single socket would need to tag messages with unique identifiers.
Transports may be either stateless or stateful. In a stateless transport, messaging assumes no established connection state, while stateful transports establish connections that may be used for multiple messages. This distinction is discussed further in the handshake section below.
HTTP as Transport
When HTTP is used as a transport, each Avro message exchange is an HTTP request/response pair. All messages of an Avro protocol should share a single URL at an HTTP server. Other protocols may also use that URL. Both normal and error Avro response messages should use the 200 (OK) response code. The chunked encoding may be used for requests and responses, but, regardless the Avro request and response are the entire content of an HTTP request and response. The HTTP Content-Type of requests and responses should be specified as “avro/binary”. Requests should be made using the POST method.
HTTP is used by Avro as a stateless transport.
Message Framing
Avro messages are framed as a list of buffers.
Framing is a layer between messages and the transport. It exists to optimize certain operations.
The format of framed message data is:
- a series of buffers, where each buffer consists of:
- a four-byte, big-endian buffer length, followed by
- that many bytes of buffer data.
- a message is always terminated by a zero-length buffer.
Framing is transparent to request and response message formats (described below). Any message may be presented as a single or multiple buffers.
Framing can permit readers to more efficiently get different buffers from different sources and for writers to more efficiently store different buffers to different destinations. In particular, it can reduce the number of times large binary objects are copied. For example, if an RPC parameter consists of a megabyte of file data, that data can be copied directly to a socket from a file descriptor, and, on the other end, it could be written directly to a file descriptor, never entering user space.
A simple, recommended, framing policy is for writers to create a new segment whenever a single binary object is written that is larger than a normal output buffer. Small objects are then appended in buffers, while larger objects are written as their own buffers. When a reader then tries to read a large object the runtime can hand it an entire buffer directly, without having to copy it.
Handshake
The purpose of the handshake is to ensure that the client and the server have each other’s protocol definition, so that the client can correctly deserialize responses, and the server can correctly deserialize requests. Both clients and servers should maintain a cache of recently seen protocols, so that, in most cases, a handshake will be completed without extra round-trip network exchanges or the transmission of full protocol text.
RPC requests and responses may not be processed until a handshake has been completed. With a stateless transport, all requests and responses are prefixed by handshakes. With a stateful transport, handshakes are only attached to requests and responses until a successful handshake response has been returned over a connection. After this, request and response payloads are sent without handshakes for the lifetime of that connection.
The handshake process uses the following record schemas:
{
"type": "record",
"name": "HandshakeRequest", "namespace":"org.apache.avro.ipc",
"fields": [
{"name": "clientHash",
"type": {"type": "fixed", "name": "MD5", "size": 16}},
{"name": "clientProtocol", "type": ["null", "string"]},
{"name": "serverHash", "type": "MD5"},
{"name": "meta", "type": ["null", {"type": "map", "values": "bytes"}]}
]
}
{
"type": "record",
"name": "HandshakeResponse", "namespace": "org.apache.avro.ipc",
"fields": [
{"name": "match",
"type": {"type": "enum", "name": "HandshakeMatch",
"symbols": ["BOTH", "CLIENT", "NONE"]}},
{"name": "serverProtocol",
"type": ["null", "string"]},
{"name": "serverHash",
"type": ["null", {"type": "fixed", "name": "MD5", "size": 16}]},
{"name": "meta",
"type": ["null", {"type": "map", "values": "bytes"}]}
]
}
- A client first prefixes each request with a
HandshakeRequest containing just the hash of its protocol and of the server’s protocol (clientHash!=null, clientProtocol=null, serverHash!=null), where the hashes are 128-bit MD5 hashes of the JSON protocol text. If a client has never connected to a given server, it sends its hash as a guess of the server’s hash, otherwise it sends the hash that it previously obtained from this server.
The server responds with a HandshakeResponse containing one of:
-
match=BOTH, serverProtocol=null, serverHash=null if the client sent the valid hash of the server’s protocol and the server knows what protocol corresponds to the client’s hash. In this case, the request is complete and the response data immediately follows the HandshakeResponse.
-
match=CLIENT, serverProtocol!=null, serverHash!=null if the server has previously seen the client’s protocol, but the client sent an incorrect hash of the server’s protocol. The request is complete and the response data immediately follows the HandshakeResponse. The client must use the returned protocol to process the response and should also cache that protocol and its hash for future interactions with this server.
-
match=NONE if the server has not previously seen the client’s protocol. The serverHash and serverProtocol may also be non-null if the server’s protocol hash was incorrect.
In this case the client must then re-submit its request with its protocol text (clientHash!=null, clientProtocol!=null, serverHash!=null) and the server should respond with a successful match (match=BOTH, serverProtocol=null, serverHash=null) as above.
The meta field is reserved for future handshake enhancements.
A call consists of a request message paired with its resulting response or error message. Requests and responses contain extensible metadata, and both kinds of messages are framed as described above.
The format of a call request is:
- request metadata, a map with values of type bytes
- the message name, an Avro string, followed by
- the message parameters. Parameters are serialized according to the message’s request declaration.
When the empty string is used as a message name a server should ignore the parameters and return an empty response. A client may use this to ping a server or to perform a handshake without sending a protocol message.
When a message is declared one-way and a stateful connection has been established by a successful handshake response, no response data is sent. Otherwise the format of the call response is:
- response metadata, a map with values of type bytes
- a one-byte error flag boolean, followed by either:
- if the error flag is false, the message response, serialized per the message’s response schema.
- if the error flag is true, the error, serialized per the message’s effective error union schema.
Schema Resolution
A reader of Avro data, whether from an RPC or a file, can always parse that data because the original schema must be provided along with the data. However, the reader may be programmed to read data into a different schema. For example, if the data was written with a different version of the software than it is read, then fields may have been added or removed from records. This section specifies how such schema differences should be resolved.
We refer to the schema used to write the data as the writer’s schema, and the schema that the application expects the reader’s schema. Differences between these should be resolved as follows:
- It is an error if the two schemas do not match.
To match, one of the following must hold:
-
both schemas are arrays whose item types match
-
both schemas are maps whose value types match
-
both schemas are enums whose (unqualified) names match
-
both schemas are fixed whose sizes and (unqualified) names match
-
both schemas are records with the same (unqualified) name
-
either schema is a union
-
both schemas have same primitive type
-
the writer’s schema may be promoted to the reader’s as follows:
- int is promotable to long, float, or double
- long is promotable to float or double
- float is promotable to double
- string is promotable to bytes
- bytes is promotable to string
-
if both are records:
- the ordering of fields may be different: fields are matched by name.
- schemas for fields with the same name in both records are resolved recursively.
- if the writer’s record contains a field with a name not present in the reader’s record, the writer’s value for that field is ignored.
- if the reader’s record schema has a field that contains a default value, and writer’s schema does not have a field with the same name, then the reader should use the default value from its field.
- if the reader’s record schema has a field with no default value, and writer’s schema does not have a field with the same name, an error is signalled.
-
if both are enums:
if the writer’s symbol is not present in the reader’s enum and the reader has a default value, then that value is used, otherwise an error is signalled.
-
if both are arrays:
This resolution algorithm is applied recursively to the reader’s and writer’s array item schemas.
-
if both are maps:
This resolution algorithm is applied recursively to the reader’s and writer’s value schemas.
-
if both are unions:
The first schema in the reader’s union that matches the selected writer’s union schema is recursively resolved against it. if none match, an error is signalled.
-
if reader’s is a union, but writer’s is not
The first schema in the reader’s union that matches the writer’s schema is recursively resolved against it. If none match, an error is signalled.
-
if writer’s is a union, but reader’s is not
If the reader’s schema matches the selected writer’s schema, it is recursively resolved against it. If they do not match, an error is signalled.
A schema’s doc fields are ignored for the purposes of schema resolution. Hence, the doc portion of a schema may be dropped at serialization.
One of the defining characteristics of Avro is that a reader must use the schema used by the writer of the data in order to know how to read the data. This assumption results in a data format that’s compact and also amenable to many forms of schema evolution. However, the specification so far has not defined what it means for the reader to have the “same” schema as the writer. Does the schema need to be textually identical? Well, clearly adding or removing some whitespace to a JSON expression does not change its meaning. At the same time, reordering the fields of records clearly does change the meaning. So what does it mean for a reader to have “the same” schema as a writer?
Parsing Canonical Form is a transformation of a writer’s schema that let’s us define what it means for two schemas to be “the same” for the purpose of reading data written against the schema. It is called Parsing Canonical Form because the transformations strip away parts of the schema, like “doc” attributes, that are irrelevant to readers trying to parse incoming data. It is called Canonical Form because the transformations normalize the JSON text (such as the order of attributes) in a way that eliminates unimportant differences between schemas. If the Parsing Canonical Forms of two different schemas are textually equal, then those schemas are “the same” as far as any reader is concerned, i.e., there is no serialized data that would allow a reader to distinguish data generated by a writer using one of the original schemas from data generated by a writing using the other original schema. (We sketch a proof of this property in a companion document.)
The next subsection specifies the transformations that define Parsing Canonical Form. But with a well-defined canonical form, it can be convenient to go one step further, transforming these canonical forms into simple integers (“fingerprints”) that can be used to uniquely identify schemas. The subsection after next recommends some standard practices for generating such fingerprints.
Assuming an input schema (in JSON form) that’s already UTF-8 text for a valid Avro schema (including all quotes as required by JSON), the following transformations will produce its Parsing Canonical Form:
- [PRIMITIVES] Convert primitive schemas to their simple form (e.g., int instead of
{"type":"int"}).
- [FULLNAMES] Replace short names with fullnames, using applicable namespaces to do so. Then eliminate namespace attributes, which are now redundant.
- [STRIP] Keep only attributes that are relevant to parsing data, which are: type, name, fields, symbols, items, values, size. Strip all others (e.g., doc and aliases).
- [ORDER] Order the appearance of fields of JSON objects as follows: name, type, fields, symbols, items, values, size. For example, if an object has type, name, and size fields, then the name field should appear first, followed by the type and then the size fields.
- [STRINGS] For all JSON string literals in the schema text, replace any escaped characters (e.g., \uXXXX escapes) with their UTF-8 equivalents.
- [INTEGERS] Eliminate quotes around and any leading zeros in front of JSON integer literals (which appear in the size attributes of fixed schemas).
- [WHITESPACE] Eliminate all whitespace in JSON outside of string literals.
Schema Fingerprints
“[A] fingerprinting algorithm is a procedure that maps an arbitrarily large data item (such as a computer file) to a much shorter bit string, its fingerprint, that uniquely identifies the original data for all practical purposes” (quoted from Wikipedia). In the Avro context, fingerprints of Parsing Canonical Form can be useful in a number of applications; for example, to cache encoder and decoder objects, to tag data items with a short substitute for the writer’s full schema, and to quickly negotiate common-case schemas between readers and writers.
In designing fingerprinting algorithms, there is a fundamental trade-off between the length of the fingerprint and the probability of collisions. To help application designers find appropriate points within this trade-off space, while encouraging interoperability and ease of implementation, we recommend using one of the following three algorithms when fingerprinting Avro schemas:
- When applications can tolerate longer fingerprints, we recommend using the SHA-256 digest algorithm to generate 256-bit fingerprints of Parsing Canonical Forms. Most languages today have SHA-256 implementations in their libraries.
- At the opposite extreme, the smallest fingerprint we recommend is a 64-bit Rabin fingerprint. Below, we provide pseudo-code for this algorithm that can be easily translated into any programming language. 64-bit fingerprints should guarantee uniqueness for schema caches of up to a million entries (for such a cache, the chance of a collision is 3E-8). We don’t recommend shorter fingerprints, as the chances of collisions is too great (for example, with 32-bit fingerprints, a cache with as few as 100,000 schemas has a 50% chance of having a collision).
- Between these two extremes, we recommend using the MD5 message digest to generate 128-bit fingerprints. These make sense only where very large numbers of schemas are being manipulated (tens of millions); otherwise, 64-bit fingerprints should be sufficient. As with SHA-256, MD5 implementations are found in most libraries today.
These fingerprints are not meant to provide any security guarantees, even the longer SHA-256-based ones. Most Avro applications should be surrounded by security measures that prevent attackers from writing random data and otherwise interfering with the consumers of schemas. We recommend that these surrounding mechanisms be used to prevent collision and pre-image attacks (i.e., “forgery”) on schema fingerprints, rather than relying on the security properties of the fingerprints themselves.
Rabin fingerprints are cyclic redundancy checks computed using irreducible polynomials. In the style of the Appendix of RFC 1952 (pg 10), which defines the CRC-32 algorithm, here’s our definition of the 64-bit AVRO fingerprinting algorithm:
long fingerprint64(byte[] buf) {
if (FP_TABLE == null) initFPTable();
long fp = EMPTY;
for (int i = 0; i < buf.length; i++)
fp = (fp >>> 8) ^ FP_TABLE[(int)(fp ^ buf[i]) & 0xff];
return fp;
}
static long EMPTY = 0xc15d213aa4d7a795L;
static long[] FP_TABLE = null;
void initFPTable() {
FP_TABLE = new long[256];
for (int i = 0; i < 256; i++) {
long fp = i;
for (int j = 0; j < 8; j++)
fp = (fp >>> 1) ^ (EMPTY & -(fp & 1L));
FP_TABLE[i] = fp;
}
}
Readers interested in the mathematics behind this algorithm may want to read Chapter 14 of the Second Edition of Hacker’s Delight. (Unlike RFC-1952 and the book chapter, we prepend a single one bit to messages. We do this because CRCs ignore leading zero bits, which can be problematic. Our code prepends a one-bit by initializing fingerprints using EMPTY, rather than initializing using zero as in RFC-1952 and the book chapter.)
Logical Types
A logical type is an Avro primitive or complex type with extra attributes to represent a derived type. The attribute logicalType must always be present for a logical type, and is a string with the name of one of the logical types listed later in this section. Other attributes may be defined for particular logical types.
A logical type is always serialized using its underlying Avro type so that values are encoded in exactly the same way as the equivalent Avro type that does not have a logicalType attribute. Language implementations may choose to represent logical types with an appropriate native type, although this is not required.
Language implementations must ignore unknown logical types when reading, and should use the underlying Avro type. If a logical type is invalid, for example a decimal with scale greater than its precision, then implementations should ignore the logical type and use the underlying Avro type.
Decimal
The decimal logical type represents an arbitrary-precision signed decimal number of the form unscaled × 10-scale.
A decimal logical type annotates Avro bytes or fixed types. The byte array must contain the two’s-complement representation of the unscaled integer value in big-endian byte order. The scale is fixed, and is specified using an attribute.
The following attributes are supported:
- scale, a JSON integer representing the scale (optional). If not specified the scale is 0.
- precision, a JSON integer representing the (maximum) precision of decimals stored in this type (required).
For example, the following schema represents decimal numbers with a maximum precision of 4 and a scale of 2:
{
"type": "bytes",
"logicalType": "decimal",
"precision": 4,
"scale": 2
}
Precision must be a positive integer greater than zero. If the underlying type is a fixed, then the precision is limited by its size. An array of length n can store at most floor(log10(28 × n - 1 - 1)) base-10 digits of precision.
Scale must be zero or a positive integer less than or equal to the precision.
For the purposes of schema resolution, two schemas that are decimal logical types match if their scales and precisions match.
UUID
The uuid logical type represents a random generated universally unique identifier (UUID).
A uuid logical type annotates an Avro string. The string has to conform with RFC-4122
Date
The date logical type represents a date within the calendar, with no reference to a particular time zone or time of day.
A date logical type annotates an Avro int, where the int stores the number of days from the unix epoch, 1 January 1970 (ISO calendar).
The following schema represents a date:
{
"type": "int",
"logicalType": "date"
}
Time (millisecond precision)
The time-millis logical type represents a time of day, with no reference to a particular calendar, time zone or date, with a precision of one millisecond.
A time-millis logical type annotates an Avro int, where the int stores the number of milliseconds after midnight, 00:00:00.000.
Time (microsecond precision)
The time-micros logical type represents a time of day, with no reference to a particular calendar, time zone or date, with a precision of one microsecond.
A time-micros logical type annotates an Avro long, where the long stores the number of microseconds after midnight, 00:00:00.000000.
Timestamp (millisecond precision)
The timestamp-millis logical type represents an instant on the global timeline, independent of a particular time zone or calendar, with a precision of one millisecond. Please note that time zone information gets lost in this process. Upon reading a value back, we can only reconstruct the instant, but not the original representation. In practice, such timestamps are typically displayed to users in their local time zones, therefore they may be displayed differently depending on the execution environment.
A timestamp-millis logical type annotates an Avro long, where the long stores the number of milliseconds from the unix epoch, 1 January 1970 00:00:00.000 UTC.
Timestamp (microsecond precision)
The timestamp-micros logical type represents an instant on the global timeline, independent of a particular time zone or calendar, with a precision of one microsecond. Please note that time zone information gets lost in this process. Upon reading a value back, we can only reconstruct the instant, but not the original representation. In practice, such timestamps are typically displayed to users in their local time zones, therefore they may be displayed differently depending on the execution environment.
A timestamp-micros logical type annotates an Avro long, where the long stores the number of microseconds from the unix epoch, 1 January 1970 00:00:00.000000 UTC.
Local timestamp (millisecond precision)
The local-timestamp-millis logical type represents a timestamp in a local timezone, regardless of what specific time zone is considered local, with a precision of one millisecond.
A local-timestamp-millis logical type annotates an Avro long, where the long stores the number of milliseconds, from 1 January 1970 00:00:00.000.
Local timestamp (microsecond precision)
The local-timestamp-micros logical type represents a timestamp in a local timezone, regardless of what specific time zone is considered local, with a precision of one microsecond.
A local-timestamp-micros logical type annotates an Avro long, where the long stores the number of microseconds, from 1 January 1970 00:00:00.000000.
Duration
The duration logical type represents an amount of time defined by a number of months, days and milliseconds. This is not equivalent to a number of milliseconds, because, depending on the moment in time from which the duration is measured, the number of days in the month and number of milliseconds in a day may differ. Other standard periods such as years, quarters, hours and minutes can be expressed through these basic periods.
A duration logical type annotates Avro fixed type of size 12, which stores three little-endian unsigned integers that represent durations at different granularities of time. The first stores a number in months, the second stores a number in days, and the third stores a number in milliseconds.
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
3.4 - Trevni
Version 0.1
DRAFT
This document is the authoritative specification of a file format. Its intent is to permit compatible, independent implementations that read and/or write files in this format.
Introduction
Data sets are often described as a table composed of rows and columns. Each record in the dataset is considered a row, with each field of the record occupying a different column. Writing records to a file one-by-one as they are created results in a row-major format, like Hadoop’s SequenceFile or Avro data files.
In many cases higher query performance may be achieved if the data is instead organized in a column-major format, where multiple values of a given column are stored adjacently. This document defines such a column-major file format for datasets.
To permit scalable, distributed query evaluation, datasets are partitioned into row groups, containing distinct collections of rows. Each row group is organized in column-major order, while row groups form a row-major partitioning of the entire dataset.
Rationale
Goals
The format is meant satisfy the following goals:
Maximize the size of row groups. Disc drives are used most efficiently when sequentially accessing data. Consider a drive that takes 10ms to seek and transfers at 100MB/second. If a 10-column dataset whose values are all the same size is split into 10MB row groups, then accessing a single column will require a sequence of seek+1MB reads, for a cost of 20ms/MB processed. If the same dataset is split into 100MB row groups then this drops to 11ms/MB processed. This effect is exaggerated for datasets with larger numbers of columns and with columns whose values are smaller than average. So we’d prefer row groups that are 100MB or greater.
Permit random access within a row group. Some queries will first examine one column, and, only when certain relatively rare criteria are met, examine other columns. Rather than iterating through selected columns of the row-group in parallel, one might iterate through one column and randomly access another. This is called support for WHERE clauses, after the SQL operator of that name.
Minimize the number of files per dataset. HDFS is a primary intended deployment platform for these files. The HDFS Namenode requires memory for each file in the filesystem, thus for a format to be HDFS-friendly it should strive to require the minimum number of distinct files.
Support co-location of columns within row-groups. Row groups are the unit of parallel operation on a column dataset. For efficient file i/o, the entirety of a row-group should ideally reside on the host that is evaluating the query in order to avoid network latencies and bottlenecks.
Data integrity. The format should permit applications to detect data corruption. Many file systems may prevent corruption, but files may be moved between filesystems and be subject to corruption at points in that process. It is best if the data in a file can be validated independently.
Extensibility. The format should permit applications to store additional annotations about a datasets in the files, such as type information, origin, etc. Some environments may have metadata stores for such information, but not all do, and files might be moved among systems with different metadata systems. The ability to keep such information within the file simplifies the coordination of such information.
Minimal overhead. The column format should not make datasets appreciably larger. Storage is a primary cost and a choice to use this format should not require additional storage.
Primary format. The column format should be usable as a primary format for datasets, not as an auxiliary, accelerated format. Applications that process a dataset in row-major order should be able to easily consume column files and applications that produce datasets in row-major order should be able to easily generate column files.
Design
To meet these goals we propose the following design.
Each row group is a separate file. All values of a column in a file are written contiguously. This maximizes the row group size, optimizing performance when querying few and small columns.
Each file occupies a single HDFS block. A larger than normal block size may be specified, e.g., ~1GB instead of the typical ~100MB. This guarantees co-location and eliminates network use when query processing can be co-located with the file. This also moderates the memory impact on the HDFS Namenode since no small files are written.
Each column in a file is written as a sequence of ~64kB compressed blocks. The sequence is prefixed by a table describing all of the blocks in the column to permit random access within the column.
Application-specific metadata may be added at the file, column, and block levels.
Checksums are included with each block, providing data integrity.
Discussion
The use of a single block per file achieves the same effect as the custom block placement policy described in the CIF paper, but while still permitting HDFS rebalancing and not increasing the number of files in the namespace.
This section formally describes the proposed column file format.
Data Model
We assume a simple data model, where a record is a set of named fields, and the value of each field is a sequence of untyped bytes. A type system may be layered on top of this, as specified in the Type Mapping section below.
Primitive Values
We define the following primitive value types:
- Signed 64-bit long values are written using a variable-length zig-zag coding, where the high-order bit in each byte determines whether subsequent bytes are present. For example:
decimal value hex bytes
| decimal value |
hex bytes |
|
| 0 |
00 |
|
| -1 |
01 |
|
| 1 |
02 |
|
| … |
|
|
| -64 |
7f |
|
| 64 |
80 01 |
|
| … |
|
|
- bytes are encoded as a long followed by that many bytes of data.
- a string is encoded as a long followed by that many bytes of UTF-8 encoded character data.
For example, the three-character string “foo” would be encoded as the long value 3 (encoded as hex 06) followed by the UTF-8 encoding of ‘f’, ‘o’, and ‘o’ (the hex bytes 66 6f 6f): 06 66 6f 6f
Type Names
The following type names are used to describe column values:
- null, requires zero bytes. Sometimes used in array columns.
- boolean, one bit, packed into bytes, little-endian;
- int, like long, but restricted to 32-bit signed values
- long 64-bit signed values, represented as above
- fixed32 32-bit values stored as four bytes, little-endian.
- fixed64 64-bit values stored as eight bytes, little-endian.
- float 32-bit IEEE floating point value, little-endian
- double 64-bit IEEE floating point value, little-endian
- string as above
- bytes as above, may be used to encapsulate more complex objects
Type names are represented as strings (UTF-8 encoded, length-prefixed).
Metadata consists of:
- A long indicating the number of metadata key/value pairs.
- For each pair, a string key and bytes value.
All metadata properties that start with “trevni.” are reserved.
The following file metadata properties are defined:
- trevni.codec the name of the default compression codec used to compress blocks, as a string. Implementations are required to support the “null” codec. Optional. If absent, it is assumed to be “null”. Codecs are described in more detail below.
- trevni.checksum the name of the checksum algorithm used in this file, as a string. Implementations are required to support the “crc-32” checksum. Optional. If absent, it is assumed to be “null”. Checksums are described in more detail below.
The following column metadata properties are defined:
- trevni.codec the name of the compression codec used to compress the blocks of this column, as a string. Implementations are required to support the “null” codec. Optional. If absent, it is assumed to be “null”. Codecs are described in more detail below.
- trevni.name the name of the column, as a string. Required.
- trevni.type the type of data in the column. One of the type names above. Required.
- trevni.values if present, indicates that the initial value of each block in this column will be stored in the block’s descriptor. Not permitted for array columns or columns that specify a parent.
- trevni.array if present, indicates that each row in this column contains a sequence of values of the named type rather than just a single value. An integer length precedes each sequence of values indicating the count of values in the sequence. If the length is negative then it indicates a sequence of zero or one lengths, where -1 indicates two zeros, -2 two ones, -3 three zeros, -4 three ones, etc.
- trevni.parent if present, the name of an array column whose lengths are also used by this column. Thus values of this column are sequences but no lengths are stored in this column.
For example, consider the following row, as JSON, where all values are primitive types, but one has multiple values.
{"id"=566, "date"=23423234234
"from"="foo@bar.com",
"to"=["bar@baz.com", "bang@foo.com"],
"content"="Hi!"}
The columns for this might be specified as:
name=id type=int
name=date type=long
name=from type=string
name=to type=string array=true
name=content type=string
If a row contains an array of records, e.g. “received” in the following:
{"id"=566, "date"=23423234234
"from"="foo@bar.com",
"to"=["bar@baz.com", "bang@foo.com"],
"content"="Hi!"
"received"=[{"date"=234234234234, "host"="192.168.0.0.1"},
{"date"=234234545645, "host"="192.168.0.0.2"}]
}
Then one can define a parent column followed by a column for each field in the record, adding the following columns:
name=received type=null array=true
name=date type=long parent=received
name=host type=string parent=received
If an array value itself contains an array, e.g. the “sigs” below:
{"id"=566, "date"=23423234234
"from"="foo@bar.com",
"to"=["bar@baz.com", "bang@foo.com"],
"content"="Hi!"
"received"=[{"date"=234234234234, "host"="192.168.0.0.1",
"sigs"=[{"algo"="weak", "value"="0af345de"}]},
{"date"=234234545645, "host"="192.168.0.0.2",
"sigs"=[]}]
}
Then a parent column may be defined that itself has a parent column.
name=sigs type=null array=true parent=received
name=algo type=string parent=sigs
name=value type=string parent=sigs
No block metadata properties are currently defined.
A file consists of:
- A file header, followed by
- one or more columns.
A file header consists of:
- Four bytes, ASCII ‘T’, ‘r’, ‘v’, followed by 0x02.
- a fixed64 indicating the number of rows in the file
- a fixed32 indicating the number of columns in the file
- file metadata.
- for each column, its column metadata
- for each column, its starting position in the file as a fixed64.
A column consists of:
- A fixed32 indicating the number of blocks in this column.
- For each block, a block descriptor
- One or more blocks.
A block descriptor consists of:
- A fixed32 indicating the number of rows in the block
- A fixed32 indicating the size in bytes of the block before the codec is applied (excluding checksum).
- A fixed32 indicating the size in bytes of the block after the codec is applied (excluding checksum).
- If this column’s metadata declares it to include values, the first value in the column, serialized according to this column’s type.
A block consists of:
- The serialized column values. If a column is an array column then value sequences are preceded by their length, as an int. If a codec is specified, the values and lengths are compressed by that codec.
- The checksum, as determined by the file metadata.
Codecs
null
The “null” codec simply passes data through uncompressed.
deflate
The “deflate” codec writes the data block using the deflate algorithm as specified in RFC 1951.
snappy
The “snappy” codec uses Google’s Snappy compression library.
Checksum algorithms
null
The “null” checksum contains zero bytes.
crc-32
Each “crc-32” checksum contains the four bytes of an ISO 3309 CRC-32 checksum of the uncompressed block data as a fixed32.
Type Mappings
We define a standard mapping for how types defined in various serialization systems are represented in a column file. Records from these systems are shredded into columns. When records are nested, a depth-first recursive walk can assign a separate column for each primitive value.
Avro
Protocol Buffers
Thrift
Implementation Notes
Some possible techniques for writing column files include:
- Use a standard ~100MB block, buffer in memory up to the block size, then flush the file directly to HDFS. A single reduce task might create multiple output files. The namenode requires memory proportional to the number of names and blocks*replication. This would increase the number of names but not blocks, so this should still be much better than a file per column.
- Spill each column to a separate local, temporary file then, when the file is closed, append these files, writing a single file to HDFS whose block size is set to be that of the entire file. This would be a bit slower than and may have trouble when the local disk is full, but it would better use HDFS namespace and further reduce seeks when processing columns whose values are small.
- Use a separate mapreduce job to convert row-major files to column-major. The map output would output a by (row#, column#, value) tuple, partitioned by row# but sorted by column# then row#. The reducer could directly write the column file. But the column file format would need to be changed to write counts, descriptors, etc. at the end of files rather than at the front.
(1) is the simplest to implement and most implementations should start with it.
References
CIF Column-Oriented Storage Techniques for MapReduce, Floratou, Patel, Shekita, & Tata, VLDB 2011.
DREMEL Dremel: Interactive Analysis of Web-Scale Datasets, Melnik, Gubarev, Long, Romer, Shivakumar, & Tolton, VLDB 2010.
3.5 - MapReduce guide
Avro provides a convenient way to represent complex data structures within a Hadoop MapReduce job. Avro data can be used as both input to and output from a MapReduce job, as well as the intermediate format. The example in this guide uses Avro data for all three, but it’s possible to mix and match; for instance, MapReduce can be used to aggregate a particular field in an Avro record.
This guide assumes basic familiarity with both Hadoop MapReduce and Avro. See the Hadoop documentation and the Avro getting started guide for introductions to these projects. This guide uses the old MapReduce API (org.apache.hadoop.mapred) and the new MapReduce API (org.apache.hadoop.mapreduce).
Setup
The code from this guide is included in the Avro docs under examples/mr-example. The example is set up as a Maven project that includes the necessary Avro and MapReduce dependencies and the Avro Maven plugin for code generation, so no external jars are needed to run the example. In particular, the POM includes the following dependencies:
<dependency>
<groupId>org.apache.avro</groupId>
<artifactId>avro</artifactId>
<version>1.10.2</version>
</dependency>
<dependency>
<groupId>org.apache.avro</groupId>
<artifactId>avro-mapred</artifactId>
<version>1.10.2</version>
</dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-client</artifactId>
<version>3.1.2</version>
</dependency>
And the following plugin:
<plugin>
<groupId>org.apache.avro</groupId>
<artifactId>avro-maven-plugin</artifactId>
<version>1.10.2</version>
<executions>
<execution>
<phase>generate-sources</phase>
<goals>
<goal>schema</goal>
</goals>
<configuration>
<sourceDirectory>${project.basedir}/../</sourceDirectory>
<outputDirectory>${project.basedir}/target/generated-sources/</outputDirectory>
</configuration>
</execution>
</executions>
</plugin>
If you do not configure the sourceDirectory and outputDirectory properties, the defaults will be used. The sourceDirectory property defaults to src/main/avro. The outputDirectory property defaults to target/generated-sources. You can change the paths to match your project layout.
Alternatively, Avro jars can be downloaded directly from the Apache Avro™ Releases page. The relevant Avro jars for this guide are avro-1.10.2.jar and avro-mapred-1.10.2.jar, as well as avro-tools-1.10.2.jar for code generation and viewing Avro data files as JSON. In addition, you will need to install Hadoop in order to use MapReduce.
Example: ColorCount
Below is a simple example of a MapReduce that uses Avro. There is an example for both the old (org.apache.hadoop.mapred) and new (org.apache.hadoop.mapreduce) APIs under examples/mr-example/src/main/java/example/. MapredColorCount is the example for the older mapred API while MapReduceColorCount is the example for the newer mapreduce API. Both examples are below, but we will detail the mapred API in our subsequent examples.
MapredColorCount:
package example;
import java.io.IOException;
import org.apache.avro.*;
import org.apache.avro.Schema.Type;
import org.apache.avro.mapred.*;
import org.apache.hadoop.conf.*;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.mapred.*;
import org.apache.hadoop.util.*;
import example.avro.User;
public class MapredColorCount extends Configured implements Tool {
public static class ColorCountMapper extends AvroMapper<User, Pair<CharSequence, Integer>> {
@Override
public void map(User user, AvroCollector<Pair<CharSequence, Integer>> collector, Reporter reporter)
throws IOException {
CharSequence color = user.getFavoriteColor();
// We need this check because the User.favorite_color field has type ["string", "null"]
if (color == null) {
color = "none";
}
collector.collect(new Pair<CharSequence, Integer>(color, 1));
}
}
public static class ColorCountReducer extends AvroReducer<CharSequence, Integer,
Pair<CharSequence, Integer>> {
@Override
public void reduce(CharSequence key, Iterable<Integer> values,
AvroCollector<Pair<CharSequence, Integer>> collector,
Reporter reporter)
throws IOException {
int sum = 0;
for (Integer value : values) {
sum += value;
}
collector.collect(new Pair<CharSequence, Integer>(key, sum));
}
}
public int run(String[] args) throws Exception {
if (args.length != 2) {
System.err.println("Usage: MapredColorCount <input path> <output path>");
return -1;
}
JobConf conf = new JobConf(getConf(), MapredColorCount.class);
conf.setJobName("colorcount");
FileInputFormat.setInputPaths(conf, new Path(args[0]));
FileOutputFormat.setOutputPath(conf, new Path(args[1]));
AvroJob.setMapperClass(conf, ColorCountMapper.class);
AvroJob.setReducerClass(conf, ColorCountReducer.class);
// Note that AvroJob.setInputSchema and AvroJob.setOutputSchema set
// relevant config options such as input/output format, map output
// classes, and output key class.
AvroJob.setInputSchema(conf, User.getClassSchema());
AvroJob.setOutputSchema(conf, Pair.getPairSchema(Schema.create(Type.STRING),
Schema.create(Type.INT)));
JobClient.runJob(conf);
return 0;
}
public static void main(String[] args) throws Exception {
int res = ToolRunner.run(new Configuration(), new MapredColorCount(), args);
System.exit(res);
}
}
MapReduceColorCount:
package example;
import java.io.IOException;
import org.apache.avro.Schema;
import org.apache.avro.mapred.AvroKey;
import org.apache.avro.mapred.AvroValue;
import org.apache.avro.mapreduce.AvroJob;
import org.apache.avro.mapreduce.AvroKeyInputFormat;
import org.apache.avro.mapreduce.AvroKeyValueOutputFormat;
import org.apache.hadoop.conf.Configured;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.IntWritable;
import org.apache.hadoop.io.NullWritable;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.mapreduce.Job;
import org.apache.hadoop.mapreduce.Mapper;
import org.apache.hadoop.mapreduce.Reducer;
import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
import org.apache.hadoop.util.Tool;
import org.apache.hadoop.util.ToolRunner;
import example.avro.User;
public class MapReduceColorCount extends Configured implements Tool {
public static class ColorCountMapper extends
Mapper<AvroKey<User>, NullWritable, Text, IntWritable> {
@Override
public void map(AvroKey<User> key, NullWritable value, Context context)
throws IOException, InterruptedException {
CharSequence color = key.datum().getFavoriteColor();
if (color == null) {
color = "none";
}
context.write(new Text(color.toString()), new IntWritable(1));
}
}
public static class ColorCountReducer extends
Reducer<Text, IntWritable, AvroKey<CharSequence>, AvroValue<Integer>> {
@Override
public void reduce(Text key, Iterable<IntWritable> values,
Context context) throws IOException, InterruptedException {
int sum = 0;
for (IntWritable value : values) {
sum += value.get();
}
context.write(new AvroKey<CharSequence>(key.toString()), new AvroValue<Integer>(sum));
}
}
public int run(String[] args) throws Exception {
if (args.length != 2) {
System.err.println("Usage: MapReduceColorCount <input path> <output path>");
return -1;
}
Job job = new Job(getConf());
job.setJarByClass(MapReduceColorCount.class);
job.setJobName("Color Count");
FileInputFormat.setInputPaths(job, new Path(args[0]));
FileOutputFormat.setOutputPath(job, new Path(args[1]));
job.setInputFormatClass(AvroKeyInputFormat.class);
job.setMapperClass(ColorCountMapper.class);
AvroJob.setInputKeySchema(job, User.getClassSchema());
job.setMapOutputKeyClass(Text.class);
job.setMapOutputValueClass(IntWritable.class);
job.setOutputFormatClass(AvroKeyValueOutputFormat.class);
job.setReducerClass(ColorCountReducer.class);
AvroJob.setOutputKeySchema(job, Schema.create(Schema.Type.STRING));
AvroJob.setOutputValueSchema(job, Schema.create(Schema.Type.INT));
return (job.waitForCompletion(true) ? 0 : 1);
}
public static void main(String[] args) throws Exception {
int res = ToolRunner.run(new MapReduceColorCount(), args);
System.exit(res);
}
}
ColorCount reads in data files containing User records, defined in examples/user.avsc, and counts the number of instances of each favorite color. (This example draws inspiration from the canonical WordCount MapReduce application.) This example uses the old MapReduce API. See MapReduceAvroWordCount, found under doc/examples/mr-example/src/main/java/example/ to see the new MapReduce API example. The User schema is defined as follows:
{"namespace": "example.avro",
"type": "record",
"name": "User",
"fields": [
{"name": "name", "type": "string"},
{"name": "favorite_number", "type": ["int", "null"]},
{"name": "favorite_color", "type": ["string", "null"]}
]
}
This schema is compiled into the User class used by ColorCount via the Avro Maven plugin (see examples/mr-example/pom.xml for how this is set up).
ColorCountMapper essentially takes a User as input and extracts the User’s favorite color, emitting the key-value pair <favoriteColor, 1>. ColorCountReducer then adds up how many occurrences of a particular favorite color were emitted, and outputs the result as a Pair record. These Pairs are serialized to an Avro data file.
Running ColorCount
The ColorCount application is provided as a Maven project in the Avro docs under examples/mr-example. To build the project, including the code generation of the User schema, run:
Next, run GenerateData from examples/mr-examples to create an Avro data file, input/users.avro, containing 20 Users with favorite colors chosen randomly from a list:
mvn exec:java -q -Dexec.mainClass=example.GenerateData
Besides creating the data file, GenerateData prints the JSON representations of the Users generated to stdout, for example:
{"name": "user", "favorite_number": null, "favorite_color": "red"}
{"name": "user", "favorite_number": null, "favorite_color": "green"}
{"name": "user", "favorite_number": null, "favorite_color": "purple"}
{"name": "user", "favorite_number": null, "favorite_color": null}
...
Now we’re ready to run ColorCount. We specify our freshly-generated input folder as the input path and output as our output folder (note that MapReduce will not start a job if the output folder already exists):
mvn exec:java -q -Dexec.mainClass=example.MapredColorCount -Dexec.args="input output"
Once ColorCount completes, checking the contents of the new output directory should yield the following:
$ ls output/
part-00000.avro _SUCCESS
You can check the contents of the generated Avro file using the avro-tools jar:
$ java -jar /path/to/avro-tools-1.10.2.jar tojson output/part-00000.avro
{"value": 3, "key": "blue"}
{"value": 7, "key": "green"}
{"value": 1, "key": "none"}
{"value": 2, "key": "orange"}
{"value": 3, "key": "purple"}
{"value": 2, "key": "red"}
{"value": 2, "key": "yellow"}
Now let’s go over the ColorCount example in detail.
Mapper - org.apache.hadoop.mapred API
The easiest way to use Avro data files as input to a MapReduce job is to subclass AvroMapper. An AvroMapper defines a map function that takes an Avro datum as input and outputs a key/value pair represented as a Pair record. In the ColorCount example, ColorCountMapper is an AvroMapper that takes a User as input and outputs a Pair<CharSequence, Integer>>, where the CharSequence key is the user’s favorite color and the Integer value is 1.
public static class ColorCountMapper extends AvroMapper<User, Pair<CharSequence, Integer>> {
@Override
public void map(User user, AvroCollector<Pair<CharSequence, Integer>> collector, Reporter reporter)
throws IOException {
CharSequence color = user.getFavoriteColor();
// We need this check because the User.favorite_color field has type ["string", "null"]
if (color == null) {
color = "none";
}
collector.collect(new Pair<CharSequence, Integer>(color, 1));
}
}
In order to use our AvroMapper, we must call AvroJob.setMapperClass and AvroJob.setInputSchema.
AvroJob.setMapperClass(conf, ColorCountMapper.class);
AvroJob.setInputSchema(conf, User.getClassSchema());
Note that AvroMapper does not implement the Mapper interface. Under the hood, the specified Avro data files are deserialized into AvroWrappers containing the actual data, which are processed by a Mapper that calls the configured AvroMapper’s map function. AvroJob.setInputSchema sets up the relevant configuration parameters needed to make this happen, thus you should not need to call JobConf.setMapperClass, JobConf.setInputFormat, JobConf.setMapOutputKeyClass, JobConf.setMapOutputValueClass, or JobConf.setOutputKeyComparatorClass.
Mapper - org.apache.hadoop.mapreduce API
This document will not go into all the differences between the mapred and mapreduce APIs, however will describe the main differences. As you can see, ColorCountMapper is now a subclass of the Hadoop Mapper class and is passed an AvroKey as it’s key. Additionally, the AvroJob method calls were slightly changed.
public static class ColorCountMapper extends
Mapper<AvroKey<User>, NullWritable, Text, IntWritable> {
@Override
public void map(AvroKey<User> key, NullWritable value, Context context)
throws IOException, InterruptedException {
CharSequence color = key.datum().getFavoriteColor();
if (color == null) {
color = "none";
}
context.write(new Text(color.toString()), new IntWritable(1));
}
}
Reducer - org.apache.hadoop.mapred API
Analogously to AvroMapper, an AvroReducer defines a reducer function that takes the key/value types output by an AvroMapper (or any mapper that outputs Pairs) and outputs a key/value pair represented a Pair record. In the ColorCount example, ColorCountReducer is an AvroReducer that takes the CharSequence key representing a favorite color and the Iterable<Integer> representing the counts for that color (they should all be 1 in this example) and adds up the counts.
public static class ColorCountReducer extends AvroReducer<CharSequence, Integer,
Pair<CharSequence, Integer>> {
@Override
public void reduce(CharSequence key, Iterable<Integer> values,
AvroCollector<Pair<CharSequence, Integer>> collector,
Reporter reporter)
throws IOException {
int sum = 0;
for (Integer value : values) {
sum += value;
}
collector.collect(new Pair<CharSequence, Integer>(key, sum));
}
}
In order to use our AvroReducer, we must call AvroJob.setReducerClass and AvroJob.setOutputSchema.
AvroJob.setReducerClass(conf, ColorCountReducer.class);
AvroJob.setOutputSchema(conf, Pair.getPairSchema(Schema.create(Type.STRING),
Schema.create(Type.INT)));
Note that AvroReducer does not implement the Reducer interface. The intermediate Pairs output by the mapper are split into AvroKeys and AvroValues, which are processed by a Reducer that calls the configured AvroReducer’s reduce function. AvroJob.setOutputSchema sets up the relevant configuration parameters needed to make this happen, thus you should not need to call JobConf.setReducerClass, JobConf.setOutputFormat, JobConf.setOutputKeyClass, JobConf.setMapOutputKeyClass, JobConf.setMapOutputValueClass, or JobConf.setOutputKeyComparatorClass.
Reduce - org.apache.hadoop.mapreduce API
As before we not detail every difference between the APIs. As with the Mapper change ColorCountReducer is now a subclass of Reducer and AvroKey and AvroValue are emitted. Additionally, the AvroJob method calls were slightly changed.
public static class ColorCountReducer extends
Reducer<Text, IntWritable, AvroKey<CharSequence>, AvroValue<Integer>> {
@Override
public void reduce(Text key, Iterable<IntWritable> values,
Context context) throws IOException, InterruptedException {
int sum = 0;
for (IntWritable value : values) {
sum += value.get();
}
context.write(new AvroKey<CharSequence>(key.toString()), new AvroValue<Integer>(sum));
}
}
Learning more
The mapred API allows users to mix Avro AvroMappers and AvroReducers with non-Avro Mappers and Reducers and the mapreduce API allows users input Avro and output non-Avro or vice versa.
The mapred package has API org.apache.avro.mapred documentation as does the org.apache.avro.mapreduce package. MapReduce API (org.apache.hadoop.mapreduce). Similarily to the mapreduce package, it’s possible with the mapred API to implement your own Mappers and Reducers directly using the public classes provided in these libraries. See the AvroWordCount application, found under examples/mr-example/src/main/java/example/AvroWordCount.java in the Avro documentation, for an example of implementing a Reducer that outputs Avro data using the old MapReduce API. See the MapReduceAvroWordCount application, found under examples/mr-example/src/main/java/example/MapReduceAvroWordCount.java in the Avro documentation, for an example of implementing a Reducer that outputs Avro data using the new MapReduce API.
3.6 - IDL Language
Introduction
This document defines Avro IDL, a higher-level language for authoring Avro schemata. Before reading this document, you should have familiarity with the concepts of schemata and protocols, as well as the various primitive and complex types available in Avro.
Overview
Purpose
The aim of the Avro IDL language is to enable developers to author schemata in a way that feels more similar to common programming languages like Java, C++, or Python. Additionally, the Avro IDL language may feel more familiar for those users who have previously used the interface description languages (IDLs) in other frameworks like Thrift, Protocol Buffers, or CORBA.
Usage
Each Avro IDL file defines a single Avro Protocol, and thus generates as its output a JSON-format Avro Protocol file with extension .avpr.
To convert a .avdl file into a .avpr file, it may be processed by the idl tool. For example:
$ java -jar avro-tools.jar idl src/test/idl/input/namespaces.avdl /tmp/namespaces.avpr
$ head /tmp/namespaces.avpr
{
"protocol" : "TestNamespace",
"namespace" : "avro.test.protocol",
The idl tool can also process input to and from stdin and stdout. See idl --help for full usage information.
A Maven plugin is also provided to compile .avdl files. To use it, add something like the following to your pom.xml:
<build>
<plugins>
<plugin>
<groupId>org.apache.avro</groupId>
<artifactId>avro-maven-plugin</artifactId>
<executions>
<execution>
<goals>
<goal>idl-protocol</goal>
</goals>
</execution>
</executions>
</plugin>
</plugins>
</build>
Defining a Protocol in Avro IDL
An Avro IDL file consists of exactly one protocol definition. The minimal protocol is defined by the following code:
This is equivalent to (and generates) the following JSON protocol definition:
{
"protocol" : "MyProtocol",
"types" : [ ],
"messages" : {
}
}
The namespace of the protocol may be changed using the @namespace annotation:
@namespace("mynamespace")
protocol MyProtocol {
}
This notation is used throughout Avro IDL as a way of specifying properties for the annotated element, as will be described later in this document.
Protocols in Avro IDL can contain the following items:
- Imports of external protocol and schema files.
- Definitions of named schemata, including records, errors, enums, and fixeds.
- Definitions of RPC messages
Imports
Files may be imported in one of three formats:
-
An IDL file may be imported with a statement like:
import idl "foo.avdl";
-
A JSON protocol file may be imported with a statement like:
import protocol "foo.avpr";
-
A JSON schema file may be imported with a statement like:
import schema "foo.avsc";
Messages and types in the imported file are added to this file’s protocol.
Imported file names are resolved relative to the current IDL file.
Defining an Enumeration
Enums are defined in Avro IDL using a syntax similar to C or Java. An Avro Enum supports optional default values. In the case that a reader schema is unable to recognize a symbol written by the writer, the reader will fall back to using the defined default value. This default is only used when an incompatible symbol is read. It is not used if the enum field is missing.
Example Writer Enum Definition
enum Shapes {
SQUARE, TRIANGLE, CIRCLE, OVAL
}
Example Reader Enum Definition
enum Shapes {
SQUARE, TRIANGLE, CIRCLE
} = CIRCLE;
In the above example, the reader will use the default value of CIRCLE whenever reading data written with the OVAL symbol of the writer. Also note that, unlike the JSON format, anonymous enums cannot be defined.
Defining a Fixed Length Field
Fixed fields are defined using the following syntax:
fixed MD5(16);
This example defines a fixed-length type called MD5 which contains 16 bytes.
Defining Records and Errors
Records are defined in Avro IDL using a syntax similar to a struct definition in C:
record Employee {
string name;
boolean active = true;
long salary;
}
The above example defines a record with the name “Employee” with three fields.
To define an error, simply use the keyword error instead of record. For example:
error Kaboom {
string explanation;
int result_code = -1;
}
Each field in a record or error consists of a type and a name, optional property annotations and an optional default value.
A type reference in Avro IDL must be one of:
- A primitive type
- A logical type
- A named schema defined prior to this usage in the same Protocol
- A complex type (array, map, or union)
Primitive Types
The primitive types supported by Avro IDL are the same as those supported by Avro’s JSON format. This list includes int, long, string, boolean, float, double, null, and bytes.
Logical Types
Some of the logical types supported by Avro’s JSON format are also supported by Avro IDL. The currently supported types are:
For example:
record Job {
string jobid;
date submitDate;
time_ms submitTime;
timestamp_ms finishTime;
decimal(9,2) finishRatio;
}
References to Named Schemata
If a named schema has already been defined in the same Avro IDL file, it may be referenced by name as if it were a primitive type:
record Card {
Suit suit; // refers to the enum Card defined above
int number;
}
Default Values
Default values for fields may be optionally specified by using an equals sign after the field name followed by a JSON expression indicating the default value. This JSON is interpreted as described in the spec.
Complex Types
Arrays
Array types are written in a manner that will seem familiar to C++ or Java programmers. An array of any type t is denoted array<t>. For example, an array of strings is denoted array<string>, and a multidimensional array of Foo records would be array<array<Foo>>.
Maps
Map types are written similarly to array types. An array that contains values of type t is written map<t>. As in the JSON schema format, all maps contain string-type keys.
Unions
Union types are denoted as union { typeA, typeB, typeC, … }. For example, this record contains a string field that is optional (unioned with null):
record RecordWithUnion {
union { null, string } optionalString;
}
Note that the same restrictions apply to Avro IDL unions as apply to unions defined in the JSON format; namely, a record may not contain multiple elements of the same type.
Defining RPC Messages
The syntax to define an RPC message within a Avro IDL protocol is similar to the syntax for a method declaration within a C header file or a Java interface. To define an RPC message add which takes two arguments named foo and bar, returning an int, simply include the following definition within the protocol:
int add(int foo, int bar = 0);
Message arguments, like record fields, may specify default values.
To define a message with no response, you may use the alias void, equivalent to the Avro null type:
void logMessage(string message);
If you have previously defined an error type within the same protocol, you may declare that a message can throw this error using the syntax:
void goKaboom() throws Kaboom;
To define a one-way message, use the keyword oneway after the parameter list, for example:
void fireAndForget(string message) oneway;
Other Language Features
All Java-style comments are supported within a Avro IDL file. Any text following // on a line is ignored, as is any text between /* and */, possibly spanning multiple lines.
Comments that begin with /** are used as the documentation string for the type or field definition that follows the comment.
Escaping Identifiers
Occasionally, one will need to use a reserved language keyword as an identifier. In order to do so, backticks (`) may be used to escape the identifier. For example, to define a message with the literal name error, you may write:
This syntax is allowed anywhere an identifier is expected.
Annotations for Ordering and Namespaces
Java-style annotations may be used to add additional properties to types and fields throughout Avro IDL.
For example, to specify the sort order of a field within a record, one may use the @order annotation before the field name as follows:
record MyRecord {
string @order("ascending") myAscendingSortField;
string @order("descending") myDescendingField;
string @order("ignore") myIgnoredField;
}
A field’s type may also be preceded by annotations, e.g.:
record MyRecord {
@java-class("java.util.ArrayList") array<string> myStrings;
}
This can be used to support java classes that can be serialized/deserialized via their toString/String constructor, e.g.:
record MyRecord {
@java-class("java.math.BigDecimal") string value;
@java-key-class("java.io.File") map<string> fileStates;
array<@java-class("java.math.BigDecimal") string> weights;
}
Similarly, a @namespace annotation may be used to modify the namespace when defining a named schema. For example:
@namespace("org.apache.avro.firstNamespace")
protocol MyProto {
@namespace("org.apache.avro.someOtherNamespace")
record Foo {}
record Bar {}
}
will define a protocol in the firstNamespace namespace. The record Foo will be defined in someOtherNamespace and Bar will be defined in firstNamespace as it inherits its default from its container.
Type and field aliases are specified with the @aliases annotation as follows:
@aliases(["org.old.OldRecord", "org.ancient.AncientRecord"])
record MyRecord {
string @aliases(["oldField", "ancientField"]) myNewField;
}
Some annotations like those listed above are handled specially. All other annotations are added as properties to the protocol, message, schema or field.
Complete Example
The following is a complete example of a Avro IDL file that shows most of the above features:
/**
* An example protocol in Avro IDL
*/
@namespace("org.apache.avro.test")
protocol Simple {
@aliases(["org.foo.KindOf"])
enum Kind {
FOO,
BAR, // the bar enum value
BAZ
} = FOO; // For schema evolution purposes, unmatched values do not throw an error, but are resolved to FOO.
/** MD5 hash; good enough to avoid most collisions, and smaller than (for example) SHA256. */
fixed MD5(16);
record TestRecord {
@order("ignore")
string name;
@order("descending")
Kind kind;
MD5 hash;
union { MD5, null} @aliases(["hash"]) nullableHash;
array<long> arrayOfLongs;
}
error TestError {
string message;
}
string hello(string greeting);
TestRecord echo(TestRecord `record`);
int add(int arg1, int arg2);
bytes echoBytes(bytes data);
void `error`() throws TestError;
void ping() oneway;
}
Additional examples may be found in the Avro source tree under the src/test/idl/input directory.
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
3.7 - SASL profile
Introduction
SASL (RFC 2222) provides a framework for authentication and security of network protocols. Each protocol that uses SASL is meant to define a SASL profile. This document provides a SASL profile for connection-based Avro RPC.
Overview
SASL negotiation proceeds as a series of message interactions over a connection between a client and server using a selected SASL mechanism. The client starts this negotiation by sending its chosen mechanism name with an initial (possibly empty) message. Negotiation proceeds with the exchange of messages until either side indicates success or failure. The content of the messages is mechanism-specific. If the negotiation succeeds, then the session can proceed over the connection, otherwise it must be abandoned.
Some mechanisms continue to process session data after negotiation (e.g., encrypting it), while some specify that further session data is transmitted unmodified.
Negotiation
Commands
Avro SASL negotiation uses four one-byte commands.
- 0: START Used in a client’s initial message.
- 1: CONTINUE Used while negotiation is ongoing.
- 2: FAIL Terminates negotiation unsuccessfully.
- 3: COMPLETE Terminates negotiation successfully.
The format of a START message is:
| 0 | 4-byte mechanism name length | mechanism name | 4-byte payload length | payload data |
The format of a CONTINUE message is:
| 1 | 4-byte payload length | payload data |
The format of a FAIL message is:
| 2 | 4-byte message length | UTF-8 message |
The format of a COMPLETE message is:
| 3 | 4-byte payload length | payload data |
Process
Negotiation is initiated by a client sending a START command containing the client’s chosen mechanism name and any mechanism-specific payload data.
The server and client then interchange some number (possibly zero) of CONTINUE messages. Each message contains payload data that is processed by the security mechanism to generate the next message.
Once either the client or server send a FAIL message then negotiation has failed. UTF-8-encoded text is included in the failure message. Once either a FAIL message has been sent or received, or any other error occurs in the negotiation, further communication on this connection must cease.
Once either the client or server send a COMPLETE message then negotiation has completed successfully. Session data may now be transmitted over the connection until it is closed by either side.
Session Data
If no SASL QOP (quality of protection) is negotiated, then all subsequent writes to/reads over this connection are written/read unmodified. In particular, messages use Avro framing, and are of the form:
| 4-byte frame length | frame data | ... | 4 zero bytes |
If a SASL QOP is negotiated, then it must be used by the connection for all subsequent messages. This is done by wrapping each non-empty frame written using the security mechanism and unwrapping each non-empty frame read. The length written in each non-empty frame is the length of the wrapped data. Complete frames must be passed to the security mechanism for unwrapping. Unwrapped data is then passed to the application as the content of the frame.
If at any point processing fails due to wrapping, unwrapping or framing errors, then all further communication on this connection must cease.
Anonymous Mechanism
The SASL anonymous mechanism (RFC 2245) is quite simple to implement. In particular, an initial anonymous request may be prefixed by the following static sequence:
| 0 | 0009 | ANONYMOUS | 0000 |
If a server uses the anonymous mechanism, it should check that the mechanism name in the start message prefixing the first request received is ‘ANONYMOUS’, then simply prefix its initial response with a COMPLETE message of:
| 3 | 0000 |
If an anonymous server recieves some other mechanism name, then it may respond with a FAIL message as simple as:
| 2 | 0000 |
Note that the anonymous mechanism need add no additional round-trip messages between client and server. The START message can be piggybacked on the initial request and the COMPLETE or FAIL message can be piggybacked on the initial response.
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
3.8 - Wiki
The Wiki page can be found here.
3.9 - FAQ
What is Avro?
Avro is a data serialization system.
Avro provides:
- Rich data structures.
- A compact, fast, binary data format.
- A container file, to store persistent data.
- Remote procedure call (RPC).
- Simple integration with dynamic languages. Code generation is not required to read or write data files nor to use or implement RPC protocols. Code generation as an optional optimization, only worth implementing for statically typed languages.
How can I get started quickly with Avro?
Check out the Quick Start Guide.
How do I statically compile a schema or protocol into generated code?
In Java
- Add the avro jar, the jackson-mapper-asl.jar and jackson-core-asl.jar to your CLASSPATH.
- Run java org.apache.avro.specific.SpecificCompiler .
This appears to be out of date, the SpecificCompiler requires two arguments, presumably an input and and output file, but it isn’t clear that this does.
Or use the Schema or Protocol Ant tasks. Avro’s build.xml provides examples of how these are used.
Lastly, you can also use the “avro-tools” jar which ships with an Avro release. Just use the “compile (schema|protocol)” command.
How are Strings represented in Java?
They use org.apache.avro.util.Utf8, not java.lang.String.
More generally, how do Avro types map to Java types?
The mappings are documented in the package javadoc for generic, specific and reflect API.
From Doug Cutting:
HDFS splits files into blocks, and mapreduce runs a map task for each block. When the task starts, it needs to be able to seek into the file to the start of the block process through the block’s end. If the file were, e.g., a gzip file, this would not be possible, since gzip files must be decompressed from the start. One cannot seek into the middle of a gzip file and start decompressing. So Hadoop’s SequenceFile places a marker periodically (~64k) in the file at record and compression boundaries, where processing can be sensibly started. Then, when a map task starts processing an HDFS block, it finds the first marker after the block’s start and continues through the first marker in the next block of the file. This requires a bit of non-local access (~0.1%). Avro’s data file uses the same method as SequenceFile.
Why isn’t every value in Avro nullable?
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 of every value stored and
corresponding computational costs to create this bit on write and
interpret it on read. These costs are wasted when values may not in
fact be null, as is the case in many datasets. In Avro such costs are
only paid when values may actually be null.
Also, allowing values to be null is a well-known source of errors. In
Avro, a value declared as non-null will always be non-null and programs
need not test for null values when processing it nor will they ever fail
for lack of such tests.
Tony Hoare calls his invention of null references his “Billion Dollar
Mistake”.
http://qconlondon.com/london-2009/presentation/Null+References:+The+Billion+Dollar+Mistake
Also note that in some programming languages not all values are permitted to be null. For example, in Java, values of type boolean, byte, short, char, int, float, long, and double may not be null.
How can I serialize directly to/from a byte array?
As pointed out in the specification, Avro data should always be stored with its schema. The Avro provided classes DataFileWriter, DataFileReader, and DataFileStream all ensure this by serializing the Schema in a container header. In some special cases, such as when implementing a new storage system or writing unit tests, you may need to write and read directly with the bare Avro serialized values. The following examples use code generated for Java from an Avro Schema, specifically the User example from the Quickstart guide.
Serializing to a byte array
This example takes a User object and returns a newly allocated byte array with the Avro serialization of that user.
ByteArrayOutputStream out = new ByteArrayOutputStream();
BinaryEncoder encoder = EncoderFactory.get().binaryEncoder(out, null);
DatumWriter<User> writer = new SpecificDatumWriter<User>(User.getClassSchema());
writer.write(user, encoder);
encoder.flush();
out.close();
byte[] serializedBytes = out.toByteArray();
Deserializing from a byte array
This example takes a byte array containing the Avro serialization of a user and returns a User object.SpecificDatumReader reader = new SpecificDatumReader(User.getClassSchema());
Decoder decoder = DecoderFactory.get().binaryDecoder(bytes, null);
User user = reader.read(null, decoder);
Note: If you are serializing or deserializing in a loop or as a method, you should be reusing objects, readers and/or writers. Check the JavaDoc to see how to reuse the objects.
4 - Apache Avro™ 1.10.1 Documentation
Introduction
Apache Avro™ is a data serialization system.
Avro provides:
- Rich data structures.
- A compact, fast, binary data format.
- A container file, to store persistent data.
- Remote procedure call (RPC).
- Simple integration with dynamic languages. Code generation is not required to read or write data files nor to use or implement RPC protocols. Code generation as an optional optimization, only worth implementing for statically typed languages.
Schemas
Avro relies on schemas. When Avro data is read, the schema used when writing it is always present. This permits each datum to be written with no per-value overheads, making serialization both fast and small. This also facilitates use with dynamic, scripting languages, since data, together with its schema, is fully self-describing.
When Avro data is stored in a file, its schema is stored with it, so that files may be processed later by any program. If the program reading the data expects a different schema this can be easily resolved, since both schemas are present.
When Avro is used in RPC, the client and server exchange schemas in the connection handshake. (This can be optimized so that, for most calls, no schemas are actually transmitted.) Since both client and server both have the other’s full schema, correspondence between same named fields, missing fields, extra fields, etc. can all be easily resolved.
Avro schemas are defined with JSON . This facilitates implementation in languages that already have JSON libraries.
Comparison with other systems
Avro provides functionality similar to systems such as Thrift, Protocol Buffers, etc. Avro differs from these systems in the following fundamental aspects.
- Dynamic typing: Avro does not require that code be generated. Data is always accompanied by a schema that permits full processing of that data without code generation, static datatypes, etc. This facilitates construction of generic data-processing systems and languages.
- Untagged data: Since the schema is present when data is read, considerably less type information need be encoded with data, resulting in smaller serialization size.
- No manually-assigned field IDs: When a schema changes, both the old and new schema are always present when processing data, so differences may be resolved symbolically, using field names.
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
4.1 - Getting Started (Java)
This is a short guide for getting started with Apache Avro™ using Java. This guide only covers using Avro for data serialization; see Patrick Hunt’s Avro RPC Quick Start for a good introduction to using Avro for RPC.
Download
Avro implementations for C, C++, C#, Java, PHP, Python, and Ruby can be downloaded from the Apache Avro™ Releases page. This guide uses Avro 1.10.1, the latest version at the time of writing. For the examples in this guide, download avro-1.10.1.jar and avro-tools-1.10.1.jar.
Alternatively, if you are using Maven, add the following dependency to your POM:
<dependency>
<groupId>org.apache.avro</groupId>
<artifactId>avro</artifactId>
<version>1.10.1</version>
</dependency>
As well as the Avro Maven plugin (for performing code generation):
<plugin>
<groupId>org.apache.avro</groupId>
<artifactId>avro-maven-plugin</artifactId>
<version>1.10.1</version>
<executions>
<execution>
<phase>generate-sources</phase>
<goals>
<goal>schema</goal>
</goals>
<configuration>
<sourceDirectory>${project.basedir}/src/main/avro/</sourceDirectory>
<outputDirectory>${project.basedir}/src/main/java/</outputDirectory>
</configuration>
</execution>
</executions>
</plugin>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-compiler-plugin</artifactId>
<configuration>
<source>1.8</source>
<target>1.8</target>
</configuration>
</plugin>
You may also build the required Avro jars from source. Building Avro is beyond the scope of this guide; see the Build Documentation page in the wiki for more information.
Defining a schema
Avro schemas are defined using JSON. Schemas are composed of primitive types (null, boolean, int, long, float, double, bytes, and string) and complex types (record, enum, array, map, union, and fixed). You can learn more about Avro schemas and types from the specification, but for now let’s start with a simple schema example, user.avsc:
{"namespace": "example.avro",
"type": "record",
"name": "User",
"fields": [
{"name": "name", "type": "string"},
{"name": "favorite_number", "type": ["int", "null"]},
{"name": "favorite_color", "type": ["string", "null"]}
]
}
This schema defines a record representing a hypothetical user. (Note that a schema file can only contain a single schema definition.) At minimum, a record definition must include its type (“type”: “record”), a name (“name”: “User”), and fields, in this case name, favorite_number, and favorite_color. We also define a namespace (“namespace”: “example.avro”), which together with the name attribute defines the “full name” of the schema (example.avro.User in this case).
Fields are defined via an array of objects, each of which defines a name and type (other attributes are optional, see the record specification for more details). The type attribute of a field is another schema object, which can be either a primitive or complex type. For example, the name field of our User schema is the primitive type string, whereas the favorite_number and favorite_color fields are both unions, represented by JSON arrays. unions are a complex type that can be any of the types listed in the array; e.g., favorite_number can either be an int or null, essentially making it an optional field.
Serializing and deserializing with code generation
Compiling the schema
Code generation allows us to automatically create classes based on our previously-defined schema. Once we have defined the relevant classes, there is no need to use the schema directly in our programs. We use the avro-tools jar to generate code as follows:
java -jar /path/to/avro-tools-1.10.1.jar compile schema <schema file> <destination>
This will generate the appropriate source files in a package based on the schema’s namespace in the provided destination folder. For instance, to generate a User class in package example.avro from the schema defined above, run
java -jar /path/to/avro-tools-1.10.1.jar compile schema user.avsc .
Note that if you using the Avro Maven plugin, there is no need to manually invoke the schema compiler; the plugin automatically performs code generation on any .avsc files present in the configured source directory.
Creating Users
Now that we’ve completed the code generation, let’s create some Users, serialize them to a data file on disk, and then read back the file and deserialize the User objects.
First let’s create some Users and set their fields.
User user1 = new User();
user1.setName("Alyssa");
user1.setFavoriteNumber(256);
// Leave favorite color null
// Alternate constructor
User user2 = new User("Ben", 7, "red");
// Construct via builder
User user3 = User.newBuilder()
.setName("Charlie")
.setFavoriteColor("blue")
.setFavoriteNumber(null)
.build();
As shown in this example, Avro objects can be created either by invoking a constructor directly or by using a builder. Unlike constructors, builders will automatically set any default values specified in the schema. Additionally, builders validate the data as it set, whereas objects constructed directly will not cause an error until the object is serialized. However, using constructors directly generally offers better performance, as builders create a copy of the datastructure before it is written.
Note that we do not set user1’s favorite color. Since that record is of type [“string”, “null”], we can either set it to a string or leave it null; it is essentially optional. Similarly, we set user3’s favorite number to null (using a builder requires setting all fields, even if they are null).
Serializing
Now let’s serialize our Users to disk.
/// Serialize user1, user2 and user3 to disk
DatumWriter<User> userDatumWriter = new SpecificDatumWriter<User>(User.class);
DataFileWriter<User> dataFileWriter = new DataFileWriter<User>(userDatumWriter);
dataFileWriter.create(user1.getSchema(), new File("users.avro"));
dataFileWriter.append(user1);
dataFileWriter.append(user2);
dataFileWriter.append(user3);
dataFileWriter.close();
We create a DatumWriter, which converts Java objects into an in-memory serialized format. The SpecificDatumWriter class is used with generated classes and extracts the schema from the specified generated type.
Next we create a DataFileWriter, which writes the serialized records, as well as the schema, to the file specified in the dataFileWriter.create call. We write our users to the file via calls to the dataFileWriter.append method. When we are done writing, we close the data file.
Deserializing
Finally, let’s deserialize the data file we just created.
// Deserialize Users from disk
DatumReader<User> userDatumReader = new SpecificDatumReader<User>(User.class);
DataFileReader<User> dataFileReader = new DataFileReader<User>(file, userDatumReader);
User user = null;
while (dataFileReader.hasNext()) {
// Reuse user object by passing it to next(). This saves us from
// allocating and garbage collecting many objects for files with
// many items.
user = dataFileReader.next(user);
System.out.println(user);
}
This snippet will output:
{"name": "Alyssa", "favorite_number": 256, "favorite_color": null}
{"name": "Ben", "favorite_number": 7, "favorite_color": "red"}
{"name": "Charlie", "favorite_number": null, "favorite_color": "blue"}
Deserializing is very similar to serializing. We create a SpecificDatumReader, analogous to the SpecificDatumWriter we used in serialization, which converts in-memory serialized items into instances of our generated class, in this case User. We pass the DatumReader and the previously created File to a DataFileReader, analogous to the DataFileWriter, which reads both the schema used by the writer as well as the data from the file on disk. The data will be read using the writer’s schema included in the file and the schema provided by the reader, in this case the User class. The writer’s schema is needed to know the order in which fields were written, while the reader’s schema is needed to know what fields are expected and how to fill in default values for fields added since the file was written. If there are differences between the two schemas, they are resolved according to the Schema Resolution specification.
Next we use the DataFileReader to iterate through the serialized Users and print the deserialized object to stdout. Note how we perform the iteration: we create a single User object which we store the current deserialized user in, and pass this record object to every call of dataFileReader.next. This is a performance optimization that allows the DataFileReader to reuse the same User object rather than allocating a new User for every iteration, which can be very expensive in terms of object allocation and garbage collection if we deserialize a large data file. While this technique is the standard way to iterate through a data file, it’s also possible to use for (User user : dataFileReader) if performance is not a concern.
Compiling and running the example code
This example code is included as a Maven project in the examples/java-example directory in the Avro docs. From this directory, execute the following commands to build and run the example:
$ mvn compile # includes code generation via Avro Maven plugin
$ mvn -q exec:java -Dexec.mainClass=example.SpecificMain
Beta feature: Generating faster code
In this release we have introduced a new approach to generating code that speeds up decoding of objects by more than 10% and encoding by more than 30% (future performance enhancements are underway). To ensure a smooth introduction of this change into production systems, this feature is controlled by a feature flag, the system property org.apache.avro.specific.use_custom_coders. In this first release, this feature is off by default. To turn it on, set the system flag to true at runtime. In the sample above, for example, you could enable the fater coders as follows:
$ mvn -q exec:java -Dexec.mainClass=example.SpecificMain \
-Dorg.apache.avro.specific.use_custom_coders=true
Note that you do not have to recompile your Avro schema to have access to this feature. The feature is compiled and built into your code, and you turn it on and off at runtime using the feature flag. As a result, you can turn it on during testing, for example, and then off in production. Or you can turn it on in production, and quickly turn it off if something breaks.
We encourage the Avro community to exercise this new feature early to help build confidence. (For those paying one-demand for compute resources in the cloud, it can lead to meaningful cost savings.) As confidence builds, we will turn this feature on by default, and eventually eliminate the feature flag (and the old code).
Serializing and deserializing without code generation
Data in Avro is always stored with its corresponding schema, meaning we can always read a serialized item regardless of whether we know the schema ahead of time. This allows us to perform serialization and deserialization without code generation.
Let’s go over the same example as in the previous section, but without using code generation: we’ll create some users, serialize them to a data file on disk, and then read back the file and deserialize the users objects.
Creating users
First, we use a Parser to read our schema definition and create a Schema object.
Schema schema = new Schema.Parser().parse(new File("user.avsc"));
Using this schema, let’s create some users.
GenericRecord user1 = new GenericData.Record(schema);
user1.put("name", "Alyssa");
user1.put("favorite_number", 256);
// Leave favorite color null
GenericRecord user2 = new GenericData.Record(schema);
user2.put("name", "Ben");
user2.put("favorite_number", 7);
user2.put("favorite_color", "red");
Since we’re not using code generation, we use GenericRecords to represent users. GenericRecord uses the schema to verify that we only specify valid fields. If we try to set a non-existent field (e.g., user1.put(“favorite_animal”, “cat”)), we’ll get an AvroRuntimeException when we run the program.
Note that we do not set user1’s favorite color. Since that record is of type [“string”, “null”], we can either set it to a string or leave it null; it is essentially optional.
Serializing
Now that we’ve created our user objects, serializing and deserializing them is almost identical to the example above which uses code generation. The main difference is that we use generic instead of specific readers and writers.
First we’ll serialize our users to a data file on disk.
// Serialize user1 and user2 to disk
File file = new File("users.avro");
DatumWriter<GenericRecord> datumWriter = new GenericDatumWriter<GenericRecord>(schema);
DataFileWriter<GenericRecord> dataFileWriter = new DataFileWriter<GenericRecord>(datumWriter);
dataFileWriter.create(schema, file);
dataFileWriter.append(user1);
dataFileWriter.append(user2);
dataFileWriter.close();
We create a DatumWriter, which converts Java objects into an in-memory serialized format. Since we are not using code generation, we create a GenericDatumWriter. It requires the schema both to determine how to write the GenericRecords and to verify that all non-nullable fields are present.
As in the code generation example, we also create a DataFileWriter, which writes the serialized records, as well as the schema, to the file specified in the dataFileWriter.create call. We write our users to the file via calls to the dataFileWriter.append method. When we are done writing, we close the data file.
Deserializing
Finally, we’ll deserialize the data file we just created.
// Deserialize users from disk
DatumReader<GenericRecord> datumReader = new GenericDatumReader<GenericRecord>(schema);
DataFileReader<GenericRecord> dataFileReader = new DataFileReader<GenericRecord>(file, datumReader);
GenericRecord user = null;
while (dataFileReader.hasNext()) {
// Reuse user object by passing it to next(). This saves us from
// allocating and garbage collecting many objects for files with
// many items.
user = dataFileReader.next(user);
System.out.println(user);
This outputs:
{"name": "Alyssa", "favorite_number": 256, "favorite_color": null}
{"name": "Ben", "favorite_number": 7, "favorite_color": "red"}
Deserializing is very similar to serializing. We create a GenericDatumReader, analogous to the GenericDatumWriter we used in serialization, which converts in-memory serialized items into GenericRecords. We pass the DatumReader and the previously created File to a DataFileReader, analogous to the DataFileWriter, which reads both the schema used by the writer as well as the data from the file on disk. The data will be read using the writer’s schema included in the file, and the reader’s schema provided to the GenericDatumReader. The writer’s schema is needed to know the order in which fields were written, while the reader’s schema is needed to know what fields are expected and how to fill in default values for fields added since the file was written. If there are differences between the two schemas, they are resolved according to the Schema Resolution specification.
Next, we use the DataFileReader to iterate through the serialized users and print the deserialized object to stdout. Note how we perform the iteration: we create a single GenericRecord object which we store the current deserialized user in, and pass this record object to every call of dataFileReader.next. This is a performance optimization that allows the DataFileReader to reuse the same record object rather than allocating a new GenericRecord for every iteration, which can be very expensive in terms of object allocation and garbage collection if we deserialize a large data file. While this technique is the standard way to iterate through a data file, it’s also possible to use for (GenericRecord user : dataFileReader) if performance is not a concern.
Compiling and running the example code
This example code is included as a Maven project in the examples/java-example directory in the Avro docs. From this directory, execute the following commands to build and run the example:
$ mvn compile
$ mvn -q exec:java -Dexec.mainClass=example.GenericMain
4.2 - Getting Started (Python)
This is a short guide for getting started with Apache Avro™ using Python. This guide only covers using Avro for data serialization; see Patrick Hunt’s Avro RPC Quick Start for a good introduction to using Avro for RPC.
Notice for Python 3 users
A package called “avro-python3” had been provided to support Python 3 previously, but the codebase was consolidated into the “avro” package and that supports both Python 2 and 3 now. The avro-python3 package will be removed in the near future, so users should use the “avro” package instead. They are mostly API compatible, but there’s a few minor difference (e.g., function name capitalization, such as avro.schema.Parse vs avro.schema.parse).
Download
For Python, the easiest way to get started is to install it from PyPI. Python’s Avro API is available over PyPi.
$ python3 -m pip install avro
The official releases of the Avro implementations for C, C++, C#, Java, PHP, Python, and Ruby can be downloaded from the Apache Avro™ Releases page. This guide uses Avro 1.10.1, the latest version at the time of writing. Download and unzip avro-1.10.1.tar.gz, and install via python setup.py (this will probably require root privileges). Ensure that you can import avro from a Python prompt.
$ tar xvf avro-1.10.1.tar.gz
$ cd avro-1.10.1
$ python setup.py install
$ python
>>> import avro # should not raise ImportError
Alternatively, you may build the Avro Python library from source. From your the root Avro directory, run the commands
$ cd lang/py/
$ python3 -m pip install -e .
$ python
Defining a schema
Avro schemas are defined using JSON. Schemas are composed of primitive types (null, boolean, int, long, float, double, bytes, and string) and complex types (record, enum, array, map, union, and fixed). You can learn more about Avro schemas and types from the specification, but for now let’s start with a simple schema example, user.avsc:
{"namespace": "example.avro",
"type": "record",
"name": "User",
"fields": [
{"name": "name", "type": "string"},
{"name": "favorite_number", "type": ["int", "null"]},
{"name": "favorite_color", "type": ["string", "null"]}
]
}
This schema defines a record representing a hypothetical user. (Note that a schema file can only contain a single schema definition.) At minimum, a record definition must include its type (“type”: “record”), a name (“name”: “User”), and fields, in this case name, favorite_number, and favorite_color. We also define a namespace (“namespace”: “example.avro”), which together with the name attribute defines the “full name” of the schema (example.avro.User in this case).
Fields are defined via an array of objects, each of which defines a name and type (other attributes are optional, see the record specification for more details). The type attribute of a field is another schema object, which can be either a primitive or complex type. For example, the name field of our User schema is the primitive type string, whereas the favorite_number and favorite_color fields are both unions, represented by JSON arrays. unions are a complex type that can be any of the types listed in the array; e.g., favorite_number can either be an int or null, essentially making it an optional field.
Serializing and deserializing without code generation
Data in Avro is always stored with its corresponding schema, meaning we can always read a serialized item, regardless of whether we know the schema ahead of time. This allows us to perform serialization and deserialization without code generation. Note that the Avro Python library does not support code generation.
Try running the following code snippet, which serializes two users to a data file on disk, and then reads back and deserializes the data file:
import avro.schema
from avro.datafile import DataFileReader, DataFileWriter
from avro.io import DatumReader, DatumWriter
schema = avro.schema.parse(open("user.avsc", "rb").read())
writer = DataFileWriter(open("users.avro", "wb"), DatumWriter(), schema)
writer.append({"name": "Alyssa", "favorite_number": 256})
writer.append({"name": "Ben", "favorite_number": 7, "favorite_color": "red"})
writer.close()
reader = DataFileReader(open("users.avro", "rb"), DatumReader())
for user in reader:
print user
reader.close()
This outputs:
{u'favorite_color': None, u'favorite_number': 256, u'name': u'Alyssa'}
{u'favorite_color': u'red', u'favorite_number': 7, u'name': u'Ben'}
Do make sure that you open your files in binary mode (i.e. using the modes wb or rb respectively). Otherwise you might generate corrupt files due to automatic replacement of newline characters with the platform-specific representations.
Let’s take a closer look at what’s going on here.
schema = avro.schema.parse(open("user.avsc", "rb").read())
avro.schema.parse takes a string containing a JSON schema definition as input and outputs a avro.schema.Schema object (specifically a subclass of Schema, in this case RecordSchema). We’re passing in the contents of our user.avsc schema file here.
writer = DataFileWriter(open("users.avro", "wb"), DatumWriter(), schema)
We create a DataFileWriter, which we’ll use to write serialized items to a data file on disk. The DataFileWriter constructor takes three arguments:
- The file we’ll serialize to
- A DatumWriter, which is responsible for actually serializing the items to Avro’s binary format (DatumWriters can be used separately from DataFileWriters, e.g., to perform IPC with Avro).
- The schema we’re using. The DataFileWriter needs the schema both to write the schema to the data file, and to verify that the items we write are valid items and write the appropriate fields.
writer.append({"name": "Alyssa", "favorite_number": 256})
writer.append({"name": "Ben", "favorite_number": 7, "favorite_color": "red"})
We use DataFileWriter.append to add items to our data file. Avro records are represented as Python dicts. Since the field favorite_color has type [“int”, “null”], we are not required to specify this field, as shown in the first append. Were we to omit the required name field, an exception would be raised. Any extra entries not corresponding to a field are present in the dict are ignored.
reader = DataFileReader(open("users.avro", "rb"), DatumReader())
We open the file again, this time for reading back from disk. We use a DataFileReader and DatumReader analagous to the DataFileWriter and DatumWriter above.
for user in reader:
print user
The DataFileReader is an iterator that returns dicts corresponding to the serialized items.
4.3 - Specification
Introduction
This document defines Apache Avro. It is intended to be the authoritative specification. Implementations of Avro must adhere to this document.
Schema Declaration
A Schema is represented in JSON by one of:
- A JSON string, naming a defined type.
- A JSON object, of the form:
{"type": "typeName" ...attributes...}
where typeName is either a primitive or derived type name, as defined below. Attributes not defined in this document are permitted as metadata, but must not affect the format of serialized data.
- A JSON array, representing a union of embedded types.
Primitive Types
The set of primitive type names is:
- null: no value
- boolean: a binary value
- int: 32-bit signed integer
- long: 64-bit signed integer
- float: single precision (32-bit) IEEE 754 floating-point number
- double: double precision (64-bit) IEEE 754 floating-point number
- bytes: sequence of 8-bit unsigned bytes
- string: unicode character sequence
Primitive types have no specified attributes.
Primitive type names are also defined type names. Thus, for example, the schema “string” is equivalent to:
Complex Types
Avro supports six kinds of complex types: records, enums, arrays, maps, unions and fixed.
Records
Records use the type name “record” and support the three attributes:
- name: a JSON string providing the name of the record (required).
- namespace, a JSON string that qualifies the name;
- doc: a JSON string providing documentation to the user of this schema (optional).
- aliases: a JSON array of strings, providing alternate names for this record (optional).
- fields: a JSON array, listing fields (required). Each field is a JSON object with the following attributes:
- name: a JSON string providing the name of the field (required), and
- doc: a JSON string describing this field for users (optional).
- type: a schema, as defined above
- _default: A default value for this field, used when reading instances that lack this field (optional). 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. Default values for bytes and fixed fields are JSON strings, where Unicode code points 0-255 are mapped to unsigned 8-bit byte values 0-255.
field default values
| avro type |
json type |
example |
| null |
null |
null |
| boolean |
boolean |
true |
| int,long |
integer |
1 |
| float,double |
number |
1.1 |
| bytes |
string |
"\u00FF" |
| string |
string |
"foo" |
| record |
object |
{"a": 1} |
| enum |
string |
"FOO" |
| array |
array |
[1] |
| map |
object |
{"a": 1} |
| fixed |
string |
"\u00ff" |
- order: specifies how this field impacts sort ordering of this record (optional). Valid values are “ascending” (the default), “descending”, or “ignore”. For more details on how this is used, see the sort order section below.
- aliases: a JSON array of strings, providing alternate names for this field (optional).
For example, a linked-list of 64-bit values may be defined with:
{
"type": "record",
"name": "LongList",
"aliases": ["LinkedLongs"], // old name for this
"fields" : [
{"name": "value", "type": "long"}, // each element has a long
{"name": "next", "type": ["null", "LongList"]} // optional next element
]
}
Enums
Enums use the type name “enum” and support two attributes:
- name: a JSON string providing the name of the enum (required).
- namespace, a JSON string that qualifies the name ;
- aliases: a JSON array of strings, providing alternate names for this enum (optional).
- doc: a JSON string providing documentation to the user of this schema (optional).
- symbols: a JSON array, listing symbols, as JSON strings (required). All symbols in an enum must be unique; duplicates are prohibited. Every symbol must match the regular expression [A-Za-z_][A-Za-z0-9_]* (the same requirement as for names).
- default: A default value for this enumeration, used during resolution when the reader encounters a symbol from the writer that isn’t defined in the reader’s schema (optional). The value provided here must be a JSON string that’s a member of the symbols array. See documentation on schema resolution for how this gets used.
For example, playing card suits might be defined with:
{
"type": "enum",
"name": "Suit",
"symbols" : ["SPADES", "HEARTS", "DIAMONDS", "CLUBS"]
}
Arrays
Arrays use the type name “array” and support a single attribute:
- items: the schema of the array’s items.
For example, an array of strings is declared with:
{
"type": "array",
"items" : "string",
"default": []
}
Maps
Maps use the type name “map” and support one attribute:
- values: the schema of the map’s values.
Map keys are assumed to be strings.
For example, a map from string to long is declared with:
{
"type": "map",
"values" : "long",
"default": {}
}
Unions
Unions, as mentioned above, are represented using JSON arrays. For example, ["null", "string"] declares a schema which may be either a null or string.
(Note that when a default value is specified for a record field whose type is a union, the type of the default value must match the first element of the union. Thus, for unions containing “null”, the “null” is usually listed first, since the default value of such unions is typically null.)
Unions may not contain more than one schema with the same type, except for the named types record, fixed and enum. For example, unions containing two array types or two map types are not permitted, but two types with different names are permitted. (Names permit efficient resolution when reading and writing unions.)
Unions may not immediately contain other unions.
Fixed
Fixed uses the type name “fixed” and supports the following attributes:
- name: a string naming this fixed (required).
- namespace, a string that qualifies the name ;
- aliases: a JSON array of strings, providing alternate names for this enum (optional).
- doc: a JSON string providing documentation to the user of this schema (optional).
- size: an integer, specifying the number of bytes per value (required).
For example, 16-byte quantity may be declared with:
{"type": "fixed", "size": 16, "name": "md5"}
Names
Record, enums and fixed are named types. Each has a fullname that is composed of two parts; a name and a namespace. Equality of names is defined on the fullname.
The name portion of a fullname, record field names, and enum symbols must:
- start with [A-Za-z_]
- subsequently contain only [A-Za-z0-9_]
A namespace is a dot-separated sequence of such names. The empty string may also be used as a namespace to indicate the null namespace. Equality of names (including field names and enum symbols) as well as fullnames is case-sensitive.
The null namespace may not be used in a dot-separated sequence of names. So the grammar for a namespace is:
<empty> | <name>[(<dot><name>)*]
In record, enum and fixed definitions, the fullname is determined in one of the following ways:
A name and namespace are both specified. For example, one might use “name”: “X”, “namespace”: “org.foo” to indicate the fullname org.foo.X.
-
A fullname is specified. If the name specified contains a dot, then it is assumed to be a fullname, and any namespace also specified is ignored. For example, use “name”: “org.foo.X” to indicate the fullname org.foo.X.
-
A name only is specified, i.e., a name that contains no dots. In this case the namespace is taken from the most tightly enclosing schema or protocol. For example, if “name”: “X” is specified, and this occurs within a field of the record definition of org.foo.Y, then the fullname is org.foo.X. If there is no enclosing namespace then the null namespace is used.
References to previously defined names are as in the latter two cases above: if they contain a dot they are a fullname, if they do not contain a dot, the namespace is the namespace of the enclosing definition.
Primitive type names have no namespace and their names may not be defined in any namespace.
A schema or protocol may not contain multiple definitions of a fullname. Further, a name must be defined before it is used (“before” in the depth-first, left-to-right traversal of the JSON parse tree, where the types attribute of a protocol is always deemed to come “before” the messages attribute.)
Aliases
Named types and fields may have aliases. An implementation may optionally use aliases to map a writer’s schema to the reader’s. This facilitates both schema evolution as well as processing disparate datasets.
Aliases function by re-writing the writer’s schema using aliases from the reader’s schema. For example, if the writer’s schema was named “Foo” and the reader’s schema is named “Bar” and has an alias of “Foo”, then the implementation would act as though “Foo” were named “Bar” when reading. Similarly, if data was written as a record with a field named “x” and is read as a record with a field named “y” with alias “x”, then the implementation would act as though “x” were named “y” when reading.
A type alias may be specified either as a fully namespace-qualified, or relative to the namespace of the name it is an alias for. For example, if a type named “a.b” has aliases of “c” and “x.y”, then the fully qualified names of its aliases are “a.c” and “x.y”.
Data Serialization and Deserialization
Binary encoded Avro data does not include type information or field names. The benefit is that the serialized data is small, but as a result a schema must always be used in order to read Avro data correctly. The best way to ensure that the schema is structurally identical to the one used to write the data is to use the exact same schema.
Therefore, files or systems that store Avro data should always include the writer’s schema for that data. Avro-based remote procedure call (RPC) systems must also guarantee that remote recipients of data have a copy of the schema used to write that data. In general, it is advisable that any reader of Avro data should use a schema that is the same (as defined more fully in Parsing Canonical Form for Schemas) as the schema that was used to write the data in order to deserialize it correctly. Deserializing data into a newer schema is accomplished by specifying an additional schema, the results of which are described in Schema Resolution.
In general, both serialization and deserialization proceed as a depth-first, left-to-right traversal of the schema, serializing or deserializing primitive types as they are encountered. Therefore, it is possible, though not advisable, to read Avro data with a schema that does not have the same Parsing Canonical Form as the schema with which the data was written. In order for this to work, the serialized primitive values must be compatible, in order value by value, with the items in the deserialization schema. For example, int and long are always serialized the same way, so an int could be deserialized as a long. Since the compatibility of two schemas depends on both the data and the serialization format (eg. binary is more permissive than JSON because JSON includes field names, eg. a long that is too large will overflow an int), it is simpler and more reliable to use schemas with identical Parsing Canonical Form.
Encodings
Avro specifies two serialization encodings: binary and JSON. Most applications will use the binary encoding, as it is smaller and faster. But, for debugging and web-based applications, the JSON encoding may sometimes be appropriate.
Binary Encoding
Binary encoding does not include field names, self-contained information about the types of individual bytes, nor field or record separators. Therefore readers are wholly reliant on the schema used when the data was encoded.
Primitive Types
Primitive types are encoded in binary as follows:
- null is written as zero bytes.
- a boolean is written as a single byte whose value is either 0 (false) or 1 (true).
- int and long values are written using variable-length zig-zag coding. Some examples:
| value |
hex |
| 0 |
00 |
| -1 |
01 |
| 1 |
02 |
| -2 |
03 |
| 2 |
04 |
| … |
|
| -64 |
7f |
| 64 |
80 01 |
| … |
|
- a float is written as 4 bytes. The float is converted into a 32-bit integer using a method equivalent to Java’s floatToIntBits and then encoded in little-endian format.
- a double is written as 8 bytes. The double is converted into a 64-bit integer using a method equivalent to Java’s doubleToLongBits and then encoded in little-endian format.
- bytes are encoded as a long followed by that many bytes of data.
- a string is encoded as a long followed by that many bytes of UTF-8 encoded character data.
For example, the three-character string “foo” would be encoded as the long value 3 (encoded as hex 06) followed by the UTF-8 encoding of ‘f’, ‘o’, and ‘o’ (the hex bytes 66 6f 6f):
06 66 6f 6f
Complex Types
Complex types are encoded in binary as follows:
Records
A record is encoded by encoding the values of its fields in the order that they are declared. In other words, a record is encoded as just the concatenation of the encodings of its fields. Field values are encoded per their schema.
For example, the record schema
{
"type": "record",
"name": "test",
"fields" : [
{"name": "a", "type": "long"},
{"name": "b", "type": "string"}
]
}
An instance of this record whose a field has value 27 (encoded as hex 36) and whose b field has value “foo” (encoded as hex bytes 06 66 6f 6f), would be encoded simply as the concatenation of these, namely the hex byte sequence:
36 06 66 6f 6f
Enums
An enum is encoded by a int, representing the zero-based position of the symbol in the schema.
For example, consider the enum:
{"type": "enum", "name": "Foo", "symbols": ["A", "B", "C", "D"] }
This would be encoded by an int between zero and three, with zero indicating “A”, and 3 indicating “D”.
Arrays
Arrays are encoded as a series of blocks. Each block consists of a long count value, followed by that many array items. A block with count zero indicates the end of the array. Each item is encoded per the array’s item schema.
If a block’s count is negative, its absolute value is used, and the count is followed immediately by a long block size indicating the number of bytes in the block. This block size permits fast skipping through data, e.g., when projecting a record to a subset of its fields.
For example, the array schema
{"type": "array", "items": "long"}
an array containing the items 3 and 27 could be encoded as the long value 2 (encoded as hex 04) followed by long values 3 and 27 (encoded as hex 06 36) terminated by zero:
04 06 36 00
The blocked representation permits one to read and write arrays larger than can be buffered in memory, since one can start writing items without knowing the full length of the array.
Maps
Maps are encoded as a series of blocks. Each block consists of a long count value, followed by that many key/value pairs. A block with count zero indicates the end of the map. Each item is encoded per the map’s value schema.
If a block’s count is negative, its absolute value is used, and the count is followed immediately by a long block size indicating the number of bytes in the block. This block size permits fast skipping through data, e.g., when projecting a record to a subset of its fields.
The blocked representation permits one to read and write maps larger than can be buffered in memory, since one can start writing items without knowing the full length of the map.
Unions
A union is encoded by first writing an int value indicating the zero-based position within the union of the schema of its value. The value is then encoded per the indicated schema within the union.
For example, the union schema ["null","string"] would encode:
-
null as zero (the index of “null” in the union):
00
-
the string “a” as one (the index of “string” in the union, 1, encoded as hex 02), followed by the serialized string:
02 02 61
NOTE: Currently for C/C++ implementations, the positions are practically an int, but theoretically a long. In reality, we don’t expect unions with 215M members
Fixed
Fixed instances are encoded using the number of bytes declared in the schema.
JSON Encoding
Except for unions, the JSON encoding is the same as is used to encode field default values.
The value of a union is encoded in JSON as follows:
- if its type is null, then it is encoded as a JSON null;
- otherwise it is encoded as a JSON object with one name/value pair whose name is the type’s name and whose value is the recursively encoded value. For Avro’s named types (record, fixed or enum) the user-specified name is used, for other types the type name is used.
For example, the union schema ["null","string","Foo"], where Foo is a record name, would encode:
- null as null;
- the string “a” as
{"string": "a"} and
- a Foo instance as
{"Foo": {...}}, where {...} indicates the JSON encoding of a Foo instance.
Note that the original schema is still required to correctly process JSON-encoded data. For example, the JSON encoding does not distinguish between int and long, float and double, records and maps, enums and strings, etc.
Single-object encoding
In some situations a single Avro serialized object is to be stored for a longer period of time. One very common example is storing Avro records for several weeks in an Apache Kafka topic.
In the period after a schema change this persistence system will contain records that have been written with different schemas. So the need arises to know which schema was used to write a record to support schema evolution correctly. In most cases the schema itself is too large to include in the message, so this binary wrapper format supports the use case more effectively.
Single object encoding specification
Single Avro objects are encoded as follows:
- A two-byte marker,
C3 01, to show that the message is Avro and uses this single-record format (version 1).
- The 8-byte little-endian CRC-64-AVRO fingerprint of the object’s schema.
- The Avro object encoded using Avro’s binary encoding.
Implementations use the 2-byte marker to determine whether a payload is Avro. This check helps avoid expensive lookups that resolve the schema from a fingerprint, when the message is not an encoded Avro payload.
Sort Order
Avro defines a standard sort order for data. This permits data written by one system to be efficiently sorted by another system. This can be an important optimization, as sort order comparisons are sometimes the most frequent per-object operation. Note also that Avro binary-encoded data can be efficiently ordered without deserializing it to objects.
Data items may only be compared if they have identical schemas. Pairwise comparisons are implemented recursively with a depth-first, left-to-right traversal of the schema. The first mismatch encountered determines the order of the items.
Two items with the same schema are compared according to the following rules.
- null data is always equal.
- boolean data is ordered with false before true.
- int, long, float and double data is ordered by ascending numeric value.
- bytes and fixed data are compared lexicographically by unsigned 8-bit values.
- string data is compared lexicographically by Unicode code point. Note that since UTF-8 is used as the binary encoding for strings, sorting of bytes and string binary data is identical.
- array data is compared lexicographically by element.
- enum data is ordered by the symbol’s position in the enum schema. For example, an enum whose symbols are
["z", "a"] would sort “z” values before “a” values.
- union data is first ordered by the branch within the union, and, within that, by the type of the branch. For example, an
["int", "string"] union would order all int values before all string values, with the ints and strings themselves ordered as defined above.
- record data is ordered lexicographically by field. If a field specifies that its order is:
- “ascending”, then the order of its values is unaltered.
- “descending”, then the order of its values is reversed.
- “ignore”, then its values are ignored when sorting.
- map data may not be compared. It is an error to attempt to compare data containing maps unless those maps are in an
"order":"ignore" record field.
Object Container Files
Avro includes a simple object container file format. A file has a schema, and all objects stored in the file must be written according to that schema, using binary encoding. Objects are stored in blocks that may be compressed. Syncronization markers are used between blocks to permit efficient splitting of files for MapReduce processing.
Files may include arbitrary user-specified metadata.
A file consists of:
- A file header, followed by
- one or more file data blocks.
A file header consists of:
- Four bytes, ASCII ‘O’, ‘b’, ‘j’, followed by 1.
- file metadata, including the schema.
- The 16-byte, randomly-generated sync marker for this file.
File metadata is written as if defined by the following map schema:
{"type": "map", "values": "bytes"}
All metadata properties that start with “avro.” are reserved. The following file metadata properties are currently used:
- avro.schema contains the schema of objects stored in the file, as JSON data (required).
- avro.codec the name of the compression codec used to compress blocks, as a string. Implementations are required to support the following codecs: “null” and “deflate”. If codec is absent, it is assumed to be “null”. The codecs are described with more detail below.
A file header is thus described by the following schema:
{"type": "record", "name": "org.apache.avro.file.Header",
"fields" : [
{"name": "magic", "type": {"type": "fixed", "name": "Magic", "size": 4}},
{"name": "meta", "type": {"type": "map", "values": "bytes"}},
{"name": "sync", "type": {"type": "fixed", "name": "Sync", "size": 16}}
]
}
A file data block consists of:
- A long indicating the count of objects in this block.
- A long indicating the size in bytes of the serialized objects in the current block, after any codec is applied
- The serialized objects. If a codec is specified, this is compressed by that codec.
- The file’s 16-byte sync marker.
Thus, each block’s binary data can be efficiently extracted or skipped without deserializing the contents. The combination of block size, object counts, and sync markers enable detection of corrupt blocks and help ensure data integrity.
Required Codecs
null
The “null” codec simply passes through data uncompressed.
deflate
The “deflate” codec writes the data block using the deflate algorithm as specified in RFC 1951, and typically implemented using the zlib library. Note that this format (unlike the “zlib format” in RFC 1950) does not have a checksum.
Optional Codecs
bzip2
The “bzip2” codec uses the bzip2 compression library.
snappy
The “snappy” codec uses Google’s Snappy compression library. Each compressed block is followed by the 4-byte, big-endian CRC32 checksum of the uncompressed data in the block.
xz
The “xz” codec uses the XZ compression library.
zstandard
The “zstandard” codec uses Facebook’s Zstandard compression library.
Protocol Declaration
Avro protocols describe RPC interfaces. Like schemas, they are defined with JSON text.
A protocol is a JSON object with the following attributes:
- protocol, a string, the name of the protocol (required);
- namespace, an optional string that qualifies the name;
- doc, an optional string describing this protocol;
- types, an optional list of definitions of named types (records, enums, fixed and errors). An error definition is just like a record definition except it uses “error” instead of “record”. Note that forward references to named types are not permitted.
- messages, an optional JSON object whose keys are message names and whose values are objects whose attributes are described below. No two messages may have the same name.
The name and namespace qualification rules defined for schema objects apply to protocols as well.
Messages
A message has attributes:
- a doc, an optional description of the message,
- a request, a list of named, typed parameter schemas (this has the same form as the fields of a record declaration);
- a response schema;
- an optional union of declared error schemas. The effective union has “string” prepended to the declared union, to permit transmission of undeclared “system” errors. For example, if the declared error union is
["AccessError"], then the effective union is ["string", "AccessError"]. When no errors are declared, the effective error union is ["string"]. Errors are serialized using the effective union; however, a protocol’s JSON declaration contains only the declared union.
- an optional one-way boolean parameter.
A request parameter list is processed equivalently to an anonymous record. Since record field lists may vary between reader and writer, request parameters may also differ between the caller and responder, and such differences are resolved in the same manner as record field differences.
The one-way parameter may only be true when the response type is "null" and no errors are listed.
Sample Protocol
For example, one may define a simple HelloWorld protocol with:
{
"namespace": "com.acme",
"protocol": "HelloWorld",
"doc": "Protocol Greetings",
"types": [
{"name": "Greeting", "type": "record", "fields": [
{"name": "message", "type": "string"}]},
{"name": "Curse", "type": "error", "fields": [
{"name": "message", "type": "string"}]}
],
"messages": {
"hello": {
"doc": "Say hello.",
"request": [{"name": "greeting", "type": "Greeting" }],
"response": "Greeting",
"errors": ["Curse"]
}
}
}
Message Transport
Messages may be transmitted via different transport mechanisms.
To the transport, a message is an opaque byte sequence.
A transport is a system that supports:
- transmission of request messages
- receipt of corresponding response messages
Servers may send a response message back to the client corresponding to a request message. The mechanism of correspondence is transport-specific. For example, in HTTP it is implicit, since HTTP directly supports requests and responses. But a transport that multiplexes many client threads over a single socket would need to tag messages with unique identifiers.
Transports may be either stateless or stateful. In a stateless transport, messaging assumes no established connection state, while stateful transports establish connections that may be used for multiple messages. This distinction is discussed further in the handshake section below.
HTTP as Transport
When HTTP is used as a transport, each Avro message exchange is an HTTP request/response pair. All messages of an Avro protocol should share a single URL at an HTTP server. Other protocols may also use that URL. Both normal and error Avro response messages should use the 200 (OK) response code. The chunked encoding may be used for requests and responses, but, regardless the Avro request and response are the entire content of an HTTP request and response. The HTTP Content-Type of requests and responses should be specified as “avro/binary”. Requests should be made using the POST method.
HTTP is used by Avro as a stateless transport.
Message Framing
Avro messages are framed as a list of buffers.
Framing is a layer between messages and the transport. It exists to optimize certain operations.
The format of framed message data is:
- a series of buffers, where each buffer consists of:
- a four-byte, big-endian buffer length, followed by
- that many bytes of buffer data.
- a message is always terminated by a zero-length buffer.
Framing is transparent to request and response message formats (described below). Any message may be presented as a single or multiple buffers.
Framing can permit readers to more efficiently get different buffers from different sources and for writers to more efficiently store different buffers to different destinations. In particular, it can reduce the number of times large binary objects are copied. For example, if an RPC parameter consists of a megabyte of file data, that data can be copied directly to a socket from a file descriptor, and, on the other end, it could be written directly to a file descriptor, never entering user space.
A simple, recommended, framing policy is for writers to create a new segment whenever a single binary object is written that is larger than a normal output buffer. Small objects are then appended in buffers, while larger objects are written as their own buffers. When a reader then tries to read a large object the runtime can hand it an entire buffer directly, without having to copy it.
Handshake
The purpose of the handshake is to ensure that the client and the server have each other’s protocol definition, so that the client can correctly deserialize responses, and the server can correctly deserialize requests. Both clients and servers should maintain a cache of recently seen protocols, so that, in most cases, a handshake will be completed without extra round-trip network exchanges or the transmission of full protocol text.
RPC requests and responses may not be processed until a handshake has been completed. With a stateless transport, all requests and responses are prefixed by handshakes. With a stateful transport, handshakes are only attached to requests and responses until a successful handshake response has been returned over a connection. After this, request and response payloads are sent without handshakes for the lifetime of that connection.
The handshake process uses the following record schemas:
{
"type": "record",
"name": "HandshakeRequest", "namespace":"org.apache.avro.ipc",
"fields": [
{"name": "clientHash",
"type": {"type": "fixed", "name": "MD5", "size": 16}},
{"name": "clientProtocol", "type": ["null", "string"]},
{"name": "serverHash", "type": "MD5"},
{"name": "meta", "type": ["null", {"type": "map", "values": "bytes"}]}
]
}
{
"type": "record",
"name": "HandshakeResponse", "namespace": "org.apache.avro.ipc",
"fields": [
{"name": "match",
"type": {"type": "enum", "name": "HandshakeMatch",
"symbols": ["BOTH", "CLIENT", "NONE"]}},
{"name": "serverProtocol",
"type": ["null", "string"]},
{"name": "serverHash",
"type": ["null", {"type": "fixed", "name": "MD5", "size": 16}]},
{"name": "meta",
"type": ["null", {"type": "map", "values": "bytes"}]}
]
}
- A client first prefixes each request with a
HandshakeRequest containing just the hash of its protocol and of the server’s protocol (clientHash!=null, clientProtocol=null, serverHash!=null), where the hashes are 128-bit MD5 hashes of the JSON protocol text. If a client has never connected to a given server, it sends its hash as a guess of the server’s hash, otherwise it sends the hash that it previously obtained from this server.
The server responds with a HandshakeResponse containing one of:
-
match=BOTH, serverProtocol=null, serverHash=null if the client sent the valid hash of the server’s protocol and the server knows what protocol corresponds to the client’s hash. In this case, the request is complete and the response data immediately follows the HandshakeResponse.
-
match=CLIENT, serverProtocol!=null, serverHash!=null if the server has previously seen the client’s protocol, but the client sent an incorrect hash of the server’s protocol. The request is complete and the response data immediately follows the HandshakeResponse. The client must use the returned protocol to process the response and should also cache that protocol and its hash for future interactions with this server.
-
match=NONE if the server has not previously seen the client’s protocol. The serverHash and serverProtocol may also be non-null if the server’s protocol hash was incorrect.
In this case the client must then re-submit its request with its protocol text (clientHash!=null, clientProtocol!=null, serverHash!=null) and the server should respond with a successful match (match=BOTH, serverProtocol=null, serverHash=null) as above.
The meta field is reserved for future handshake enhancements.
A call consists of a request message paired with its resulting response or error message. Requests and responses contain extensible metadata, and both kinds of messages are framed as described above.
The format of a call request is:
- request metadata, a map with values of type bytes
- the message name, an Avro string, followed by
- the message parameters. Parameters are serialized according to the message’s request declaration.
When the empty string is used as a message name a server should ignore the parameters and return an empty response. A client may use this to ping a server or to perform a handshake without sending a protocol message.
When a message is declared one-way and a stateful connection has been established by a successful handshake response, no response data is sent. Otherwise the format of the call response is:
- response metadata, a map with values of type bytes
- a one-byte error flag boolean, followed by either:
- if the error flag is false, the message response, serialized per the message’s response schema.
- if the error flag is true, the error, serialized per the message’s effective error union schema.
Schema Resolution
A reader of Avro data, whether from an RPC or a file, can always parse that data because the original schema must be provided along with the data. However, the reader may be programmed to read data into a different schema. For example, if the data was written with a different version of the software than it is read, then fields may have been added or removed from records. This section specifies how such schema differences should be resolved.
We refer to the schema used to write the data as the writer’s schema, and the schema that the application expects the reader’s schema. Differences between these should be resolved as follows:
- It is an error if the two schemas do not match.
To match, one of the following must hold:
-
both schemas are arrays whose item types match
-
both schemas are maps whose value types match
-
both schemas are enums whose (unqualified) names match
-
both schemas are fixed whose sizes and (unqualified) names match
-
both schemas are records with the same (unqualified) name
-
either schema is a union
-
both schemas have same primitive type
-
the writer’s schema may be promoted to the reader’s as follows:
- int is promotable to long, float, or double
- long is promotable to float or double
- float is promotable to double
- string is promotable to bytes
- bytes is promotable to string
-
if both are records:
- the ordering of fields may be different: fields are matched by name.
- schemas for fields with the same name in both records are resolved recursively.
- if the writer’s record contains a field with a name not present in the reader’s record, the writer’s value for that field is ignored.
- if the reader’s record schema has a field that contains a default value, and writer’s schema does not have a field with the same name, then the reader should use the default value from its field.
- if the reader’s record schema has a field with no default value, and writer’s schema does not have a field with the same name, an error is signalled.
-
if both are enums:
if the writer’s symbol is not present in the reader’s enum and the reader has a default value, then that value is used, otherwise an error is signalled.
-
if both are arrays:
This resolution algorithm is applied recursively to the reader’s and writer’s array item schemas.
-
if both are maps:
This resolution algorithm is applied recursively to the reader’s and writer’s value schemas.
-
if both are unions:
The first schema in the reader’s union that matches the selected writer’s union schema is recursively resolved against it. if none match, an error is signalled.
-
if reader’s is a union, but writer’s is not
The first schema in the reader’s union that matches the writer’s schema is recursively resolved against it. If none match, an error is signalled.
-
if writer’s is a union, but reader’s is not
If the reader’s schema matches the selected writer’s schema, it is recursively resolved against it. If they do not match, an error is signalled.
A schema’s doc fields are ignored for the purposes of schema resolution. Hence, the doc portion of a schema may be dropped at serialization.
One of the defining characteristics of Avro is that a reader must use the schema used by the writer of the data in order to know how to read the data. This assumption results in a data format that’s compact and also amenable to many forms of schema evolution. However, the specification so far has not defined what it means for the reader to have the “same” schema as the writer. Does the schema need to be textually identical? Well, clearly adding or removing some whitespace to a JSON expression does not change its meaning. At the same time, reordering the fields of records clearly does change the meaning. So what does it mean for a reader to have “the same” schema as a writer?
Parsing Canonical Form is a transformation of a writer’s schema that let’s us define what it means for two schemas to be “the same” for the purpose of reading data written against the schema. It is called Parsing Canonical Form because the transformations strip away parts of the schema, like “doc” attributes, that are irrelevant to readers trying to parse incoming data. It is called Canonical Form because the transformations normalize the JSON text (such as the order of attributes) in a way that eliminates unimportant differences between schemas. If the Parsing Canonical Forms of two different schemas are textually equal, then those schemas are “the same” as far as any reader is concerned, i.e., there is no serialized data that would allow a reader to distinguish data generated by a writer using one of the original schemas from data generated by a writing using the other original schema. (We sketch a proof of this property in a companion document.)
The next subsection specifies the transformations that define Parsing Canonical Form. But with a well-defined canonical form, it can be convenient to go one step further, transforming these canonical forms into simple integers (“fingerprints”) that can be used to uniquely identify schemas. The subsection after next recommends some standard practices for generating such fingerprints.
Assuming an input schema (in JSON form) that’s already UTF-8 text for a valid Avro schema (including all quotes as required by JSON), the following transformations will produce its Parsing Canonical Form:
- [PRIMITIVES] Convert primitive schemas to their simple form (e.g., int instead of
{"type":"int"}).
- [FULLNAMES] Replace short names with fullnames, using applicable namespaces to do so. Then eliminate namespace attributes, which are now redundant.
- [STRIP] Keep only attributes that are relevant to parsing data, which are: type, name, fields, symbols, items, values, size. Strip all others (e.g., doc and aliases).
- [ORDER] Order the appearance of fields of JSON objects as follows: name, type, fields, symbols, items, values, size. For example, if an object has type, name, and size fields, then the name field should appear first, followed by the type and then the size fields.
- [STRINGS] For all JSON string literals in the schema text, replace any escaped characters (e.g., \uXXXX escapes) with their UTF-8 equivalents.
- [INTEGERS] Eliminate quotes around and any leading zeros in front of JSON integer literals (which appear in the size attributes of fixed schemas).
- [WHITESPACE] Eliminate all whitespace in JSON outside of string literals.
Schema Fingerprints
“[A] fingerprinting algorithm is a procedure that maps an arbitrarily large data item (such as a computer file) to a much shorter bit string, its fingerprint, that uniquely identifies the original data for all practical purposes” (quoted from Wikipedia). In the Avro context, fingerprints of Parsing Canonical Form can be useful in a number of applications; for example, to cache encoder and decoder objects, to tag data items with a short substitute for the writer’s full schema, and to quickly negotiate common-case schemas between readers and writers.
In designing fingerprinting algorithms, there is a fundamental trade-off between the length of the fingerprint and the probability of collisions. To help application designers find appropriate points within this trade-off space, while encouraging interoperability and ease of implementation, we recommend using one of the following three algorithms when fingerprinting Avro schemas:
- When applications can tolerate longer fingerprints, we recommend using the SHA-256 digest algorithm to generate 256-bit fingerprints of Parsing Canonical Forms. Most languages today have SHA-256 implementations in their libraries.
- At the opposite extreme, the smallest fingerprint we recommend is a 64-bit Rabin fingerprint. Below, we provide pseudo-code for this algorithm that can be easily translated into any programming language. 64-bit fingerprints should guarantee uniqueness for schema caches of up to a million entries (for such a cache, the chance of a collision is 3E-8). We don’t recommend shorter fingerprints, as the chances of collisions is too great (for example, with 32-bit fingerprints, a cache with as few as 100,000 schemas has a 50% chance of having a collision).
- Between these two extremes, we recommend using the MD5 message digest to generate 128-bit fingerprints. These make sense only where very large numbers of schemas are being manipulated (tens of millions); otherwise, 64-bit fingerprints should be sufficient. As with SHA-256, MD5 implementations are found in most libraries today.
These fingerprints are not meant to provide any security guarantees, even the longer SHA-256-based ones. Most Avro applications should be surrounded by security measures that prevent attackers from writing random data and otherwise interfering with the consumers of schemas. We recommend that these surrounding mechanisms be used to prevent collision and pre-image attacks (i.e., “forgery”) on schema fingerprints, rather than relying on the security properties of the fingerprints themselves.
Rabin fingerprints are cyclic redundancy checks computed using irreducible polynomials. In the style of the Appendix of RFC 1952 (pg 10), which defines the CRC-32 algorithm, here’s our definition of the 64-bit AVRO fingerprinting algorithm:
long fingerprint64(byte[] buf) {
if (FP_TABLE == null) initFPTable();
long fp = EMPTY;
for (int i = 0; i < buf.length; i++)
fp = (fp >>> 8) ^ FP_TABLE[(int)(fp ^ buf[i]) & 0xff];
return fp;
}
static long EMPTY = 0xc15d213aa4d7a795L;
static long[] FP_TABLE = null;
void initFPTable() {
FP_TABLE = new long[256];
for (int i = 0; i < 256; i++) {
long fp = i;
for (int j = 0; j < 8; j++)
fp = (fp >>> 1) ^ (EMPTY & -(fp & 1L));
FP_TABLE[i] = fp;
}
}
Readers interested in the mathematics behind this algorithm may want to read Chapter 14 of the Second Edition of Hacker’s Delight. (Unlike RFC-1952 and the book chapter, we prepend a single one bit to messages. We do this because CRCs ignore leading zero bits, which can be problematic. Our code prepends a one-bit by initializing fingerprints using EMPTY, rather than initializing using zero as in RFC-1952 and the book chapter.)
Logical Types
A logical type is an Avro primitive or complex type with extra attributes to represent a derived type. The attribute logicalType must always be present for a logical type, and is a string with the name of one of the logical types listed later in this section. Other attributes may be defined for particular logical types.
A logical type is always serialized using its underlying Avro type so that values are encoded in exactly the same way as the equivalent Avro type that does not have a logicalType attribute. Language implementations may choose to represent logical types with an appropriate native type, although this is not required.
Language implementations must ignore unknown logical types when reading, and should use the underlying Avro type. If a logical type is invalid, for example a decimal with scale greater than its precision, then implementations should ignore the logical type and use the underlying Avro type.
Decimal
The decimal logical type represents an arbitrary-precision signed decimal number of the form unscaled × 10-scale.
A decimal logical type annotates Avro bytes or fixed types. The byte array must contain the two’s-complement representation of the unscaled integer value in big-endian byte order. The scale is fixed, and is specified using an attribute.
The following attributes are supported:
- scale, a JSON integer representing the scale (optional). If not specified the scale is 0.
- precision, a JSON integer representing the (maximum) precision of decimals stored in this type (required).
For example, the following schema represents decimal numbers with a maximum precision of 4 and a scale of 2:
{
"type": "bytes",
"logicalType": "decimal",
"precision": 4,
"scale": 2
}
Precision must be a positive integer greater than zero. If the underlying type is a fixed, then the precision is limited by its size. An array of length n can store at most floor(log10(28 × n - 1 - 1)) base-10 digits of precision.
Scale must be zero or a positive integer less than or equal to the precision.
For the purposes of schema resolution, two schemas that are decimal logical types match if their scales and precisions match.
UUID
The uuid logical type represents a random generated universally unique identifier (UUID).
A uuid logical type annotates an Avro string. The string has to conform with RFC-4122
Date
The date logical type represents a date within the calendar, with no reference to a particular time zone or time of day.
A date logical type annotates an Avro int, where the int stores the number of days from the unix epoch, 1 January 1970 (ISO calendar).
The following schema represents a date:
{
"type": "int",
"logicalType": "date"
}
Time (millisecond precision)
The time-millis logical type represents a time of day, with no reference to a particular calendar, time zone or date, with a precision of one millisecond.
A time-millis logical type annotates an Avro int, where the int stores the number of milliseconds after midnight, 00:00:00.000.
Time (microsecond precision)
The time-micros logical type represents a time of day, with no reference to a particular calendar, time zone or date, with a precision of one microsecond.
A time-micros logical type annotates an Avro long, where the long stores the number of microseconds after midnight, 00:00:00.000000.
Timestamp (millisecond precision)
The timestamp-millis logical type represents an instant on the global timeline, independent of a particular time zone or calendar, with a precision of one millisecond. Please note that time zone information gets lost in this process. Upon reading a value back, we can only reconstruct the instant, but not the original representation. In practice, such timestamps are typically displayed to users in their local time zones, therefore they may be displayed differently depending on the execution environment.
A timestamp-millis logical type annotates an Avro long, where the long stores the number of milliseconds from the unix epoch, 1 January 1970 00:00:00.000 UTC.
Timestamp (microsecond precision)
The timestamp-micros logical type represents an instant on the global timeline, independent of a particular time zone or calendar, with a precision of one microsecond. Please note that time zone information gets lost in this process. Upon reading a value back, we can only reconstruct the instant, but not the original representation. In practice, such timestamps are typically displayed to users in their local time zones, therefore they may be displayed differently depending on the execution environment.
A timestamp-micros logical type annotates an Avro long, where the long stores the number of microseconds from the unix epoch, 1 January 1970 00:00:00.000000 UTC.
Local timestamp (millisecond precision)
The local-timestamp-millis logical type represents a timestamp in a local timezone, regardless of what specific time zone is considered local, with a precision of one millisecond.
A local-timestamp-millis logical type annotates an Avro long, where the long stores the number of milliseconds, from 1 January 1970 00:00:00.000.
Local timestamp (microsecond precision)
The local-timestamp-micros logical type represents a timestamp in a local timezone, regardless of what specific time zone is considered local, with a precision of one microsecond.
A local-timestamp-micros logical type annotates an Avro long, where the long stores the number of microseconds, from 1 January 1970 00:00:00.000000.
Duration
The duration logical type represents an amount of time defined by a number of months, days and milliseconds. This is not equivalent to a number of milliseconds, because, depending on the moment in time from which the duration is measured, the number of days in the month and number of milliseconds in a day may differ. Other standard periods such as years, quarters, hours and minutes can be expressed through these basic periods.
A duration logical type annotates Avro fixed type of size 12, which stores three little-endian unsigned integers that represent durations at different granularities of time. The first stores a number in months, the second stores a number in days, and the third stores a number in milliseconds.
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
4.4 - Trevni
Version 0.1
DRAFT
This document is the authoritative specification of a file format. Its intent is to permit compatible, independent implementations that read and/or write files in this format.
Introduction
Data sets are often described as a table composed of rows and columns. Each record in the dataset is considered a row, with each field of the record occupying a different column. Writing records to a file one-by-one as they are created results in a row-major format, like Hadoop’s SequenceFile or Avro data files.
In many cases higher query performance may be achieved if the data is instead organized in a column-major format, where multiple values of a given column are stored adjacently. This document defines such a column-major file format for datasets.
To permit scalable, distributed query evaluation, datasets are partitioned into row groups, containing distinct collections of rows. Each row group is organized in column-major order, while row groups form a row-major partitioning of the entire dataset.
Rationale
Goals
The format is meant satisfy the following goals:
Maximize the size of row groups. Disc drives are used most efficiently when sequentially accessing data. Consider a drive that takes 10ms to seek and transfers at 100MB/second. If a 10-column dataset whose values are all the same size is split into 10MB row groups, then accessing a single column will require a sequence of seek+1MB reads, for a cost of 20ms/MB processed. If the same dataset is split into 100MB row groups then this drops to 11ms/MB processed. This effect is exaggerated for datasets with larger numbers of columns and with columns whose values are smaller than average. So we’d prefer row groups that are 100MB or greater.
Permit random access within a row group. Some queries will first examine one column, and, only when certain relatively rare criteria are met, examine other columns. Rather than iterating through selected columns of the row-group in parallel, one might iterate through one column and randomly access another. This is called support for WHERE clauses, after the SQL operator of that name.
Minimize the number of files per dataset. HDFS is a primary intended deployment platform for these files. The HDFS Namenode requires memory for each file in the filesystem, thus for a format to be HDFS-friendly it should strive to require the minimum number of distinct files.
Support co-location of columns within row-groups. Row groups are the unit of parallel operation on a column dataset. For efficient file i/o, the entirety of a row-group should ideally reside on the host that is evaluating the query in order to avoid network latencies and bottlenecks.
Data integrity. The format should permit applications to detect data corruption. Many file systems may prevent corruption, but files may be moved between filesystems and be subject to corruption at points in that process. It is best if the data in a file can be validated independently.
Extensibility. The format should permit applications to store additional annotations about a datasets in the files, such as type information, origin, etc. Some environments may have metadata stores for such information, but not all do, and files might be moved among systems with different metadata systems. The ability to keep such information within the file simplifies the coordination of such information.
Minimal overhead. The column format should not make datasets appreciably larger. Storage is a primary cost and a choice to use this format should not require additional storage.
Primary format. The column format should be usable as a primary format for datasets, not as an auxiliary, accelerated format. Applications that process a dataset in row-major order should be able to easily consume column files and applications that produce datasets in row-major order should be able to easily generate column files.
Design
To meet these goals we propose the following design.
Each row group is a separate file. All values of a column in a file are written contiguously. This maximizes the row group size, optimizing performance when querying few and small columns.
Each file occupies a single HDFS block. A larger than normal block size may be specified, e.g., ~1GB instead of the typical ~100MB. This guarantees co-location and eliminates network use when query processing can be co-located with the file. This also moderates the memory impact on the HDFS Namenode since no small files are written.
Each column in a file is written as a sequence of ~64kB compressed blocks. The sequence is prefixed by a table describing all of the blocks in the column to permit random access within the column.
Application-specific metadata may be added at the file, column, and block levels.
Checksums are included with each block, providing data integrity.
Discussion
The use of a single block per file achieves the same effect as the custom block placement policy described in the CIF paper, but while still permitting HDFS rebalancing and not increasing the number of files in the namespace.
This section formally describes the proposed column file format.
Data Model
We assume a simple data model, where a record is a set of named fields, and the value of each field is a sequence of untyped bytes. A type system may be layered on top of this, as specified in the Type Mapping section below.
Primitive Values
We define the following primitive value types:
- Signed 64-bit long values are written using a variable-length zig-zag coding, where the high-order bit in each byte determines whether subsequent bytes are present. For example:
decimal value hex bytes
| decimal value |
hex bytes |
|
| 0 |
00 |
|
| -1 |
01 |
|
| 1 |
02 |
|
| … |
|
|
| -64 |
7f |
|
| 64 |
80 01 |
|
| … |
|
|
- bytes are encoded as a long followed by that many bytes of data.
- a string is encoded as a long followed by that many bytes of UTF-8 encoded character data.
For example, the three-character string “foo” would be encoded as the long value 3 (encoded as hex 06) followed by the UTF-8 encoding of ‘f’, ‘o’, and ‘o’ (the hex bytes 66 6f 6f): 06 66 6f 6f
Type Names
The following type names are used to describe column values:
- null, requires zero bytes. Sometimes used in array columns.
- boolean, one bit, packed into bytes, little-endian;
- int, like long, but restricted to 32-bit signed values
- long 64-bit signed values, represented as above
- fixed32 32-bit values stored as four bytes, little-endian.
- fixed64 64-bit values stored as eight bytes, little-endian.
- float 32-bit IEEE floating point value, little-endian
- double 64-bit IEEE floating point value, little-endian
- string as above
- bytes as above, may be used to encapsulate more complex objects
Type names are represented as strings (UTF-8 encoded, length-prefixed).
Metadata consists of:
- A long indicating the number of metadata key/value pairs.
- For each pair, a string key and bytes value.
All metadata properties that start with “trevni.” are reserved.
The following file metadata properties are defined:
- trevni.codec the name of the default compression codec used to compress blocks, as a string. Implementations are required to support the “null” codec. Optional. If absent, it is assumed to be “null”. Codecs are described in more detail below.
- trevni.checksum the name of the checksum algorithm used in this file, as a string. Implementations are required to support the “crc-32” checksum. Optional. If absent, it is assumed to be “null”. Checksums are described in more detail below.
The following column metadata properties are defined:
- trevni.codec the name of the compression codec used to compress the blocks of this column, as a string. Implementations are required to support the “null” codec. Optional. If absent, it is assumed to be “null”. Codecs are described in more detail below.
- trevni.name the name of the column, as a string. Required.
- trevni.type the type of data in the column. One of the type names above. Required.
- trevni.values if present, indicates that the initial value of each block in this column will be stored in the block’s descriptor. Not permitted for array columns or columns that specify a parent.
- trevni.array if present, indicates that each row in this column contains a sequence of values of the named type rather than just a single value. An integer length precedes each sequence of values indicating the count of values in the sequence. If the length is negative then it indicates a sequence of zero or one lengths, where -1 indicates two zeros, -2 two ones, -3 three zeros, -4 three ones, etc.
- trevni.parent if present, the name of an array column whose lengths are also used by this column. Thus values of this column are sequences but no lengths are stored in this column.
For example, consider the following row, as JSON, where all values are primitive types, but one has multiple values.
{"id"=566, "date"=23423234234
"from"="foo@bar.com",
"to"=["bar@baz.com", "bang@foo.com"],
"content"="Hi!"}
The columns for this might be specified as:
name=id type=int
name=date type=long
name=from type=string
name=to type=string array=true
name=content type=string
If a row contains an array of records, e.g. “received” in the following:
{"id"=566, "date"=23423234234
"from"="foo@bar.com",
"to"=["bar@baz.com", "bang@foo.com"],
"content"="Hi!"
"received"=[{"date"=234234234234, "host"="192.168.0.0.1"},
{"date"=234234545645, "host"="192.168.0.0.2"}]
}
Then one can define a parent column followed by a column for each field in the record, adding the following columns:
name=received type=null array=true
name=date type=long parent=received
name=host type=string parent=received
If an array value itself contains an array, e.g. the “sigs” below:
{"id"=566, "date"=23423234234
"from"="foo@bar.com",
"to"=["bar@baz.com", "bang@foo.com"],
"content"="Hi!"
"received"=[{"date"=234234234234, "host"="192.168.0.0.1",
"sigs"=[{"algo"="weak", "value"="0af345de"}]},
{"date"=234234545645, "host"="192.168.0.0.2",
"sigs"=[]}]
}
Then a parent column may be defined that itself has a parent column.
name=sigs type=null array=true parent=received
name=algo type=string parent=sigs
name=value type=string parent=sigs
No block metadata properties are currently defined.
A file consists of:
- A file header, followed by
- one or more columns.
A file header consists of:
- Four bytes, ASCII ‘T’, ‘r’, ‘v’, followed by 0x02.
- a fixed64 indicating the number of rows in the file
- a fixed32 indicating the number of columns in the file
- file metadata.
- for each column, its column metadata
- for each column, its starting position in the file as a fixed64.
A column consists of:
- A fixed32 indicating the number of blocks in this column.
- For each block, a block descriptor
- One or more blocks.
A block descriptor consists of:
- A fixed32 indicating the number of rows in the block
- A fixed32 indicating the size in bytes of the block before the codec is applied (excluding checksum).
- A fixed32 indicating the size in bytes of the block after the codec is applied (excluding checksum).
- If this column’s metadata declares it to include values, the first value in the column, serialized according to this column’s type.
A block consists of:
- The serialized column values. If a column is an array column then value sequences are preceded by their length, as an int. If a codec is specified, the values and lengths are compressed by that codec.
- The checksum, as determined by the file metadata.
Codecs
null
The “null” codec simply passes data through uncompressed.
deflate
The “deflate” codec writes the data block using the deflate algorithm as specified in RFC 1951.
snappy
The “snappy” codec uses Google’s Snappy compression library.
Checksum algorithms
null
The “null” checksum contains zero bytes.
crc-32
Each “crc-32” checksum contains the four bytes of an ISO 3309 CRC-32 checksum of the uncompressed block data as a fixed32.
Type Mappings
We define a standard mapping for how types defined in various serialization systems are represented in a column file. Records from these systems are shredded into columns. When records are nested, a depth-first recursive walk can assign a separate column for each primitive value.
Avro
Protocol Buffers
Thrift
Implementation Notes
Some possible techniques for writing column files include:
- Use a standard ~100MB block, buffer in memory up to the block size, then flush the file directly to HDFS. A single reduce task might create multiple output files. The namenode requires memory proportional to the number of names and blocks*replication. This would increase the number of names but not blocks, so this should still be much better than a file per column.
- Spill each column to a separate local, temporary file then, when the file is closed, append these files, writing a single file to HDFS whose block size is set to be that of the entire file. This would be a bit slower than and may have trouble when the local disk is full, but it would better use HDFS namespace and further reduce seeks when processing columns whose values are small.
- Use a separate mapreduce job to convert row-major files to column-major. The map output would output a by (row#, column#, value) tuple, partitioned by row# but sorted by column# then row#. The reducer could directly write the column file. But the column file format would need to be changed to write counts, descriptors, etc. at the end of files rather than at the front.
(1) is the simplest to implement and most implementations should start with it.
References
CIF Column-Oriented Storage Techniques for MapReduce, Floratou, Patel, Shekita, & Tata, VLDB 2011.
DREMEL Dremel: Interactive Analysis of Web-Scale Datasets, Melnik, Gubarev, Long, Romer, Shivakumar, & Tolton, VLDB 2010.
4.5 - MapReduce guide
Avro provides a convenient way to represent complex data structures within a Hadoop MapReduce job. Avro data can be used as both input to and output from a MapReduce job, as well as the intermediate format. The example in this guide uses Avro data for all three, but it’s possible to mix and match; for instance, MapReduce can be used to aggregate a particular field in an Avro record.
This guide assumes basic familiarity with both Hadoop MapReduce and Avro. See the Hadoop documentation and the Avro getting started guide for introductions to these projects. This guide uses the old MapReduce API (org.apache.hadoop.mapred) and the new MapReduce API (org.apache.hadoop.mapreduce).
Setup
The code from this guide is included in the Avro docs under examples/mr-example. The example is set up as a Maven project that includes the necessary Avro and MapReduce dependencies and the Avro Maven plugin for code generation, so no external jars are needed to run the example. In particular, the POM includes the following dependencies:
<dependency>
<groupId>org.apache.avro</groupId>
<artifactId>avro</artifactId>
<version>1.10.1</version>
</dependency>
<dependency>
<groupId>org.apache.avro</groupId>
<artifactId>avro-mapred</artifactId>
<version>1.10.1</version>
</dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-client</artifactId>
<version>3.1.2</version>
</dependency>
And the following plugin:
<plugin>
<groupId>org.apache.avro</groupId>
<artifactId>avro-maven-plugin</artifactId>
<version>1.10.1</version>
<executions>
<execution>
<phase>generate-sources</phase>
<goals>
<goal>schema</goal>
</goals>
<configuration>
<sourceDirectory>${project.basedir}/../</sourceDirectory>
<outputDirectory>${project.basedir}/target/generated-sources/</outputDirectory>
</configuration>
</execution>
</executions>
</plugin>
If you do not configure the sourceDirectory and outputDirectory properties, the defaults will be used. The sourceDirectory property defaults to src/main/avro. The outputDirectory property defaults to target/generated-sources. You can change the paths to match your project layout.
Alternatively, Avro jars can be downloaded directly from the Apache Avro™ Releases page. The relevant Avro jars for this guide are avro-1.10.1.jar and avro-mapred-1.10.1.jar, as well as avro-tools-1.10.1.jar for code generation and viewing Avro data files as JSON. In addition, you will need to install Hadoop in order to use MapReduce.
Example: ColorCount
Below is a simple example of a MapReduce that uses Avro. There is an example for both the old (org.apache.hadoop.mapred) and new (org.apache.hadoop.mapreduce) APIs under examples/mr-example/src/main/java/example/. MapredColorCount is the example for the older mapred API while MapReduceColorCount is the example for the newer mapreduce API. Both examples are below, but we will detail the mapred API in our subsequent examples.
MapredColorCount:
package example;
import java.io.IOException;
import org.apache.avro.*;
import org.apache.avro.Schema.Type;
import org.apache.avro.mapred.*;
import org.apache.hadoop.conf.*;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.mapred.*;
import org.apache.hadoop.util.*;
import example.avro.User;
public class MapredColorCount extends Configured implements Tool {
public static class ColorCountMapper extends AvroMapper<User, Pair<CharSequence, Integer>> {
@Override
public void map(User user, AvroCollector<Pair<CharSequence, Integer>> collector, Reporter reporter)
throws IOException {
CharSequence color = user.getFavoriteColor();
// We need this check because the User.favorite_color field has type ["string", "null"]
if (color == null) {
color = "none";
}
collector.collect(new Pair<CharSequence, Integer>(color, 1));
}
}
public static class ColorCountReducer extends AvroReducer<CharSequence, Integer,
Pair<CharSequence, Integer>> {
@Override
public void reduce(CharSequence key, Iterable<Integer> values,
AvroCollector<Pair<CharSequence, Integer>> collector,
Reporter reporter)
throws IOException {
int sum = 0;
for (Integer value : values) {
sum += value;
}
collector.collect(new Pair<CharSequence, Integer>(key, sum));
}
}
public int run(String[] args) throws Exception {
if (args.length != 2) {
System.err.println("Usage: MapredColorCount <input path> <output path>");
return -1;
}
JobConf conf = new JobConf(getConf(), MapredColorCount.class);
conf.setJobName("colorcount");
FileInputFormat.setInputPaths(conf, new Path(args[0]));
FileOutputFormat.setOutputPath(conf, new Path(args[1]));
AvroJob.setMapperClass(conf, ColorCountMapper.class);
AvroJob.setReducerClass(conf, ColorCountReducer.class);
// Note that AvroJob.setInputSchema and AvroJob.setOutputSchema set
// relevant config options such as input/output format, map output
// classes, and output key class.
AvroJob.setInputSchema(conf, User.getClassSchema());
AvroJob.setOutputSchema(conf, Pair.getPairSchema(Schema.create(Type.STRING),
Schema.create(Type.INT)));
JobClient.runJob(conf);
return 0;
}
public static void main(String[] args) throws Exception {
int res = ToolRunner.run(new Configuration(), new MapredColorCount(), args);
System.exit(res);
}
}
MapReduceColorCount:
package example;
import java.io.IOException;
import org.apache.avro.Schema;
import org.apache.avro.mapred.AvroKey;
import org.apache.avro.mapred.AvroValue;
import org.apache.avro.mapreduce.AvroJob;
import org.apache.avro.mapreduce.AvroKeyInputFormat;
import org.apache.avro.mapreduce.AvroKeyValueOutputFormat;
import org.apache.hadoop.conf.Configured;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.IntWritable;
import org.apache.hadoop.io.NullWritable;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.mapreduce.Job;
import org.apache.hadoop.mapreduce.Mapper;
import org.apache.hadoop.mapreduce.Reducer;
import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
import org.apache.hadoop.util.Tool;
import org.apache.hadoop.util.ToolRunner;
import example.avro.User;
public class MapReduceColorCount extends Configured implements Tool {
public static class ColorCountMapper extends
Mapper<AvroKey<User>, NullWritable, Text, IntWritable> {
@Override
public void map(AvroKey<User> key, NullWritable value, Context context)
throws IOException, InterruptedException {
CharSequence color = key.datum().getFavoriteColor();
if (color == null) {
color = "none";
}
context.write(new Text(color.toString()), new IntWritable(1));
}
}
public static class ColorCountReducer extends
Reducer<Text, IntWritable, AvroKey<CharSequence>, AvroValue<Integer>> {
@Override
public void reduce(Text key, Iterable<IntWritable> values,
Context context) throws IOException, InterruptedException {
int sum = 0;
for (IntWritable value : values) {
sum += value.get();
}
context.write(new AvroKey<CharSequence>(key.toString()), new AvroValue<Integer>(sum));
}
}
public int run(String[] args) throws Exception {
if (args.length != 2) {
System.err.println("Usage: MapReduceColorCount <input path> <output path>");
return -1;
}
Job job = new Job(getConf());
job.setJarByClass(MapReduceColorCount.class);
job.setJobName("Color Count");
FileInputFormat.setInputPaths(job, new Path(args[0]));
FileOutputFormat.setOutputPath(job, new Path(args[1]));
job.setInputFormatClass(AvroKeyInputFormat.class);
job.setMapperClass(ColorCountMapper.class);
AvroJob.setInputKeySchema(job, User.getClassSchema());
job.setMapOutputKeyClass(Text.class);
job.setMapOutputValueClass(IntWritable.class);
job.setOutputFormatClass(AvroKeyValueOutputFormat.class);
job.setReducerClass(ColorCountReducer.class);
AvroJob.setOutputKeySchema(job, Schema.create(Schema.Type.STRING));
AvroJob.setOutputValueSchema(job, Schema.create(Schema.Type.INT));
return (job.waitForCompletion(true) ? 0 : 1);
}
public static void main(String[] args) throws Exception {
int res = ToolRunner.run(new MapReduceColorCount(), args);
System.exit(res);
}
}
ColorCount reads in data files containing User records, defined in examples/user.avsc, and counts the number of instances of each favorite color. (This example draws inspiration from the canonical WordCount MapReduce application.) This example uses the old MapReduce API. See MapReduceAvroWordCount, found under doc/examples/mr-example/src/main/java/example/ to see the new MapReduce API example. The User schema is defined as follows:
{"namespace": "example.avro",
"type": "record",
"name": "User",
"fields": [
{"name": "name", "type": "string"},
{"name": "favorite_number", "type": ["int", "null"]},
{"name": "favorite_color", "type": ["string", "null"]}
]
}
This schema is compiled into the User class used by ColorCount via the Avro Maven plugin (see examples/mr-example/pom.xml for how this is set up).
ColorCountMapper essentially takes a User as input and extracts the User’s favorite color, emitting the key-value pair <favoriteColor, 1>. ColorCountReducer then adds up how many occurrences of a particular favorite color were emitted, and outputs the result as a Pair record. These Pairs are serialized to an Avro data file.
Running ColorCount
The ColorCount application is provided as a Maven project in the Avro docs under examples/mr-example. To build the project, including the code generation of the User schema, run:
Next, run GenerateData from examples/mr-examples to create an Avro data file, input/users.avro, containing 20 Users with favorite colors chosen randomly from a list:
mvn exec:java -q -Dexec.mainClass=example.GenerateData
Besides creating the data file, GenerateData prints the JSON representations of the Users generated to stdout, for example:
{"name": "user", "favorite_number": null, "favorite_color": "red"}
{"name": "user", "favorite_number": null, "favorite_color": "green"}
{"name": "user", "favorite_number": null, "favorite_color": "purple"}
{"name": "user", "favorite_number": null, "favorite_color": null}
...
Now we’re ready to run ColorCount. We specify our freshly-generated input folder as the input path and output as our output folder (note that MapReduce will not start a job if the output folder already exists):
mvn exec:java -q -Dexec.mainClass=example.MapredColorCount -Dexec.args="input output"
Once ColorCount completes, checking the contents of the new output directory should yield the following:
$ ls output/
part-00000.avro _SUCCESS
You can check the contents of the generated Avro file using the avro-tools jar:
$ java -jar /path/to/avro-tools-1.10.1.jar tojson output/part-00000.avro
{"value": 3, "key": "blue"}
{"value": 7, "key": "green"}
{"value": 1, "key": "none"}
{"value": 2, "key": "orange"}
{"value": 3, "key": "purple"}
{"value": 2, "key": "red"}
{"value": 2, "key": "yellow"}
Now let’s go over the ColorCount example in detail.
Mapper - org.apache.hadoop.mapred API
The easiest way to use Avro data files as input to a MapReduce job is to subclass AvroMapper. An AvroMapper defines a map function that takes an Avro datum as input and outputs a key/value pair represented as a Pair record. In the ColorCount example, ColorCountMapper is an AvroMapper that takes a User as input and outputs a Pair<CharSequence, Integer>>, where the CharSequence key is the user’s favorite color and the Integer value is 1.
public static class ColorCountMapper extends AvroMapper<User, Pair<CharSequence, Integer>> {
@Override
public void map(User user, AvroCollector<Pair<CharSequence, Integer>> collector, Reporter reporter)
throws IOException {
CharSequence color = user.getFavoriteColor();
// We need this check because the User.favorite_color field has type ["string", "null"]
if (color == null) {
color = "none";
}
collector.collect(new Pair<CharSequence, Integer>(color, 1));
}
}
In order to use our AvroMapper, we must call AvroJob.setMapperClass and AvroJob.setInputSchema.
AvroJob.setMapperClass(conf, ColorCountMapper.class);
AvroJob.setInputSchema(conf, User.getClassSchema());
Note that AvroMapper does not implement the Mapper interface. Under the hood, the specified Avro data files are deserialized into AvroWrappers containing the actual data, which are processed by a Mapper that calls the configured AvroMapper’s map function. AvroJob.setInputSchema sets up the relevant configuration parameters needed to make this happen, thus you should not need to call JobConf.setMapperClass, JobConf.setInputFormat, JobConf.setMapOutputKeyClass, JobConf.setMapOutputValueClass, or JobConf.setOutputKeyComparatorClass.
Mapper - org.apache.hadoop.mapreduce API
This document will not go into all the differences between the mapred and mapreduce APIs, however will describe the main differences. As you can see, ColorCountMapper is now a subclass of the Hadoop Mapper class and is passed an AvroKey as it’s key. Additionally, the AvroJob method calls were slightly changed.
public static class ColorCountMapper extends
Mapper<AvroKey<User>, NullWritable, Text, IntWritable> {
@Override
public void map(AvroKey<User> key, NullWritable value, Context context)
throws IOException, InterruptedException {
CharSequence color = key.datum().getFavoriteColor();
if (color == null) {
color = "none";
}
context.write(new Text(color.toString()), new IntWritable(1));
}
}
Reducer - org.apache.hadoop.mapred API
Analogously to AvroMapper, an AvroReducer defines a reducer function that takes the key/value types output by an AvroMapper (or any mapper that outputs Pairs) and outputs a key/value pair represented a Pair record. In the ColorCount example, ColorCountReducer is an AvroReducer that takes the CharSequence key representing a favorite color and the Iterable<Integer> representing the counts for that color (they should all be 1 in this example) and adds up the counts.
public static class ColorCountReducer extends AvroReducer<CharSequence, Integer,
Pair<CharSequence, Integer>> {
@Override
public void reduce(CharSequence key, Iterable<Integer> values,
AvroCollector<Pair<CharSequence, Integer>> collector,
Reporter reporter)
throws IOException {
int sum = 0;
for (Integer value : values) {
sum += value;
}
collector.collect(new Pair<CharSequence, Integer>(key, sum));
}
}
In order to use our AvroReducer, we must call AvroJob.setReducerClass and AvroJob.setOutputSchema.
AvroJob.setReducerClass(conf, ColorCountReducer.class);
AvroJob.setOutputSchema(conf, Pair.getPairSchema(Schema.create(Type.STRING),
Schema.create(Type.INT)));
Note that AvroReducer does not implement the Reducer interface. The intermediate Pairs output by the mapper are split into AvroKeys and AvroValues, which are processed by a Reducer that calls the configured AvroReducer’s reduce function. AvroJob.setOutputSchema sets up the relevant configuration parameters needed to make this happen, thus you should not need to call JobConf.setReducerClass, JobConf.setOutputFormat, JobConf.setOutputKeyClass, JobConf.setMapOutputKeyClass, JobConf.setMapOutputValueClass, or JobConf.setOutputKeyComparatorClass.
Reduce - org.apache.hadoop.mapreduce API
As before we not detail every difference between the APIs. As with the Mapper change ColorCountReducer is now a subclass of Reducer and AvroKey and AvroValue are emitted. Additionally, the AvroJob method calls were slightly changed.
public static class ColorCountReducer extends
Reducer<Text, IntWritable, AvroKey<CharSequence>, AvroValue<Integer>> {
@Override
public void reduce(Text key, Iterable<IntWritable> values,
Context context) throws IOException, InterruptedException {
int sum = 0;
for (IntWritable value : values) {
sum += value.get();
}
context.write(new AvroKey<CharSequence>(key.toString()), new AvroValue<Integer>(sum));
}
}
Learning more
The mapred API allows users to mix Avro AvroMappers and AvroReducers with non-Avro Mappers and Reducers and the mapreduce API allows users input Avro and output non-Avro or vice versa.
The mapred package has API org.apache.avro.mapred documentation as does the org.apache.avro.mapreduce package. MapReduce API (org.apache.hadoop.mapreduce). Similarily to the mapreduce package, it’s possible with the mapred API to implement your own Mappers and Reducers directly using the public classes provided in these libraries. See the AvroWordCount application, found under examples/mr-example/src/main/java/example/AvroWordCount.java in the Avro documentation, for an example of implementing a Reducer that outputs Avro data using the old MapReduce API. See the MapReduceAvroWordCount application, found under examples/mr-example/src/main/java/example/MapReduceAvroWordCount.java in the Avro documentation, for an example of implementing a Reducer that outputs Avro data using the new MapReduce API.
4.6 - IDL Language
Introduction
This document defines Avro IDL, a higher-level language for authoring Avro schemata. Before reading this document, you should have familiarity with the concepts of schemata and protocols, as well as the various primitive and complex types available in Avro.
Overview
Purpose
The aim of the Avro IDL language is to enable developers to author schemata in a way that feels more similar to common programming languages like Java, C++, or Python. Additionally, the Avro IDL language may feel more familiar for those users who have previously used the interface description languages (IDLs) in other frameworks like Thrift, Protocol Buffers, or CORBA.
Usage
Each Avro IDL file defines a single Avro Protocol, and thus generates as its output a JSON-format Avro Protocol file with extension .avpr.
To convert a .avdl file into a .avpr file, it may be processed by the idl tool. For example:
$ java -jar avro-tools.jar idl src/test/idl/input/namespaces.avdl /tmp/namespaces.avpr
$ head /tmp/namespaces.avpr
{
"protocol" : "TestNamespace",
"namespace" : "avro.test.protocol",
The idl tool can also process input to and from stdin and stdout. See idl --help for full usage information.
A Maven plugin is also provided to compile .avdl files. To use it, add something like the following to your pom.xml:
<build>
<plugins>
<plugin>
<groupId>org.apache.avro</groupId>
<artifactId>avro-maven-plugin</artifactId>
<executions>
<execution>
<goals>
<goal>idl-protocol</goal>
</goals>
</execution>
</executions>
</plugin>
</plugins>
</build>
Defining a Protocol in Avro IDL
An Avro IDL file consists of exactly one protocol definition. The minimal protocol is defined by the following code:
This is equivalent to (and generates) the following JSON protocol definition:
{
"protocol" : "MyProtocol",
"types" : [ ],
"messages" : {
}
}
The namespace of the protocol may be changed using the @namespace annotation:
@namespace("mynamespace")
protocol MyProtocol {
}
This notation is used throughout Avro IDL as a way of specifying properties for the annotated element, as will be described later in this document.
Protocols in Avro IDL can contain the following items:
- Imports of external protocol and schema files.
- Definitions of named schemata, including records, errors, enums, and fixeds.
- Definitions of RPC messages
Imports
Files may be imported in one of three formats:
-
An IDL file may be imported with a statement like:
import idl "foo.avdl";
-
A JSON protocol file may be imported with a statement like:
import protocol "foo.avpr";
-
A JSON schema file may be imported with a statement like:
import schema "foo.avsc";
Messages and types in the imported file are added to this file’s protocol.
Imported file names are resolved relative to the current IDL file.
Defining an Enumeration
Enums are defined in Avro IDL using a syntax similar to C or Java. An Avro Enum supports optional default values. In the case that a reader schema is unable to recognize a symbol written by the writer, the reader will fall back to using the defined default value. This default is only used when an incompatible symbol is read. It is not used if the enum field is missing.
Example Writer Enum Definition
enum Shapes {
SQUARE, TRIANGLE, CIRCLE, OVAL
}
Example Reader Enum Definition
enum Shapes {
SQUARE, TRIANGLE, CIRCLE
} = CIRCLE;
In the above example, the reader will use the default value of CIRCLE whenever reading data written with the OVAL symbol of the writer. Also note that, unlike the JSON format, anonymous enums cannot be defined.
Defining a Fixed Length Field
Fixed fields are defined using the following syntax:
fixed MD5(16);
This example defines a fixed-length type called MD5 which contains 16 bytes.
Defining Records and Errors
Records are defined in Avro IDL using a syntax similar to a struct definition in C:
record Employee {
string name;
boolean active = true;
long salary;
}
The above example defines a record with the name “Employee” with three fields.
To define an error, simply use the keyword error instead of record. For example:
error Kaboom {
string explanation;
int result_code = -1;
}
Each field in a record or error consists of a type and a name, optional property annotations and an optional default value.
A type reference in Avro IDL must be one of:
- A primitive type
- A logical type
- A named schema defined prior to this usage in the same Protocol
- A complex type (array, map, or union)
Primitive Types
The primitive types supported by Avro IDL are the same as those supported by Avro’s JSON format. This list includes int, long, string, boolean, float, double, null, and bytes.
Logical Types
Some of the logical types supported by Avro’s JSON format are also supported by Avro IDL. The currently supported types are:
For example:
record Job {
string jobid;
date submitDate;
time_ms submitTime;
timestamp_ms finishTime;
decimal(9,2) finishRatio;
}
References to Named Schemata
If a named schema has already been defined in the same Avro IDL file, it may be referenced by name as if it were a primitive type:
record Card {
Suit suit; // refers to the enum Card defined above
int number;
}
Default Values
Default values for fields may be optionally specified by using an equals sign after the field name followed by a JSON expression indicating the default value. This JSON is interpreted as described in the spec.
Complex Types
Arrays
Array types are written in a manner that will seem familiar to C++ or Java programmers. An array of any type t is denoted array<t>. For example, an array of strings is denoted array<string>, and a multidimensional array of Foo records would be array<array<Foo>>.
Maps
Map types are written similarly to array types. An array that contains values of type t is written map<t>. As in the JSON schema format, all maps contain string-type keys.
Unions
Union types are denoted as union { typeA, typeB, typeC, … }. For example, this record contains a string field that is optional (unioned with null):
record RecordWithUnion {
union { null, string } optionalString;
}
Note that the same restrictions apply to Avro IDL unions as apply to unions defined in the JSON format; namely, a record may not contain multiple elements of the same type.
Defining RPC Messages
The syntax to define an RPC message within a Avro IDL protocol is similar to the syntax for a method declaration within a C header file or a Java interface. To define an RPC message add which takes two arguments named foo and bar, returning an int, simply include the following definition within the protocol:
int add(int foo, int bar = 0);
Message arguments, like record fields, may specify default values.
To define a message with no response, you may use the alias void, equivalent to the Avro null type:
void logMessage(string message);
If you have previously defined an error type within the same protocol, you may declare that a message can throw this error using the syntax:
void goKaboom() throws Kaboom;
To define a one-way message, use the keyword oneway after the parameter list, for example:
void fireAndForget(string message) oneway;
Other Language Features
All Java-style comments are supported within a Avro IDL file. Any text following // on a line is ignored, as is any text between /* and */, possibly spanning multiple lines.
Comments that begin with /** are used as the documentation string for the type or field definition that follows the comment.
Escaping Identifiers
Occasionally, one will need to use a reserved language keyword as an identifier. In order to do so, backticks (`) may be used to escape the identifier. For example, to define a message with the literal name error, you may write:
This syntax is allowed anywhere an identifier is expected.
Annotations for Ordering and Namespaces
Java-style annotations may be used to add additional properties to types and fields throughout Avro IDL.
For example, to specify the sort order of a field within a record, one may use the @order annotation before the field name as follows:
record MyRecord {
string @order("ascending") myAscendingSortField;
string @order("descending") myDescendingField;
string @order("ignore") myIgnoredField;
}
A field’s type may also be preceded by annotations, e.g.:
record MyRecord {
@java-class("java.util.ArrayList") array<string> myStrings;
}
This can be used to support java classes that can be serialized/deserialized via their toString/String constructor, e.g.:
record MyRecord {
@java-class("java.math.BigDecimal") string value;
@java-key-class("java.io.File") map<string> fileStates;
array<@java-class("java.math.BigDecimal") string> weights;
}
Similarly, a @namespace annotation may be used to modify the namespace when defining a named schema. For example:
@namespace("org.apache.avro.firstNamespace")
protocol MyProto {
@namespace("org.apache.avro.someOtherNamespace")
record Foo {}
record Bar {}
}
will define a protocol in the firstNamespace namespace. The record Foo will be defined in someOtherNamespace and Bar will be defined in firstNamespace as it inherits its default from its container.
Type and field aliases are specified with the @aliases annotation as follows:
@aliases(["org.old.OldRecord", "org.ancient.AncientRecord"])
record MyRecord {
string @aliases(["oldField", "ancientField"]) myNewField;
}
Some annotations like those listed above are handled specially. All other annotations are added as properties to the protocol, message, schema or field.
Complete Example
The following is a complete example of a Avro IDL file that shows most of the above features:
/**
* An example protocol in Avro IDL
*/
@namespace("org.apache.avro.test")
protocol Simple {
@aliases(["org.foo.KindOf"])
enum Kind {
FOO,
BAR, // the bar enum value
BAZ
}
record TestRecord {
@order("ignore")
string name;
@order("descending")
Kind kind;
MD5 hash;
union { MD5, null} @aliases(["hash"]) nullableHash;
array<long> arrayOfLongs;
}
error TestError {
string message;
}
string hello(string greeting);
TestRecord echo(TestRecord `record`);
int add(int arg1, int arg2);
bytes echoBytes(bytes data);
void `error`() throws TestError;
void ping() oneway;
}
Additional examples may be found in the Avro source tree under the src/test/idl/input directory.
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
4.7 - SASL profile
Introduction
SASL (RFC 2222) provides a framework for authentication and security of network protocols. Each protocol that uses SASL is meant to define a SASL profile. This document provides a SASL profile for connection-based Avro RPC.
Overview
SASL negotiation proceeds as a series of message interactions over a connection between a client and server using a selected SASL mechanism. The client starts this negotiation by sending its chosen mechanism name with an initial (possibly empty) message. Negotiation proceeds with the exchange of messages until either side indicates success or failure. The content of the messages is mechanism-specific. If the negotiation succeeds, then the session can proceed over the connection, otherwise it must be abandoned.
Some mechanisms continue to process session data after negotiation (e.g., encrypting it), while some specify that further session data is transmitted unmodified.
Negotiation
Commands
Avro SASL negotiation uses four one-byte commands.
- 0: START Used in a client’s initial message.
- 1: CONTINUE Used while negotiation is ongoing.
- 2: FAIL Terminates negotiation unsuccessfully.
- 3: COMPLETE Terminates negotiation successfully.
The format of a START message is:
| 0 | 4-byte mechanism name length | mechanism name | 4-byte payload length | payload data |
The format of a CONTINUE message is:
| 1 | 4-byte payload length | payload data |
The format of a FAIL message is:
| 2 | 4-byte message length | UTF-8 message |
The format of a COMPLETE message is:
| 3 | 4-byte payload length | payload data |
Process
Negotiation is initiated by a client sending a START command containing the client’s chosen mechanism name and any mechanism-specific payload data.
The server and client then interchange some number (possibly zero) of CONTINUE messages. Each message contains payload data that is processed by the security mechanism to generate the next message.
Once either the client or server send a FAIL message then negotiation has failed. UTF-8-encoded text is included in the failure message. Once either a FAIL message has been sent or received, or any other error occurs in the negotiation, further communication on this connection must cease.
Once either the client or server send a COMPLETE message then negotiation has completed successfully. Session data may now be transmitted over the connection until it is closed by either side.
Session Data
If no SASL QOP (quality of protection) is negotiated, then all subsequent writes to/reads over this connection are written/read unmodified. In particular, messages use Avro framing, and are of the form:
| 4-byte frame length | frame data | ... | 4 zero bytes |
If a SASL QOP is negotiated, then it must be used by the connection for all subsequent messages. This is done by wrapping each non-empty frame written using the security mechanism and unwrapping each non-empty frame read. The length written in each non-empty frame is the length of the wrapped data. Complete frames must be passed to the security mechanism for unwrapping. Unwrapped data is then passed to the application as the content of the frame.
If at any point processing fails due to wrapping, unwrapping or framing errors, then all further communication on this connection must cease.
Anonymous Mechanism
The SASL anonymous mechanism (RFC 2245) is quite simple to implement. In particular, an initial anonymous request may be prefixed by the following static sequence:
| 0 | 0009 | ANONYMOUS | 0000 |
If a server uses the anonymous mechanism, it should check that the mechanism name in the start message prefixing the first request received is ‘ANONYMOUS’, then simply prefix its initial response with a COMPLETE message of:
| 3 | 0000 |
If an anonymous server recieves some other mechanism name, then it may respond with a FAIL message as simple as:
| 2 | 0000 |
Note that the anonymous mechanism need add no additional round-trip messages between client and server. The START message can be piggybacked on the initial request and the COMPLETE or FAIL message can be piggybacked on the initial response.
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
4.8 - Wiki
The Wiki page can be found here.
4.9 - FAQ
What is Avro?
Avro is a data serialization system.
Avro provides:
- Rich data structures.
- A compact, fast, binary data format.
- A container file, to store persistent data.
- Remote procedure call (RPC).
- Simple integration with dynamic languages. Code generation is not required to read or write data files nor to use or implement RPC protocols. Code generation as an optional optimization, only worth implementing for statically typed languages.
How can I get started quickly with Avro?
Check out the Quick Start Guide.
How do I statically compile a schema or protocol into generated code?
In Java
- Add the avro jar, the jackson-mapper-asl.jar and jackson-core-asl.jar to your CLASSPATH.
- Run java org.apache.avro.specific.SpecificCompiler .
This appears to be out of date, the SpecificCompiler requires two arguments, presumably an input and and output file, but it isn’t clear that this does.
Or use the Schema or Protocol Ant tasks. Avro’s build.xml provides examples of how these are used.
Lastly, you can also use the “avro-tools” jar which ships with an Avro release. Just use the “compile (schema|protocol)” command.
How are Strings represented in Java?
They use org.apache.avro.util.Utf8, not java.lang.String.
More generally, how do Avro types map to Java types?
The mappings are documented in the package javadoc for generic, specific and reflect API.
From Doug Cutting:
HDFS splits files into blocks, and mapreduce runs a map task for each block. When the task starts, it needs to be able to seek into the file to the start of the block process through the block’s end. If the file were, e.g., a gzip file, this would not be possible, since gzip files must be decompressed from the start. One cannot seek into the middle of a gzip file and start decompressing. So Hadoop’s SequenceFile places a marker periodically (~64k) in the file at record and compression boundaries, where processing can be sensibly started. Then, when a map task starts processing an HDFS block, it finds the first marker after the block’s start and continues through the first marker in the next block of the file. This requires a bit of non-local access (~0.1%). Avro’s data file uses the same method as SequenceFile.
Why isn’t every value in Avro nullable?
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 of every value stored and
corresponding computational costs to create this bit on write and
interpret it on read. These costs are wasted when values may not in
fact be null, as is the case in many datasets. In Avro such costs are
only paid when values may actually be null.
Also, allowing values to be null is a well-known source of errors. In
Avro, a value declared as non-null will always be non-null and programs
need not test for null values when processing it nor will they ever fail
for lack of such tests.
Tony Hoare calls his invention of null references his “Billion Dollar
Mistake”.
http://qconlondon.com/london-2009/presentation/Null+References:+The+Billion+Dollar+Mistake
Also note that in some programming languages not all values are permitted to be null. For example, in Java, values of type boolean, byte, short, char, int, float, long, and double may not be null.
How can I serialize directly to/from a byte array?
As pointed out in the specification, Avro data should always be stored with its schema. The Avro provided classes DataFileWriter, DataFileReader, and DataFileStream all ensure this by serializing the Schema in a container header. In some special cases, such as when implementing a new storage system or writing unit tests, you may need to write and read directly with the bare Avro serialized values. The following examples use code generated for Java from an Avro Schema, specifically the User example from the Quickstart guide.
Serializing to a byte array
This example takes a User object and returns a newly allocated byte array with the Avro serialization of that user.
ByteArrayOutputStream out = new ByteArrayOutputStream();
BinaryEncoder encoder = EncoderFactory.get().binaryEncoder(out, null);
DatumWriter<User> writer = new SpecificDatumWriter<User>(User.getClassSchema());
writer.write(user, encoder);
encoder.flush();
out.close();
byte[] serializedBytes = out.toByteArray();
Deserializing from a byte array
This example takes a byte array containing the Avro serialization of a user and returns a User object.SpecificDatumReader reader = new SpecificDatumReader(User.getClassSchema());
Decoder decoder = DecoderFactory.get().binaryDecoder(bytes, null);
User user = reader.read(null, decoder);
Note: If you are serializing or deserializing in a loop or as a method, you should be reusing objects, readers and/or writers. Check the JavaDoc to see how to reuse the objects.
5 - Apache Avro™ 1.10.0 Documentation
Introduction
Apache Avro™ is a data serialization system.
Avro provides:
- Rich data structures.
- A compact, fast, binary data format.
- A container file, to store persistent data.
- Remote procedure call (RPC).
- Simple integration with dynamic languages. Code generation is not required to read or write data files nor to use or implement RPC protocols. Code generation as an optional optimization, only worth implementing for statically typed languages.
Schemas
Avro relies on schemas. When Avro data is read, the schema used when writing it is always present. This permits each datum to be written with no per-value overheads, making serialization both fast and small. This also facilitates use with dynamic, scripting languages, since data, together with its schema, is fully self-describing.
When Avro data is stored in a file, its schema is stored with it, so that files may be processed later by any program. If the program reading the data expects a different schema this can be easily resolved, since both schemas are present.
When Avro is used in RPC, the client and server exchange schemas in the connection handshake. (This can be optimized so that, for most calls, no schemas are actually transmitted.) Since both client and server both have the other’s full schema, correspondence between same named fields, missing fields, extra fields, etc. can all be easily resolved.
Avro schemas are defined with JSON . This facilitates implementation in languages that already have JSON libraries.
Comparison with other systems
Avro provides functionality similar to systems such as Thrift, Protocol Buffers, etc. Avro differs from these systems in the following fundamental aspects.
- Dynamic typing: Avro does not require that code be generated. Data is always accompanied by a schema that permits full processing of that data without code generation, static datatypes, etc. This facilitates construction of generic data-processing systems and languages.
- Untagged data: Since the schema is present when data is read, considerably less type information need be encoded with data, resulting in smaller serialization size.
- No manually-assigned field IDs: When a schema changes, both the old and new schema are always present when processing data, so differences may be resolved symbolically, using field names.
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
5.1 - Getting Started (Java)
This is a short guide for getting started with Apache Avro™ using Java. This guide only covers using Avro for data serialization; see Patrick Hunt’s Avro RPC Quick Start for a good introduction to using Avro for RPC.
Download
Avro implementations for C, C++, C#, Java, PHP, Python, and Ruby can be downloaded from the Apache Avro™ Releases page. This guide uses Avro 1.10.0, the latest version at the time of writing. For the examples in this guide, download avro-1.10.0.jar and avro-tools-1.10.0.jar.
Alternatively, if you are using Maven, add the following dependency to your POM:
<dependency>
<groupId>org.apache.avro</groupId>
<artifactId>avro</artifactId>
<version>1.10.0</version>
</dependency>
As well as the Avro Maven plugin (for performing code generation):
<plugin>
<groupId>org.apache.avro</groupId>
<artifactId>avro-maven-plugin</artifactId>
<version>1.10.0</version>
<executions>
<execution>
<phase>generate-sources</phase>
<goals>
<goal>schema</goal>
</goals>
<configuration>
<sourceDirectory>${project.basedir}/src/main/avro/</sourceDirectory>
<outputDirectory>${project.basedir}/src/main/java/</outputDirectory>
</configuration>
</execution>
</executions>
</plugin>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-compiler-plugin</artifactId>
<configuration>
<source>1.8</source>
<target>1.8</target>
</configuration>
</plugin>
You may also build the required Avro jars from source. Building Avro is beyond the scope of this guide; see the Build Documentation page in the wiki for more information.
Defining a schema
Avro schemas are defined using JSON. Schemas are composed of primitive types (null, boolean, int, long, float, double, bytes, and string) and complex types (record, enum, array, map, union, and fixed). You can learn more about Avro schemas and types from the specification, but for now let’s start with a simple schema example, user.avsc:
{"namespace": "example.avro",
"type": "record",
"name": "User",
"fields": [
{"name": "name", "type": "string"},
{"name": "favorite_number", "type": ["int", "null"]},
{"name": "favorite_color", "type": ["string", "null"]}
]
}
This schema defines a record representing a hypothetical user. (Note that a schema file can only contain a single schema definition.) At minimum, a record definition must include its type (“type”: “record”), a name (“name”: “User”), and fields, in this case name, favorite_number, and favorite_color. We also define a namespace (“namespace”: “example.avro”), which together with the name attribute defines the “full name” of the schema (example.avro.User in this case).
Fields are defined via an array of objects, each of which defines a name and type (other attributes are optional, see the record specification for more details). The type attribute of a field is another schema object, which can be either a primitive or complex type. For example, the name field of our User schema is the primitive type string, whereas the favorite_number and favorite_color fields are both unions, represented by JSON arrays. unions are a complex type that can be any of the types listed in the array; e.g., favorite_number can either be an int or null, essentially making it an optional field.
Serializing and deserializing with code generation
Compiling the schema
Code generation allows us to automatically create classes based on our previously-defined schema. Once we have defined the relevant classes, there is no need to use the schema directly in our programs. We use the avro-tools jar to generate code as follows:
java -jar /path/to/avro-tools-1.10.0.jar compile schema <schema file> <destination>
This will generate the appropriate source files in a package based on the schema’s namespace in the provided destination folder. For instance, to generate a User class in package example.avro from the schema defined above, run
java -jar /path/to/avro-tools-1.10.0.jar compile schema user.avsc .
Note that if you using the Avro Maven plugin, there is no need to manually invoke the schema compiler; the plugin automatically performs code generation on any .avsc files present in the configured source directory.
Creating Users
Now that we’ve completed the code generation, let’s create some Users, serialize them to a data file on disk, and then read back the file and deserialize the User objects.
First let’s create some Users and set their fields.
User user1 = new User();
user1.setName("Alyssa");
user1.setFavoriteNumber(256);
// Leave favorite color null
// Alternate constructor
User user2 = new User("Ben", 7, "red");
// Construct via builder
User user3 = User.newBuilder()
.setName("Charlie")
.setFavoriteColor("blue")
.setFavoriteNumber(null)
.build();
As shown in this example, Avro objects can be created either by invoking a constructor directly or by using a builder. Unlike constructors, builders will automatically set any default values specified in the schema. Additionally, builders validate the data as it set, whereas objects constructed directly will not cause an error until the object is serialized. However, using constructors directly generally offers better performance, as builders create a copy of the datastructure before it is written.
Note that we do not set user1’s favorite color. Since that record is of type [“string”, “null”], we can either set it to a string or leave it null; it is essentially optional. Similarly, we set user3’s favorite number to null (using a builder requires setting all fields, even if they are null).
Serializing
Now let’s serialize our Users to disk.
/// Serialize user1, user2 and user3 to disk
DatumWriter<User> userDatumWriter = new SpecificDatumWriter<User>(User.class);
DataFileWriter<User> dataFileWriter = new DataFileWriter<User>(userDatumWriter);
dataFileWriter.create(user1.getSchema(), new File("users.avro"));
dataFileWriter.append(user1);
dataFileWriter.append(user2);
dataFileWriter.append(user3);
dataFileWriter.close();
We create a DatumWriter, which converts Java objects into an in-memory serialized format. The SpecificDatumWriter class is used with generated classes and extracts the schema from the specified generated type.
Next we create a DataFileWriter, which writes the serialized records, as well as the schema, to the file specified in the dataFileWriter.create call. We write our users to the file via calls to the dataFileWriter.append method. When we are done writing, we close the data file.
Deserializing
Finally, let’s deserialize the data file we just created.
// Deserialize Users from disk
DatumReader<User> userDatumReader = new SpecificDatumReader<User>(User.class);
DataFileReader<User> dataFileReader = new DataFileReader<User>(file, userDatumReader);
User user = null;
while (dataFileReader.hasNext()) {
// Reuse user object by passing it to next(). This saves us from
// allocating and garbage collecting many objects for files with
// many items.
user = dataFileReader.next(user);
System.out.println(user);
}
This snippet will output:
{"name": "Alyssa", "favorite_number": 256, "favorite_color": null}
{"name": "Ben", "favorite_number": 7, "favorite_color": "red"}
{"name": "Charlie", "favorite_number": null, "favorite_color": "blue"}
Deserializing is very similar to serializing. We create a SpecificDatumReader, analogous to the SpecificDatumWriter we used in serialization, which converts in-memory serialized items into instances of our generated class, in this case User. We pass the DatumReader and the previously created File to a DataFileReader, analogous to the DataFileWriter, which reads both the schema used by the writer as well as the data from the file on disk. The data will be read using the writer’s schema included in the file and the schema provided by the reader, in this case the User class. The writer’s schema is needed to know the order in which fields were written, while the reader’s schema is needed to know what fields are expected and how to fill in default values for fields added since the file was written. If there are differences between the two schemas, they are resolved according to the Schema Resolution specification.
Next we use the DataFileReader to iterate through the serialized Users and print the deserialized object to stdout. Note how we perform the iteration: we create a single User object which we store the current deserialized user in, and pass this record object to every call of dataFileReader.next. This is a performance optimization that allows the DataFileReader to reuse the same User object rather than allocating a new User for every iteration, which can be very expensive in terms of object allocation and garbage collection if we deserialize a large data file. While this technique is the standard way to iterate through a data file, it’s also possible to use for (User user : dataFileReader) if performance is not a concern.
Compiling and running the example code
This example code is included as a Maven project in the examples/java-example directory in the Avro docs. From this directory, execute the following commands to build and run the example:
$ mvn compile # includes code generation via Avro Maven plugin
$ mvn -q exec:java -Dexec.mainClass=example.SpecificMain
Beta feature: Generating faster code
In this release we have introduced a new approach to generating code that speeds up decoding of objects by more than 10% and encoding by more than 30% (future performance enhancements are underway). To ensure a smooth introduction of this change into production systems, this feature is controlled by a feature flag, the system property org.apache.avro.specific.use_custom_coders. In this first release, this feature is off by default. To turn it on, set the system flag to true at runtime. In the sample above, for example, you could enable the fater coders as follows:
$ mvn -q exec:java -Dexec.mainClass=example.SpecificMain \
-Dorg.apache.avro.specific.use_custom_coders=true
Note that you do not have to recompile your Avro schema to have access to this feature. The feature is compiled and built into your code, and you turn it on and off at runtime using the feature flag. As a result, you can turn it on during testing, for example, and then off in production. Or you can turn it on in production, and quickly turn it off if something breaks.
We encourage the Avro community to exercise this new feature early to help build confidence. (For those paying one-demand for compute resources in the cloud, it can lead to meaningful cost savings.) As confidence builds, we will turn this feature on by default, and eventually eliminate the feature flag (and the old code).
Serializing and deserializing without code generation
Data in Avro is always stored with its corresponding schema, meaning we can always read a serialized item regardless of whether we know the schema ahead of time. This allows us to perform serialization and deserialization without code generation.
Let’s go over the same example as in the previous section, but without using code generation: we’ll create some users, serialize them to a data file on disk, and then read back the file and deserialize the users objects.
Creating users
First, we use a Parser to read our schema definition and create a Schema object.
Schema schema = new Schema.Parser().parse(new File("user.avsc"));
Using this schema, let’s create some users.
GenericRecord user1 = new GenericData.Record(schema);
user1.put("name", "Alyssa");
user1.put("favorite_number", 256);
// Leave favorite color null
GenericRecord user2 = new GenericData.Record(schema);
user2.put("name", "Ben");
user2.put("favorite_number", 7);
user2.put("favorite_color", "red");
Since we’re not using code generation, we use GenericRecords to represent users. GenericRecord uses the schema to verify that we only specify valid fields. If we try to set a non-existent field (e.g., user1.put(“favorite_animal”, “cat”)), we’ll get an AvroRuntimeException when we run the program.
Note that we do not set user1’s favorite color. Since that record is of type [“string”, “null”], we can either set it to a string or leave it null; it is essentially optional.
Serializing
Now that we’ve created our user objects, serializing and deserializing them is almost identical to the example above which uses code generation. The main difference is that we use generic instead of specific readers and writers.
First we’ll serialize our users to a data file on disk.
// Serialize user1 and user2 to disk
File file = new File("users.avro");
DatumWriter<GenericRecord> datumWriter = new GenericDatumWriter<GenericRecord>(schema);
DataFileWriter<GenericRecord> dataFileWriter = new DataFileWriter<GenericRecord>(datumWriter);
dataFileWriter.create(schema, file);
dataFileWriter.append(user1);
dataFileWriter.append(user2);
dataFileWriter.close();
We create a DatumWriter, which converts Java objects into an in-memory serialized format. Since we are not using code generation, we create a GenericDatumWriter. It requires the schema both to determine how to write the GenericRecords and to verify that all non-nullable fields are present.
As in the code generation example, we also create a DataFileWriter, which writes the serialized records, as well as the schema, to the file specified in the dataFileWriter.create call. We write our users to the file via calls to the dataFileWriter.append method. When we are done writing, we close the data file.
Deserializing
Finally, we’ll deserialize the data file we just created.
// Deserialize users from disk
DatumReader<GenericRecord> datumReader = new GenericDatumReader<GenericRecord>(schema);
DataFileReader<GenericRecord> dataFileReader = new DataFileReader<GenericRecord>(file, datumReader);
GenericRecord user = null;
while (dataFileReader.hasNext()) {
// Reuse user object by passing it to next(). This saves us from
// allocating and garbage collecting many objects for files with
// many items.
user = dataFileReader.next(user);
System.out.println(user);
This outputs:
{"name": "Alyssa", "favorite_number": 256, "favorite_color": null}
{"name": "Ben", "favorite_number": 7, "favorite_color": "red"}
Deserializing is very similar to serializing. We create a GenericDatumReader, analogous to the GenericDatumWriter we used in serialization, which converts in-memory serialized items into GenericRecords. We pass the DatumReader and the previously created File to a DataFileReader, analogous to the DataFileWriter, which reads both the schema used by the writer as well as the data from the file on disk. The data will be read using the writer’s schema included in the file, and the reader’s schema provided to the GenericDatumReader. The writer’s schema is needed to know the order in which fields were written, while the reader’s schema is needed to know what fields are expected and how to fill in default values for fields added since the file was written. If there are differences between the two schemas, they are resolved according to the Schema Resolution specification.
Next, we use the DataFileReader to iterate through the serialized users and print the deserialized object to stdout. Note how we perform the iteration: we create a single GenericRecord object which we store the current deserialized user in, and pass this record object to every call of dataFileReader.next. This is a performance optimization that allows the DataFileReader to reuse the same record object rather than allocating a new GenericRecord for every iteration, which can be very expensive in terms of object allocation and garbage collection if we deserialize a large data file. While this technique is the standard way to iterate through a data file, it’s also possible to use for (GenericRecord user : dataFileReader) if performance is not a concern.
Compiling and running the example code
This example code is included as a Maven project in the examples/java-example directory in the Avro docs. From this directory, execute the following commands to build and run the example:
$ mvn compile
$ mvn -q exec:java -Dexec.mainClass=example.GenericMain
5.2 - Getting Started (Python)
This is a short guide for getting started with Apache Avro™ using Python. This guide only covers using Avro for data serialization; see Patrick Hunt’s Avro RPC Quick Start for a good introduction to using Avro for RPC.
Notice for Python 3 users
A package called “avro-python3” had been provided to support Python 3 previously, but the codebase was consolidated into the “avro” package and that supports both Python 2 and 3 now. The avro-python3 package will be removed in the near future, so users should use the “avro” package instead. They are mostly API compatible, but there’s a few minor difference (e.g., function name capitalization, such as avro.schema.Parse vs avro.schema.parse).
Download
For Python, the easiest way to get started is to install it from PyPI. Python’s Avro API is available over PyPi.
$ python3 -m pip install avro
The official releases of the Avro implementations for C, C++, C#, Java, PHP, Python, and Ruby can be downloaded from the Apache Avro™ Releases page. This guide uses Avro 1.10.0, the latest version at the time of writing. Download and unzip avro-1.10.0.tar.gz, and install via python setup.py (this will probably require root privileges). Ensure that you can import avro from a Python prompt.
$ tar xvf avro-1.10.0.tar.gz
$ cd avro-1.10.0
$ python setup.py install
$ python
>>> import avro # should not raise ImportError
Alternatively, you may build the Avro Python library from source. From your the root Avro directory, run the commands
$ cd lang/py/
$ python3 -m pip install -e .
$ python
Defining a schema
Avro schemas are defined using JSON. Schemas are composed of primitive types (null, boolean, int, long, float, double, bytes, and string) and complex types (record, enum, array, map, union, and fixed). You can learn more about Avro schemas and types from the specification, but for now let’s start with a simple schema example, user.avsc:
{"namespace": "example.avro",
"type": "record",
"name": "User",
"fields": [
{"name": "name", "type": "string"},
{"name": "favorite_number", "type": ["int", "null"]},
{"name": "favorite_color", "type": ["string", "null"]}
]
}
This schema defines a record representing a hypothetical user. (Note that a schema file can only contain a single schema definition.) At minimum, a record definition must include its type (“type”: “record”), a name (“name”: “User”), and fields, in this case name, favorite_number, and favorite_color. We also define a namespace (“namespace”: “example.avro”), which together with the name attribute defines the “full name” of the schema (example.avro.User in this case).
Fields are defined via an array of objects, each of which defines a name and type (other attributes are optional, see the record specification for more details). The type attribute of a field is another schema object, which can be either a primitive or complex type. For example, the name field of our User schema is the primitive type string, whereas the favorite_number and favorite_color fields are both unions, represented by JSON arrays. unions are a complex type that can be any of the types listed in the array; e.g., favorite_number can either be an int or null, essentially making it an optional field.
Serializing and deserializing without code generation
Data in Avro is always stored with its corresponding schema, meaning we can always read a serialized item, regardless of whether we know the schema ahead of time. This allows us to perform serialization and deserialization without code generation. Note that the Avro Python library does not support code generation.
Try running the following code snippet, which serializes two users to a data file on disk, and then reads back and deserializes the data file:
import avro.schema
from avro.datafile import DataFileReader, DataFileWriter
from avro.io import DatumReader, DatumWriter
schema = avro.schema.parse(open("user.avsc", "rb").read())
writer = DataFileWriter(open("users.avro", "wb"), DatumWriter(), schema)
writer.append({"name": "Alyssa", "favorite_number": 256})
writer.append({"name": "Ben", "favorite_number": 7, "favorite_color": "red"})
writer.close()
reader = DataFileReader(open("users.avro", "rb"), DatumReader())
for user in reader:
print user
reader.close()
This outputs:
{u'favorite_color': None, u'favorite_number': 256, u'name': u'Alyssa'}
{u'favorite_color': u'red', u'favorite_number': 7, u'name': u'Ben'}
Do make sure that you open your files in binary mode (i.e. using the modes wb or rb respectively). Otherwise you might generate corrupt files due to automatic replacement of newline characters with the platform-specific representations.
Let’s take a closer look at what’s going on here.
schema = avro.schema.parse(open("user.avsc", "rb").read())
avro.schema.parse takes a string containing a JSON schema definition as input and outputs a avro.schema.Schema object (specifically a subclass of Schema, in this case RecordSchema). We’re passing in the contents of our user.avsc schema file here.
writer = DataFileWriter(open("users.avro", "wb"), DatumWriter(), schema)
We create a DataFileWriter, which we’ll use to write serialized items to a data file on disk. The DataFileWriter constructor takes three arguments:
- The file we’ll serialize to
- A DatumWriter, which is responsible for actually serializing the items to Avro’s binary format (DatumWriters can be used separately from DataFileWriters, e.g., to perform IPC with Avro).
- The schema we’re using. The DataFileWriter needs the schema both to write the schema to the data file, and to verify that the items we write are valid items and write the appropriate fields.
writer.append({"name": "Alyssa", "favorite_number": 256})
writer.append({"name": "Ben", "favorite_number": 7, "favorite_color": "red"})
We use DataFileWriter.append to add items to our data file. Avro records are represented as Python dicts. Since the field favorite_color has type [“int”, “null”], we are not required to specify this field, as shown in the first append. Were we to omit the required name field, an exception would be raised. Any extra entries not corresponding to a field are present in the dict are ignored.
reader = DataFileReader(open("users.avro", "rb"), DatumReader())
We open the file again, this time for reading back from disk. We use a DataFileReader and DatumReader analagous to the DataFileWriter and DatumWriter above.
for user in reader:
print user
The DataFileReader is an iterator that returns dicts corresponding to the serialized items.
5.3 - Specification
Introduction
This document defines Apache Avro. It is intended to be the authoritative specification. Implementations of Avro must adhere to this document.
Schema Declaration
A Schema is represented in JSON by one of:
- A JSON string, naming a defined type.
- A JSON object, of the form:
{"type": "typeName" ...attributes...}
where typeName is either a primitive or derived type name, as defined below. Attributes not defined in this document are permitted as metadata, but must not affect the format of serialized data.
- A JSON array, representing a union of embedded types.
Primitive Types
The set of primitive type names is:
- null: no value
- boolean: a binary value
- int: 32-bit signed integer
- long: 64-bit signed integer
- float: single precision (32-bit) IEEE 754 floating-point number
- double: double precision (64-bit) IEEE 754 floating-point number
- bytes: sequence of 8-bit unsigned bytes
- string: unicode character sequence
Primitive types have no specified attributes.
Primitive type names are also defined type names. Thus, for example, the schema “string” is equivalent to:
Complex Types
Avro supports six kinds of complex types: records, enums, arrays, maps, unions and fixed.
Records
Records use the type name “record” and support three attributes:
- name: a JSON string providing the name of the record (required).
- namespace, a JSON string that qualifies the name;
- doc: a JSON string providing documentation to the user of this schema (optional).
- aliases: a JSON array of strings, providing alternate names for this record (optional).
- fields: a JSON array, listing fields (required). Each field is a JSON object with the following attributes:
- name: a JSON string providing the name of the field (required), and
- doc: a JSON string describing this field for users (optional).
- type: a schema, as defined above
- _default: A default value for this field, used when reading instances that lack this field (optional). 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. Default values for bytes and fixed fields are JSON strings, where Unicode code points 0-255 are mapped to unsigned 8-bit byte values 0-255.
field default values
| avro type |
json type |
example |
| null |
null |
null |
| boolean |
boolean |
true |
| int,long |
integer |
1 |
| float,double |
number |
1.1 |
| bytes |
string |
"\u00FF" |
| string |
string |
"foo" |
| record |
object |
{"a": 1} |
| enum |
string |
"FOO" |
| array |
array |
[1] |
| map |
object |
{"a": 1} |
| fixed |
string |
"\u00ff" |
- order: specifies how this field impacts sort ordering of this record (optional). Valid values are “ascending” (the default), “descending”, or “ignore”. For more details on how this is used, see the sort order section below.
- aliases: a JSON array of strings, providing alternate names for this field (optional).
For example, a linked-list of 64-bit values may be defined with:
{
"type": "record",
"name": "LongList",
"aliases": ["LinkedLongs"], // old name for this
"fields" : [
{"name": "value", "type": "long"}, // each element has a long
{"name": "next", "type": ["null", "LongList"]} // optional next element
]
}
Enums
Enums use the type name “enum” and support the following attributes:
- name: a JSON string providing the name of the enum (required).
- namespace, a JSON string that qualifies the name ;
- aliases: a JSON array of strings, providing alternate names for this enum (optional).
- doc: a JSON string providing documentation to the user of this schema (optional).
- symbols: a JSON array, listing symbols, as JSON strings (required). All symbols in an enum must be unique; duplicates are prohibited. Every symbol must match the regular expression [A-Za-z_][A-Za-z0-9_]* (the same requirement as for names).
- default: A default value for this enumeration, used during resolution when the reader encounters a symbol from the writer that isn’t defined in the reader’s schema (optional). The value provided here must be a JSON string that’s a member of the symbols array. See documentation on schema resolution for how this gets used.
For example, playing card suits might be defined with:
{
"type": "enum",
"name": "Suit",
"symbols" : ["SPADES", "HEARTS", "DIAMONDS", "CLUBS"]
}
Arrays
Arrays use the type name “array” and support a single attribute:
- items: the schema of the array’s items.
For example, an array of strings is declared with:
{
"type": "array",
"items" : "string",
"default": []
}
Maps
Maps use the type name “map” and support one attribute:
- values: the schema of the map’s values.
Map keys are assumed to be strings.
For example, a map from string to long is declared with:
{
"type": "map",
"items" : "long",
"default": {}
}
Unions
Unions, as mentioned above, are represented using JSON arrays. For example, ["null", "string"] declares a schema which may be either a null or string.
(Note that when a default value is specified for a record field whose type is a union, the type of the default value must match the first element of the union. Thus, for unions containing “null”, the “null” is usually listed first, since the default value of such unions is typically null.)
Unions may not contain more than one schema with the same type, except for the named types record, fixed and enum. For example, unions containing two array types or two map types are not permitted, but two types with different names are permitted. (Names permit efficient resolution when reading and writing unions.)
Unions may not immediately contain other unions.
Fixed
Fixed uses the type name “fixed” and supports two attributes:
- name: a string naming this fixed (required).
- namespace, a string that qualifies the name;
- aliases: a JSON array of strings, providing alternate names for this enum (optional).
- doc: a JSON string providing documentation to the user of this schema (optional).
- size: an integer, specifying the number of bytes per value (required).
For example, 16-byte quantity may be declared with:
{"type": "fixed", "size": 16, "name": "md5"}
Names
Record, enums and fixed are named types. Each has a fullname that is composed of two parts; a name and a namespace. Equality of names is defined on the fullname.
The name portion of a fullname, record field names, and enum symbols must:
- start with [A-Za-z_]
- subsequently contain only [A-Za-z0-9_]
A namespace is a dot-separated sequence of such names. The empty string may also be used as a namespace to indicate the null namespace. Equality of names (including field names and enum symbols) as well as fullnames is case-sensitive.
The null namespace may not be used in a dot-separated sequence of names. So the grammar for a namespace is:
<empty> | <name>[(<dot><name>)*]
In record, enum and fixed definitions, the fullname is determined in one of the following ways:
A name and namespace are both specified. For example, one might use “name”: “X”, “namespace”: “org.foo” to indicate the fullname org.foo.X.
-
A fullname is specified. If the name specified contains a dot, then it is assumed to be a fullname, and any namespace also specified is ignored. For example, use “name”: “org.foo.X” to indicate the fullname org.foo.X.
-
A name only is specified, i.e., a name that contains no dots. In this case the namespace is taken from the most tightly enclosing schema or protocol. For example, if “name”: “X” is specified, and this occurs within a field of the record definition of org.foo.Y, then the fullname is org.foo.X. If there is no enclosing namespace then the null namespace is used.
References to previously defined names are as in the latter two cases above: if they contain a dot they are a fullname, if they do not contain a dot, the namespace is the namespace of the enclosing definition.
Primitive type names have no namespace and their names may not be defined in any namespace.
A schema or protocol may not contain multiple definitions of a fullname. Further, a name must be defined before it is used (“before” in the depth-first, left-to-right traversal of the JSON parse tree, where the types attribute of a protocol is always deemed to come “before” the messages attribute.)
Aliases
Named types and fields may have aliases. An implementation may optionally use aliases to map a writer’s schema to the reader’s. This facilitates both schema evolution as well as processing disparate datasets.
Aliases function by re-writing the writer’s schema using aliases from the reader’s schema. For example, if the writer’s schema was named “Foo” and the reader’s schema is named “Bar” and has an alias of “Foo”, then the implementation would act as though “Foo” were named “Bar” when reading. Similarly, if data was written as a record with a field named “x” and is read as a record with a field named “y” with alias “x”, then the implementation would act as though “x” were named “y” when reading.
A type alias may be specified either as a fully namespace-qualified, or relative to the namespace of the name it is an alias for. For example, if a type named “a.b” has aliases of “c” and “x.y”, then the fully qualified names of its aliases are “a.c” and “x.y”.
Data Serialization and Deserialization
Binary encoded Avro data does not include type information or field names. The benefit is that the serialized data is small, but as a result a schema must always be used in order to read Avro data correctly. The best way to ensure that the schema is structurally identical to the one used to write the data is to use the exact same schema.
Therefore, files or systems that store Avro data should always include the writer’s schema for that data. Avro-based remote procedure call (RPC) systems must also guarantee that remote recipients of data have a copy of the schema used to write that data. In general, it is advisable that any reader of Avro data should use a schema that is the same (as defined more fully in Parsing Canonical Form for Schemas) as the schema that was used to write the data in order to deserialize it correctly. Deserializing data into a newer schema is accomplished by specifying an additional schema, the results of which are described in Schema Resolution.
In general, both serialization and deserialization proceed as a depth-first, left-to-right traversal of the schema, serializing or deserializing primitive types as they are encountered. Therefore, it is possible, though not advisable, to read Avro data with a schema that does not have the same Parsing Canonical Form as the schema with which the data was written. In order for this to work, the serialized primitive values must be compatible, in order value by value, with the items in the deserialization schema. For example, int and long are always serialized the same way, so an int could be deserialized as a long. Since the compatibility of two schemas depends on both the data and the serialization format (eg. binary is more permissive than JSON because JSON includes field names, eg. a long that is too large will overflow an int), it is simpler and more reliable to use schemas with identical Parsing Canonical Form.
Encodings
Avro specifies two serialization encodings: binary and JSON. Most applications will use the binary encoding, as it is smaller and faster. But, for debugging and web-based applications, the JSON encoding may sometimes be appropriate.
Binary Encoding
Binary encoding does not include field names, self-contained information about the types of individual bytes, nor field or record separators. Therefore readers are wholly reliant on the schema used when the data was encoded.
Primitive Types
Primitive types are encoded in binary as follows:
- null is written as zero bytes.
- a boolean is written as a single byte whose value is either 0 (false) or 1 (true).
- int and long values are written using variable-length zig-zag coding. Some examples:
| value |
hex |
| 0 |
00 |
| -1 |
01 |
| 1 |
02 |
| -2 |
03 |
| 2 |
04 |
| … |
|
| -64 |
7f |
| 64 |
80 01 |
| … |
|
- a float is written as 4 bytes. The float is converted into a 32-bit integer using a method equivalent to Java’s floatToIntBits and then encoded in little-endian format.
- a double is written as 8 bytes. The double is converted into a 64-bit integer using a method equivalent to Java’s doubleToLongBits and then encoded in little-endian format.
- bytes are encoded as a long followed by that many bytes of data.
- a string is encoded as a long followed by that many bytes of UTF-8 encoded character data.
For example, the three-character string “foo” would be encoded as the long value 3 (encoded as hex 06) followed by the UTF-8 encoding of ‘f’, ‘o’, and ‘o’ (the hex bytes 66 6f 6f):
06 66 6f 6f
Complex Types
Complex types are encoded in binary as follows:
Records
A record is encoded by encoding the values of its fields in the order that they are declared. In other words, a record is encoded as just the concatenation of the encodings of its fields. Field values are encoded per their schema.
For example, the record schema
{
"type": "record",
"name": "test",
"fields" : [
{"name": "a", "type": "long"},
{"name": "b", "type": "string"}
]
}
An instance of this record whose a field has value 27 (encoded as hex 36) and whose b field has value “foo” (encoded as hex bytes 06 66 6f 6f), would be encoded simply as the concatenation of these, namely the hex byte sequence:
36 06 66 6f 6f
Enums
An enum is encoded by a int, representing the zero-based position of the symbol in the schema.
For example, consider the enum:
{"type": "enum", "name": "Foo", "symbols": ["A", "B", "C", "D"] }
This would be encoded by an int between zero and three, with zero indicating “A”, and 3 indicating “D”.
Arrays
Arrays are encoded as a series of blocks. Each block consists of a long count value, followed by that many array items. A block with count zero indicates the end of the array. Each item is encoded per the array’s item schema.
If a block’s count is negative, its absolute value is used, and the count is followed immediately by a long block size indicating the number of bytes in the block. This block size permits fast skipping through data, e.g., when projecting a record to a subset of its fields.
For example, the array schema
{"type": "array", "items": "long"}
an array containing the items 3 and 27 could be encoded as the long value 2 (encoded as hex 04) followed by long values 3 and 27 (encoded as hex 06 36) terminated by zero:
04 06 36 00
The blocked representation permits one to read and write arrays larger than can be buffered in memory, since one can start writing items without knowing the full length of the array.
Maps
Maps are encoded as a series of blocks. Each block consists of a long count value, followed by that many key/value pairs. A block with count zero indicates the end of the map. Each item is encoded per the map’s value schema.
If a block’s count is negative, its absolute value is used, and the count is followed immediately by a long block size indicating the number of bytes in the block. This block size permits fast skipping through data, e.g., when projecting a record to a subset of its fields.
The blocked representation permits one to read and write maps larger than can be buffered in memory, since one can start writing items without knowing the full length of the map.
Unions
A union is encoded by first writing an int value indicating the zero-based position within the union of the schema of its value. The value is then encoded per the indicated schema within the union.
For example, the union schema ["null","string"] would encode:
-
null as zero (the index of “null” in the union):
00
-
the string “a” as one (the index of “string” in the union, 1, encoded as hex 02), followed by the serialized string:
02 02 61
NOTE: Currently for C/C++ implementtions, the positions are practically an int, but theoretically a long. In reality, we don’t expect unions with 215M members
Fixed
Fixed instances are encoded using the number of bytes declared in the schema.
JSON Encoding
Except for unions, the JSON encoding is the same as is used to encode field default values.
The value of a union is encoded in JSON as follows:
- if its type is null, then it is encoded as a JSON null;
- otherwise it is encoded as a JSON object with one name/value pair whose name is the type’s name and whose value is the recursively encoded value. For Avro’s named types (record, fixed or enum) the user-specified name is used, for other types the type name is used.
For example, the union schema ["null","string","Foo"], where Foo is a record name, would encode:
- null as null;
- the string “a” as
{"string": "a"} and
- a Foo instance as
{"Foo": {...}}, where {...} indicates the JSON encoding of a Foo instance.
Note that the original schema is still required to correctly process JSON-encoded data. For example, the JSON encoding does not distinguish between int and long, float and double, records and maps, enums and strings, etc.
Single-object encoding
In some situations a single Avro serialized object is to be stored for a longer period of time. One very common example is storing Avro records for several weeks in an Apache Kafka topic.
In the period after a schema change this persistence system will contain records that have been written with different schemas. So the need arises to know which schema was used to write a record to support schema evolution correctly. In most cases the schema itself is too large to include in the message, so this binary wrapper format supports the use case more effectively.
Single object encoding specification
Single Avro objects are encoded as follows:
- A two-byte marker,
C3 01, to show that the message is Avro and uses this single-record format (version 1).
- The 8-byte little-endian CRC-64-AVRO fingerprint of the object’s schema.
- The Avro object encoded using Avro’s binary encoding.
Implementations use the 2-byte marker to determine whether a payload is Avro. This check helps avoid expensive lookups that resolve the schema from a fingerprint, when the message is not an encoded Avro payload.
Sort Order
Avro defines a standard sort order for data. This permits data written by one system to be efficiently sorted by another system. This can be an important optimization, as sort order comparisons are sometimes the most frequent per-object operation. Note also that Avro binary-encoded data can be efficiently ordered without deserializing it to objects.
Data items may only be compared if they have identical schemas. Pairwise comparisons are implemented recursively with a depth-first, left-to-right traversal of the schema. The first mismatch encountered determines the order of the items.
Two items with the same schema are compared according to the following rules.
- null data is always equal.
- boolean data is ordered with false before true.
- int, long, float and double data is ordered by ascending numeric value.
- bytes and fixed data are compared lexicographically by unsigned 8-bit values.
- string data is compared lexicographically by Unicode code point. Note that since UTF-8 is used as the binary encoding for strings, sorting of bytes and string binary data is identical.
- array data is compared lexicographically by element.
- enum data is ordered by the symbol’s position in the enum schema. For example, an enum whose symbols are
["z", "a"] would sort “z” values before “a” values.
- union data is first ordered by the branch within the union, and, within that, by the type of the branch. For example, an
["int", "string"] union would order all int values before all string values, with the ints and strings themselves ordered as defined above.
- record data is ordered lexicographically by field. If a field specifies that its order is:
- “ascending”, then the order of its values is unaltered.
- “descending”, then the order of its values is reversed.
- “ignore”, then its values are ignored when sorting.
- map data may not be compared. It is an error to attempt to compare data containing maps unless those maps are in an
"order":"ignore" record field.
Object Container Files
Avro includes a simple object container file format. A file has a schema, and all objects stored in the file must be written according to that schema, using binary encoding. Objects are stored in blocks that may be compressed. Syncronization markers are used between blocks to permit efficient splitting of files for MapReduce processing.
Files may include arbitrary user-specified metadata.
A file consists of:
- A file header, followed by
- one or more file data blocks.
A file header consists of:
- Four bytes, ASCII ‘O’, ‘b’, ‘j’, followed by 1.
- file metadata, including the schema.
- The 16-byte, randomly-generated sync marker for this file.
File metadata is written as if defined by the following map schema:
{"type": "map", "values": "bytes"}
All metadata properties that start with “avro.” are reserved. The following file metadata properties are currently used:
- avro.schema contains the schema of objects stored in the file, as JSON data (required).
- avro.codec the name of the compression codec used to compress blocks, as a string. Implementations are required to support the following codecs: “null” and “deflate”. If codec is absent, it is assumed to be “null”. The codecs are described with more detail below.
A file header is thus described by the following schema:
{"type": "record", "name": "org.apache.avro.file.Header",
"fields" : [
{"name": "magic", "type": {"type": "fixed", "name": "Magic", "size": 4}},
{"name": "meta", "type": {"type": "map", "values": "bytes"}},
{"name": "sync", "type": {"type": "fixed", "name": "Sync", "size": 16}}
]
}
A file data block consists of:
- A long indicating the count of objects in this block.
- A long indicating the size in bytes of the serialized objects in the current block, after any codec is applied
- The serialized objects. If a codec is specified, this is compressed by that codec.
- The file’s 16-byte sync marker.
Thus, each block’s binary data can be efficiently extracted or skipped without deserializing the contents. The combination of block size, object counts, and sync markers enable detection of corrupt blocks and help ensure data integrity.
Required Codecs
null
The “null” codec simply passes through data uncompressed.
deflate
The “deflate” codec writes the data block using the deflate algorithm as specified in RFC 1951, and typically implemented using the zlib library. Note that this format (unlike the “zlib format” in RFC 1950) does not have a checksum.
Optional Codecs
bzip2
The “bzip2” codec uses the bzip2 compression library.
snappy
The “snappy” codec uses Google’s Snappy compression library. Each compressed block is followed by the 4-byte, big-endian CRC32 checksum of the uncompressed data in the block.
xz
The “xz” codec uses the XZ compression library.
zstandard
The “zstandard” codec uses Facebook’s Zstandard compression library.
Protocol Declaration
Avro protocols describe RPC interfaces. Like schemas, they are defined with JSON text.
A protocol is a JSON object with the following attributes:
- protocol, a string, the name of the protocol (required);
- namespace, an optional string that qualifies the name;
- doc, an optional string describing this protocol;
- types, an optional list of definitions of named types (records, enums, fixed and errors). An error definition is just like a record definition except it uses “error” instead of “record”. Note that forward references to named types are not permitted.
- messages, an optional JSON object whose keys are message names and whose values are objects whose attributes are described below. No two messages may have the same name.
The name and namespace qualification rules defined for schema objects apply to protocols as well.
Messages
A message has attributes:
- a doc, an optional description of the message,
- a request, a list of named, typed parameter schemas (this has the same form as the fields of a record declaration);
- a response schema;
- an optional union of declared error schemas. The effective union has “string” prepended to the declared union, to permit transmission of undeclared “system” errors. For example, if the declared error union is
["AccessError"], then the effective union is ["string", "AccessError"]. When no errors are declared, the effective error union is ["string"]. Errors are serialized using the effective union; however, a protocol’s JSON declaration contains only the declared union.
- an optional one-way boolean parameter.
A request parameter list is processed equivalently to an anonymous record. Since record field lists may vary between reader and writer, request parameters may also differ between the caller and responder, and such differences are resolved in the same manner as record field differences.
The one-way parameter may only be true when the response type is "null" and no errors are listed.
Sample Protocol
For example, one may define a simple HelloWorld protocol with:
{
"namespace": "com.acme",
"protocol": "HelloWorld",
"doc": "Protocol Greetings",
"types": [
{"name": "Greeting", "type": "record", "fields": [
{"name": "message", "type": "string"}]},
{"name": "Curse", "type": "error", "fields": [
{"name": "message", "type": "string"}]}
],
"messages": {
"hello": {
"doc": "Say hello.",
"request": [{"name": "greeting", "type": "Greeting" }],
"response": "Greeting",
"errors": ["Curse"]
}
}
}
Message Transport
Messages may be transmitted via different transport mechanisms.
To the transport, a message is an opaque byte sequence.
A transport is a system that supports:
- transmission of request messages
- receipt of corresponding response messages
Servers may send a response message back to the client corresponding to a request message. The mechanism of correspondance is transport-specific. For example, in HTTP it is implicit, since HTTP directly supports requests and responses. But a transport that multiplexes many client threads over a single socket would need to tag messages with unique identifiers.
Transports may be either stateless or stateful. In a stateless transport, messaging assumes no established connection state, while stateful transports establish connections that may be used for multiple messages. This distinction is discussed further in the handshake section below.
HTTP as Transport
When HTTP is used as a transport, each Avro message exchange is an HTTP request/response pair. All messages of an Avro protocol should share a single URL at an HTTP server. Other protocols may also use that URL. Both normal and error Avro response messages should use the 200 (OK) response code. The chunked encoding may be used for requests and responses, but, regardless the Avro request and response are the entire content of an HTTP request and response. The HTTP Content-Type of requests and responses should be specified as “avro/binary”. Requests should be made using the POST method.
HTTP is used by Avro as a stateless transport.
Message Framing
Avro messages are framed as a list of buffers.
Framing is a layer between messages and the transport. It exists to optimize certain operations.
The format of framed message data is:
- a series of buffers, where each buffer consists of:
- a four-byte, big-endian buffer length, followed by
- that many bytes of buffer data.
- A message is always terminated by a zero-length buffer.
Framing is transparent to request and response message formats (described below). Any message may be presented as a single or multiple buffers.
Framing can permit readers to more efficiently get different buffers from different sources and for writers to more efficiently store different buffers to different destinations. In particular, it can reduce the number of times large binary objects are copied. For example, if an RPC parameter consists of a megabyte of file data, that data can be copied directly to a socket from a file descriptor, and, on the other end, it could be written directly to a file descriptor, never entering user space.
A simple, recommended, framing policy is for writers to create a new segment whenever a single binary object is written that is larger than a normal output buffer. Small objects are then appended in buffers, while larger objects are written as their own buffers. When a reader then tries to read a large object the runtime can hand it an entire buffer directly, without having to copy it.
Handshake
The purpose of the handshake is to ensure that the client and the server have each other’s protocol definition, so that the client can correctly deserialize responses, and the server can correctly deserialize requests. Both clients and servers should maintain a cache of recently seen protocols, so that, in most cases, a handshake will be completed without extra round-trip network exchanges or the transmission of full protocol text.
RPC requests and responses may not be processed until a handshake has been completed. With a stateless transport, all requests and responses are prefixed by handshakes. With a stateful transport, handshakes are only attached to requests and responses until a successful handshake response has been returned over a connection. After this, request and response payloads are sent without handshakes for the lifetime of that connection.
The handshake process uses the following record schemas:
{
"type": "record",
"name": "HandshakeRequest", "namespace":"org.apache.avro.ipc",
"fields": [
{"name": "clientHash",
"type": {"type": "fixed", "name": "MD5", "size": 16}},
{"name": "clientProtocol", "type": ["null", "string"]},
{"name": "serverHash", "type": "MD5"},
{"name": "meta", "type": ["null", {"type": "map", "values": "bytes"}]}
]
}
{
"type": "record",
"name": "HandshakeResponse", "namespace": "org.apache.avro.ipc",
"fields": [
{"name": "match",
"type": {"type": "enum", "name": "HandshakeMatch",
"symbols": ["BOTH", "CLIENT", "NONE"]}},
{"name": "serverProtocol",
"type": ["null", "string"]},
{"name": "serverHash",
"type": ["null", {"type": "fixed", "name": "MD5", "size": 16}]},
{"name": "meta",
"type": ["null", {"type": "map", "values": "bytes"}]}
]
}
- A client first prefixes each request with a
HandshakeRequest containing just the hash of its protocol and of the server’s protocol (clientHash!=null, clientProtocol=null, serverHash!=null), where the hashes are 128-bit MD5 hashes of the JSON protocol text. If a client has never connected to a given server, it sends its hash as a guess of the server’s hash, otherwise it sends the hash that it previously obtained from this server.
The server responds with a HandshakeResponse containing one of:
-
match=BOTH, serverProtocol=null, serverHash=null if the client sent the valid hash of the server’s protocol and the server knows what protocol corresponds to the client’s hash. In this case, the request is complete and the response data immediately follows the HandshakeResponse.
-
match=CLIENT, serverProtocol!=null, serverHash!=null if the server has previously seen the client’s protocol, but the client sent an incorrect hash of the server’s protocol. The request is complete and the response data immediately follows the HandshakeResponse. The client must use the returned protocol to process the response and should also cache that protocol and its hash for future interactions with this server.
-
match=NONE if the server has not previously seen the client’s protocol. The serverHash and serverProtocol may also be non-null if the server’s protocol hash was incorrect.
In this case the client must then re-submit its request with its protocol text (clientHash!=null, clientProtocol!=null, serverHash!=null) and the server should respond with a successful match (match=BOTH, serverProtocol=null, serverHash=null) as above.
The meta field is reserved for future handshake enhancements.
A call consists of a request message paired with its resulting response or error message. Requests and responses contain extensible metadata, and both kinds of messages are framed as described above.
The format of a call request is:
- request metadata, a map with values of type bytes
- the message name, an Avro string, followed by
- the message parameters. Parameters are serialized according to the message’s request declaration.
When the empty string is used as a message name a server should ignore the parameters and return an empty response. A client may use this to ping a server or to perform a handshake without sending a protocol message.
When a message is declared one-way and a stateful connection has been established by a successful handshake response, no response data is sent. Otherwise the format of the call response is:
- response metadata, a map with values of type bytes
- a one-byte error flag boolean, followed by either:
- if the error flag is false, the message response, serialized per the message’s response schema.
- if the error flag is true, the error, serialized per the message’s effective error union schema.
Schema Resolution
A reader of Avro data, whether from an RPC or a file, can always parse that data because the original schema must be provided along with the data. However, the reader may be programmed to read data into a different schema. For example, if the data was written with a different version of the software than it is read, then fields may have been added or removed from records. This section specifies how such schema differences should be resolved.
We refer to the schema used to write the data as the writer’s schema, and the schema that the application expects the reader’s schema. Differences between these should be resolved as follows:
- It is an error if the two schemas do not match.
To match, one of the following must hold:
-
both schemas are arrays whose item types match
-
both schemas are maps whose value types match
-
both schemas are enums whose (unqualified) names match
-
both schemas are fixed whose sizes and (unqualified) names match
-
both schemas are records with the same (unqualified) name
-
either schema is a union
-
both schemas have same primitive type
-
the writer’s schema may be promoted to the reader’s as follows:
- int is promotable to long, float, or double
- long is promotable to float or double
- float is promotable to double
- string is promotable to bytes
- bytes is promotable to string
-
if both are records:
- the ordering of fields may be different: fields are matched by name.
- schemas for fields with the same name in both records are resolved recursively.
- if the writer’s record contains a field with a name not present in the reader’s record, the writer’s value for that field is ignored.
- if the reader’s record schema has a field that contains a default value, and writer’s schema does not have a field with the same name, then the reader should use the default value from its field.
- if the reader’s record schema has a field with no default value, and writer’s schema does not have a field with the same name, an error is signalled.
-
if both are enums:
if the writer’s symbol is not present in the reader’s enum and the reader has a default value, then that value is used, otherwise an error is signalled.
This resolution algorithm is applied recursively to the reader’s and writer’s array item schemas.
This resolution algorithm is applied recursively to the reader’s and writer’s value schemas.
The first schema in the reader’s union that matches the selected writer’s union schema is recursively resolved against it. if none match, an error is signalled.
- if reader’s is a union, but writer’s is not
The first schema in the reader’s union that matches the writer’s schema is recursively resolved against it. If none match, an error is signalled.
- if writer’s is a union, but reader’s is not
If the reader’s schema matches the selected writer’s schema, it is recursively resolved against it. If they do not match, an error is signalled.
A schema’s doc fields are ignored for the purposes of schema resolution. Hence, the doc portion of a schema may be dropped at serialization.
One of the defining characteristics of Avro is that a reader must use the schema used by the writer of the data in order to know how to read the data. This assumption results in a data format that’s compact and also amenable to many forms of schema evolution. However, the specification so far has not defined what it means for the reader to have the “same” schema as the writer. Does the schema need to be textually identical? Well, clearly adding or removing some whitespace to a JSON expression does not change its meaning. At the same time, reordering the fields of records clearly does change the meaning. So what does it mean for a reader to have “the same” schema as a writer?
Parsing Canonical Form is a transformation of a writer’s schema that let’s us define what it means for two schemas to be “the same” for the purpose of reading data written against the schema. It is called Parsing Canonical Form because the transformations strip away parts of the schema, like “doc” attributes, that are irrelevant to readers trying to parse incoming data. It is called Canonical Form because the transformations normalize the JSON text (such as the order of attributes) in a way that eliminates unimportant differences between schemas. If the Parsing Canonical Forms of two different schemas are textually equal, then those schemas are “the same” as far as any reader is concerned, i.e., there is no serialized data that would allow a reader to distinguish data generated by a writer using one of the original schemas from data generated by a writing using the other original schema. (We sketch a proof of this property in a companion document.)
The next subsection specifies the transformations that define Parsing Canonical Form. But with a well-defined canonical form, it can be convenient to go one step further, transforming these canonical forms into simple integers (“fingerprints”) that can be used to uniquely identify schemas. The subsection after next recommends some standard practices for generating such fingerprints.
Assuming an input schema (in JSON form) that’s already UTF-8 text for a valid Avro schema (including all quotes as required by JSON), the following transformations will produce its Parsing Canonical Form:
- [PRIMITIVES] Convert primitive schemas to their simple form (e.g., int instead of
{"type":"int"}).
- [FULLNAMES] Replace short names with fullnames, using applicable namespaces to do so. Then eliminate namespace attributes, which are now redundant.
- [STRIP] Keep only attributes that are relevant to parsing data, which are: type, name, fields, symbols, items, values, size. Strip all others (e.g., doc and aliases).
- [ORDER] Order the appearance of fields of JSON objects as follows: name, type, fields, symbols, items, values, size. For example, if an object has type, name, and size fields, then the name field should appear first, followed by the type and then the size fields.
- [STRINGS] For all JSON string literals in the schema text, replace any escaped characters (e.g., \uXXXX escapes) with their UTF-8 equivalents.
- [INTEGERS] Eliminate quotes around and any leading zeros in front of JSON integer literals (which appear in the size attributes of fixed schemas).
- [WHITESPACE] Eliminate all whitespace in JSON outside of string literals.
Schema Fingerprints
“[A] fingerprinting algorithm is a procedure that maps an arbitrarily large data item (such as a computer file) to a much shorter bit string, its fingerprint, that uniquely identifies the original data for all practical purposes” (quoted from Wikipedia). In the Avro context, fingerprints of Parsing Canonical Form can be useful in a number of applications; for example, to cache encoder and decoder objects, to tag data items with a short substitute for the writer’s full schema, and to quickly negotiate common-case schemas between readers and writers.
In designing fingerprinting algorithms, there is a fundamental trade-off between the length of the fingerprint and the probability of collisions. To help application designers find appropriate points within this trade-off space, while encouraging interoperability and ease of implementation, we recommend using one of the following three algorithms when fingerprinting Avro schemas:
- When applications can tolerate longer fingerprints, we recommend using the SHA-256 digest algorithm to generate 256-bit fingerprints of Parsing Canonical Forms. Most languages today have SHA-256 implementations in their libraries.
- At the opposite extreme, the smallest fingerprint we recommend is a 64-bit Rabin fingerprint. Below, we provide pseudo-code for this algorithm that can be easily translated into any programming language. 64-bit fingerprints should guarantee uniqueness for schema caches of up to a million entries (for such a cache, the chance of a collision is 3E-8). We don’t recommend shorter fingerprints, as the chances of collisions is too great (for example, with 32-bit fingerprints, a cache with as few as 100,000 schemas has a 50% chance of having a collision).
- Between these two extremes, we recommend using the MD5 message digest to generate 128-bit fingerprints. These make sense only where very large numbers of schemas are being manipulated (tens of millions); otherwise, 64-bit fingerprints should be sufficient. As with SHA-256, MD5 implementations are found in most libraries today.
These fingerprints are not meant to provide any security guarantees, even the longer SHA-256-based ones. Most Avro applications should be surrounded by security measures that prevent attackers from writing random data and otherwise interfering with the consumers of schemas. We recommend that these surrounding mechanisms be used to prevent collision and pre-image attacks (i.e., “forgery”) on schema fingerprints, rather than relying on the security properties of the fingerprints themselves.
Rabin fingerprints are cyclic redundancy checks computed using irreducible polynomials. In the style of the Appendix of RFC 1952 (pg 10), which defines the CRC-32 algorithm, here’s our definition of the 64-bit AVRO fingerprinting algorithm:
long fingerprint64(byte[] buf) {
if (FP_TABLE == null) initFPTable();
long fp = EMPTY;
for (int i = 0; i < buf.length; i++)
fp = (fp >>> 8) ^ FP_TABLE[(int)(fp ^ buf[i]) & 0xff];
return fp;
}
static long EMPTY = 0xc15d213aa4d7a795L;
static long[] FP_TABLE = null;
void initFPTable() {
FP_TABLE = new long[256];
for (int i = 0; i < 256; i++) {
long fp = i;
for (int j = 0; j < 8; j++)
fp = (fp >>> 1) ^ (EMPTY & -(fp & 1L));
FP_TABLE[i] = fp;
}
}
Readers interested in the mathematics behind this algorithm may want to read Chapter 14 of the Second Edition of Hacker’s Delight. (Unlike RFC-1952 and the book chapter, we prepend a single one bit to messages. We do this because CRCs ignore leading zero bits, which can be problematic. Our code prepends a one-bit by initializing fingerprints using EMPTY, rather than initializing using zero as in RFC-1952 and the book chapter.)
Logical Types
A logical type is an Avro primitive or complex type with extra attributes to represent a derived type. The attribute logicalType must always be present for a logical type, and is a string with the name of one of the logical types listed later in this section. Other attributes may be defined for particular logical types.
A logical type is always serialized using its underlying Avro type so that values are encoded in exactly the same way as the equivalent Avro type that does not have a logicalType attribute. Language implementations may choose to represent logical types with an appropriate native type, although this is not required.
Language implementations must ignore unknown logical types when reading, and should use the underlying Avro type. If a logical type is invalid, for example a decimal with scale greater than its precision, then implementations should ignore the logical type and use the underlying Avro type.
Decimal
The decimal logical type represents an arbitrary-precision signed decimal number of the form unscaled × 10-scale.
A decimal logical type annotates Avro bytes or fixed types. The byte array must contain the two’s-complement representation of the unscaled integer value in big-endian byte order. The scale is fixed, and is specified using an attribute.
The following attributes are supported:
- scale, a JSON integer representing the scale (optional). If not specified the scale is 0.
- precision, a JSON integer representing the (maximum) precision of decimals stored in this type (required).
For example, the following schema represents decimal numbers with a maximum precision of 4 and a scale of 2:
{
"type": "bytes",
"logicalType": "decimal",
"precision": 4,
"scale": 2
}
Precision must be a positive integer greater than zero. If the underlying type is a fixed, then the precision is limited by its size. An array of length n can store at most floor(log10(28 × n - 1 - 1)) base-10 digits of precision.
Scale must be zero or a positive integer less than or equal to the precision.
For the purposes of schema resolution, two schemas that are decimal logical types match if their scales and precisions match.
UUID
The uuid logical type represents a random generated universally unique identifier (UUID).
A uuid logical type annotates an Avro string. The string has to conform with RFC-4122
Date
The date logical type represents a date within the calendar, with no reference to a particular time zone or time of day.
A date logical type annotates an Avro int, where the int stores the number of days from the unix epoch, 1 January 1970 (ISO calendar).
The following schema represents a date:
{
"type": "int",
"logicalType": "date"
}
Time (millisecond precision)
The time-millis logical type represents a time of day, with no reference to a particular calendar, time zone or date, with a precision of one millisecond.
A time-millis logical type annotates an Avro int, where the int stores the number of milliseconds after midnight, 00:00:00.000.
Time (microsecond precision)
The time-micros logical type represents a time of day, with no reference to a particular calendar, time zone or date, with a precision of one microsecond.
A time-micros logical type annotates an Avro long, where the long stores the number of microseconds after midnight, 00:00:00.000000.
Timestamp (millisecond precision)
The timestamp-millis logical type represents an instant on the global timeline, independent of a particular time zone or calendar, with a precision of one millisecond. Please note that time zone information gets lost in this process. Upon reading a value back, we can only reconstruct the instant, but not the original representation. In practice, such timestamps are typically displayed to users in their local time zones, therefore they may be displayed differently depending on the execution environment.
A timestamp-millis logical type annotates an Avro long, where the long stores the number of milliseconds from the unix epoch, 1 January 1970 00:00:00.000 UTC.
Timestamp (microsecond precision)
The timestamp-micros logical type represents an instant on the global timeline, independent of a particular time zone or calendar, with a precision of one microsecond. Please note that time zone information gets lost in this process. Upon reading a value back, we can only reconstruct the instant, but not the original representation. In practice, such timestamps are typically displayed to users in their local time zones, therefore they may be displayed differently depending on the execution environment.
A timestamp-micros logical type annotates an Avro long, where the long stores the number of microseconds from the unix epoch, 1 January 1970 00:00:00.000000 UTC.
Local timestamp (millisecond precision)
The local-timestamp-millis logical type represents a timestamp in a local timezone, regardless of what specific time zone is considered local, with a precision of one millisecond.
A local-timestamp-millis logical type annotates an Avro long, where the long stores the number of milliseconds, from 1 January 1970 00:00:00.000.
Local timestamp (microsecond precision)
The local-timestamp-micros logical type represents a timestamp in a local timezone, regardless of what specific time zone is considered local, with a precision of one microsecond.
A local-timestamp-micros logical type annotates an Avro long, where the long stores the number of microseconds, from 1 January 1970 00:00:00.000000.
Duration
The duration logical type represents an amount of time defined by a number of months, days and milliseconds. This is not equivalent to a number of milliseconds, because, depending on the moment in time from which the duration is measured, the number of days in the month and number of milliseconds in a day may differ. Other standard periods such as years, quarters, hours and minutes can be expressed through these basic periods.
A duration logical type annotates Avro fixed type of size 12, which stores three little-endian unsigned integers that represent durations at different granularities of time. The first stores a number in months, the second stores a number in days, and the third stores a number in milliseconds.
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
5.4 - Trevni
Version 0.1
DRAFT
This document is the authoritative specification of a file format. Its intent is to permit compatible, independent implementations that read and/or write files in this format.
Introduction
Data sets are often described as a table composed of rows and columns. Each record in the dataset is considered a row, with each field of the record occupying a different column. Writing records to a file one-by-one as they are created results in a row-major format, like Hadoop’s SequenceFile or Avro data files.
In many cases higher query performance may be achieved if the data is instead organized in a column-major format, where multiple values of a given column are stored adjacently. This document defines such a column-major file format for datasets.
To permit scalable, distributed query evaluation, datasets are partitioned into row groups, containing distinct collections of rows. Each row group is organized in column-major order, while row groups form a row-major partitioning of the entire dataset.
Rationale
Goals
The format is meant satisfy the following goals:
Maximize the size of row groups. Disc drives are used most efficiently when sequentially accessing data. Consider a drive that takes 10ms to seek and transfers at 100MB/second. If a 10-column dataset whose values are all the same size is split into 10MB row groups, then accessing a single column will require a sequence of seek+1MB reads, for a cost of 20ms/MB processed. If the same dataset is split into 100MB row groups then this drops to 11ms/MB processed. This effect is exaggerated for datasets with larger numbers of columns and with columns whose values are smaller than average. So we’d prefer row groups that are 100MB or greater.
Permit random access within a row group. Some queries will first examine one column, and, only when certain relatively rare criteria are met, examine other columns. Rather than iterating through selected columns of the row-group in parallel, one might iterate through one column and randomly access another. This is called support for WHERE clauses, after the SQL operator of that name.
Minimize the number of files per dataset. HDFS is a primary intended deployment platform for these files. The HDFS Namenode requires memory for each file in the filesystem, thus for a format to be HDFS-friendly it should strive to require the minimum number of distinct files.
Support co-location of columns within row-groups. Row groups are the unit of parallel operation on a column dataset. For efficient file i/o, the entirety of a row-group should ideally reside on the host that is evaluating the query in order to avoid network latencies and bottlenecks.
Data integrity. The format should permit applications to detect data corruption. Many file systems may prevent corruption, but files may be moved between filesystems and be subject to corruption at points in that process. It is best if the data in a file can be validated independently.
Extensibility. The format should permit applications to store additional annotations about a datasets in the files, such as type information, origin, etc. Some environments may have metadata stores for such information, but not all do, and files might be moved among systems with different metadata systems. The ability to keep such information within the file simplifies the coordination of such information.
Minimal overhead. The column format should not make datasets appreciably larger. Storage is a primary cost and a choice to use this format should not require additional storage.
Primary format. The column format should be usable as a primary format for datasets, not as an auxiliary, accelerated format. Applications that process a dataset in row-major order should be able to easily consume column files and applications that produce datasets in row-major order should be able to easily generate column files.
Design
To meet these goals we propose the following design.
Each row group is a separate file. All values of a column in a file are written contiguously. This maximizes the row group size, optimizing performance when querying few and small columns.
Each file occupies a single HDFS block. A larger than normal block size may be specified, e.g., ~1GB instead of the typical ~100MB. This guarantees co-location and eliminates network use when query processing can be co-located with the file. This also moderates the memory impact on the HDFS Namenode since no small files are written.
Each column in a file is written as a sequence of ~64kB compressed blocks. The sequence is prefixed by a table describing all of the blocks in the column to permit random access within the column.
Application-specific metadata may be added at the file, column, and block levels.
Checksums are included with each block, providing data integrity.
Discussion
The use of a single block per file achieves the same effect as the custom block placement policy described in the CIF paper, but while still permitting HDFS rebalancing and not increasing the number of files in the namespace.
This section formally describes the proposed column file format.
Data Model
We assume a simple data model, where a record is a set of named fields, and the value of each field is a sequence of untyped bytes. A type system may be layered on top of this, as specified in the Type Mapping section below.
Primitive Values
We define the following primitive value types:
- Signed 64-bit long values are written using a variable-length zig-zag coding, where the high-order bit in each byte determines whether subsequent bytes are present. For example:
decimal value hex bytes
| decimal value |
hex bytes |
|
| 0 |
00 |
|
| -1 |
01 |
|
| 1 |
02 |
|
| … |
|
|
| -64 |
7f |
|
| 64 |
80 01 |
|
| … |
|
|
- bytes are encoded as a long followed by that many bytes of data.
- a string is encoded as a long followed by that many bytes of UTF-8 encoded character data.
For example, the three-character string “foo” would be encoded as the long value 3 (encoded as hex 06) followed by the UTF-8 encoding of ‘f’, ‘o’, and ‘o’ (the hex bytes 66 6f 6f): 06 66 6f 6f
Type Names
The following type names are used to describe column values:
- null, requires zero bytes. Sometimes used in array columns.
- boolean, one bit, packed into bytes, little-endian;
- int, like long, but restricted to 32-bit signed values
- long 64-bit signed values, represented as above
- fixed32 32-bit values stored as four bytes, little-endian.
- fixed64 64-bit values stored as eight bytes, little-endian.
- float 32-bit IEEE floating point value, little-endian
- double 64-bit IEEE floating point value, little-endian
- string as above
- bytes as above, may be used to encapsulate more complex objects
Type names are represented as strings (UTF-8 encoded, length-prefixed).
Metadata consists of:
- A long indicating the number of metadata key/value pairs.
- For each pair, a string key and bytes value.
All metadata properties that start with “trevni.” are reserved.
The following file metadata properties are defined:
- trevni.codec the name of the default compression codec used to compress blocks, as a string. Implementations are required to support the “null” codec. Optional. If absent, it is assumed to be “null”. Codecs are described in more detail below.
- trevni.checksum the name of the checksum algorithm used in this file, as a string. Implementations are required to support the “crc-32” checksum. Optional. If absent, it is assumed to be “null”. Checksums are described in more detail below.
The following column metadata properties are defined:
- trevni.codec the name of the compression codec used to compress the blocks of this column, as a string. Implementations are required to support the “null” codec. Optional. If absent, it is assumed to be “null”. Codecs are described in more detail below.
- trevni.name the name of the column, as a string. Required.
- trevni.type the type of data in the column. One of the type names above. Required.
- trevni.values if present, indicates that the initial value of each block in this column will be stored in the block’s descriptor. Not permitted for array columns or columns that specify a parent.
- trevni.array if present, indicates that each row in this column contains a sequence of values of the named type rather than just a single value. An integer length precedes each sequence of values indicating the count of values in the sequence. If the length is negative then it indicates a sequence of zero or one lengths, where -1 indicates two zeros, -2 two ones, -3 three zeros, -4 three ones, etc.
- trevni.parent if present, the name of an array column whose lengths are also used by this column. Thus values of this column are sequences but no lengths are stored in this column.
For example, consider the following row, as JSON, where all values are primitive types, but one has multiple values.
{"id"=566, "date"=23423234234
"from"="foo@bar.com",
"to"=["bar@baz.com", "bang@foo.com"],
"content"="Hi!"}
The columns for this might be specified as:
name=id type=int
name=date type=long
name=from type=string
name=to type=string array=true
name=content type=string
If a row contains an array of records, e.g. “received” in the following:
{"id"=566, "date"=23423234234
"from"="foo@bar.com",
"to"=["bar@baz.com", "bang@foo.com"],
"content"="Hi!"
"received"=[{"date"=234234234234, "host"="192.168.0.0.1"},
{"date"=234234545645, "host"="192.168.0.0.2"}]
}
Then one can define a parent column followed by a column for each field in the record, adding the following columns:
name=received type=null array=true
name=date type=long parent=received
name=host type=string parent=received
If an array value itself contains an array, e.g. the “sigs” below:
{"id"=566, "date"=23423234234
"from"="foo@bar.com",
"to"=["bar@baz.com", "bang@foo.com"],
"content"="Hi!"
"received"=[{"date"=234234234234, "host"="192.168.0.0.1",
"sigs"=[{"algo"="weak", "value"="0af345de"}]},
{"date"=234234545645, "host"="192.168.0.0.2",
"sigs"=[]}]
}
Then a parent column may be defined that itself has a parent column.
name=sigs type=null array=true parent=received
name=algo type=string parent=sigs
name=value type=string parent=sigs
No block metadata properties are currently defined.
A file consists of:
- A file header, followed by
- one or more columns.
A file header consists of:
- Four bytes, ASCII ‘T’, ‘r’, ‘v’, followed by 0x02.
- a fixed64 indicating the number of rows in the file
- a fixed32 indicating the number of columns in the file
- file metadata.
- for each column, its column metadata
- for each column, its starting position in the file as a fixed64.
A column consists of:
- A fixed32 indicating the number of blocks in this column.
- For each block, a block descriptor
- One or more blocks.
A block descriptor consists of:
- A fixed32 indicating the number of rows in the block
- A fixed32 indicating the size in bytes of the block before the codec is applied (excluding checksum).
- A fixed32 indicating the size in bytes of the block after the codec is applied (excluding checksum).
- If this column’s metadata declares it to include values, the first value in the column, serialized according to this column’s type.
A block consists of:
- The serialized column values. If a column is an array column then value sequences are preceded by their length, as an int. If a codec is specified, the values and lengths are compressed by that codec.
- The checksum, as determined by the file metadata.
Codecs
null
The “null” codec simply passes data through uncompressed.
deflate
The “deflate” codec writes the data block using the deflate algorithm as specified in RFC 1951.
snappy
The “snappy” codec uses Google’s Snappy compression library.
Checksum algorithms
null
The “null” checksum contains zero bytes.
crc-32
Each “crc-32” checksum contains the four bytes of an ISO 3309 CRC-32 checksum of the uncompressed block data as a fixed32.
Type Mappings
We define a standard mapping for how types defined in various serialization systems are represented in a column file. Records from these systems are shredded into columns. When records are nested, a depth-first recursive walk can assign a separate column for each primitive value.
Avro
Protocol Buffers
Thrift
Implementation Notes
Some possible techniques for writing column files include:
- Use a standard ~100MB block, buffer in memory up to the block size, then flush the file directly to HDFS. A single reduce task might create multiple output files. The namenode requires memory proportional to the number of names and blocks*replication. This would increase the number of names but not blocks, so this should still be much better than a file per column.
- Spill each column to a separate local, temporary file then, when the file is closed, append these files, writing a single file to HDFS whose block size is set to be that of the entire file. This would be a bit slower than and may have trouble when the local disk is full, but it would better use HDFS namespace and further reduce seeks when processing columns whose values are small.
- Use a separate mapreduce job to convert row-major files to column-major. The map output would output a by (row#, column#, value) tuple, partitioned by row# but sorted by column# then row#. The reducer could directly write the column file. But the column file format would need to be changed to write counts, descriptors, etc. at the end of files rather than at the front.
(1) is the simplest to implement and most implementations should start with it.
References
CIF Column-Oriented Storage Techniques for MapReduce, Floratou, Patel, Shekita, & Tata, VLDB 2011.
DREMEL Dremel: Interactive Analysis of Web-Scale Datasets, Melnik, Gubarev, Long, Romer, Shivakumar, & Tolton, VLDB 2010.
5.5 - MapReduce guide
Avro provides a convenient way to represent complex data structures within a Hadoop MapReduce job. Avro data can be used as both input to and output from a MapReduce job, as well as the intermediate format. The example in this guide uses Avro data for all three, but it’s possible to mix and match; for instance, MapReduce can be used to aggregate a particular field in an Avro record.
This guide assumes basic familiarity with both Hadoop MapReduce and Avro. See the Hadoop documentation and the Avro getting started guide for introductions to these projects. This guide uses the old MapReduce API (org.apache.hadoop.mapred) and the new MapReduce API (org.apache.hadoop.mapreduce).
Setup
The code from this guide is included in the Avro docs under examples/mr-example. The example is set up as a Maven project that includes the necessary Avro and MapReduce dependencies and the Avro Maven plugin for code generation, so no external jars are needed to run the example. In particular, the POM includes the following dependencies:
<dependency>
<groupId>org.apache.avro</groupId>
<artifactId>avro</artifactId>
<version>1.10.0</version>
</dependency>
<dependency>
<groupId>org.apache.avro</groupId>
<artifactId>avro-mapred</artifactId>
<version>1.10.0</version>
</dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-client</artifactId>
<version>3.1.2</version>
</dependency>
And the following plugin:
<plugin>
<groupId>org.apache.avro</groupId>
<artifactId>avro-maven-plugin</artifactId>
<version>1.10.0</version>
<executions>
<execution>
<phase>generate-sources</phase>
<goals>
<goal>schema</goal>
</goals>
<configuration>
<sourceDirectory>${project.basedir}/../</sourceDirectory>
<outputDirectory>${project.basedir}/target/generated-sources/</outputDirectory>
</configuration>
</execution>
</executions>
</plugin>
If you do not configure the sourceDirectory and outputDirectory properties, the defaults will be used. The sourceDirectory property defaults to src/main/avro. The outputDirectory property defaults to target/generated-sources. You can change the paths to match your project layout.
Alternatively, Avro jars can be downloaded directly from the Apache Avro™ Releases page. The relevant Avro jars for this guide are avro-1.10.0.jar and avro-mapred-1.10.0.jar, as well as avro-tools-1.10.0.jar for code generation and viewing Avro data files as JSON. In addition, you will need to install Hadoop in order to use MapReduce.
Example: ColorCount
Below is a simple example of a MapReduce that uses Avro. There is an example for both the old (org.apache.hadoop.mapred) and new (org.apache.hadoop.mapreduce) APIs under examples/mr-example/src/main/java/example/. MapredColorCount is the example for the older mapred API while MapReduceColorCount is the example for the newer mapreduce API. Both examples are below, but we will detail the mapred API in our subsequent examples.
MapredColorCount:
package example;
import java.io.IOException;
import org.apache.avro.*;
import org.apache.avro.Schema.Type;
import org.apache.avro.mapred.*;
import org.apache.hadoop.conf.*;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.mapred.*;
import org.apache.hadoop.util.*;
import example.avro.User;
public class MapredColorCount extends Configured implements Tool {
public static class ColorCountMapper extends AvroMapper<User, Pair<CharSequence, Integer>> {
@Override
public void map(User user, AvroCollector<Pair<CharSequence, Integer>> collector, Reporter reporter)
throws IOException {
CharSequence color = user.getFavoriteColor();
// We need this check because the User.favorite_color field has type ["string", "null"]
if (color == null) {
color = "none";
}
collector.collect(new Pair<CharSequence, Integer>(color, 1));
}
}
public static class ColorCountReducer extends AvroReducer<CharSequence, Integer,
Pair<CharSequence, Integer>> {
@Override
public void reduce(CharSequence key, Iterable<Integer> values,
AvroCollector<Pair<CharSequence, Integer>> collector,
Reporter reporter)
throws IOException {
int sum = 0;
for (Integer value : values) {
sum += value;
}
collector.collect(new Pair<CharSequence, Integer>(key, sum));
}
}
public int run(String[] args) throws Exception {
if (args.length != 2) {
System.err.println("Usage: MapredColorCount <input path> <output path>");
return -1;
}
JobConf conf = new JobConf(getConf(), MapredColorCount.class);
conf.setJobName("colorcount");
FileInputFormat.setInputPaths(conf, new Path(args[0]));
FileOutputFormat.setOutputPath(conf, new Path(args[1]));
AvroJob.setMapperClass(conf, ColorCountMapper.class);
AvroJob.setReducerClass(conf, ColorCountReducer.class);
// Note that AvroJob.setInputSchema and AvroJob.setOutputSchema set
// relevant config options such as input/output format, map output
// classes, and output key class.
AvroJob.setInputSchema(conf, User.getClassSchema());
AvroJob.setOutputSchema(conf, Pair.getPairSchema(Schema.create(Type.STRING),
Schema.create(Type.INT)));
JobClient.runJob(conf);
return 0;
}
public static void main(String[] args) throws Exception {
int res = ToolRunner.run(new Configuration(), new MapredColorCount(), args);
System.exit(res);
}
}
MapReduceColorCount:
package example;
import java.io.IOException;
import org.apache.avro.Schema;
import org.apache.avro.mapred.AvroKey;
import org.apache.avro.mapred.AvroValue;
import org.apache.avro.mapreduce.AvroJob;
import org.apache.avro.mapreduce.AvroKeyInputFormat;
import org.apache.avro.mapreduce.AvroKeyValueOutputFormat;
import org.apache.hadoop.conf.Configured;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.IntWritable;
import org.apache.hadoop.io.NullWritable;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.mapreduce.Job;
import org.apache.hadoop.mapreduce.Mapper;
import org.apache.hadoop.mapreduce.Reducer;
import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
import org.apache.hadoop.util.Tool;
import org.apache.hadoop.util.ToolRunner;
import example.avro.User;
public class MapReduceColorCount extends Configured implements Tool {
public static class ColorCountMapper extends
Mapper<AvroKey<User>, NullWritable, Text, IntWritable> {
@Override
public void map(AvroKey<User> key, NullWritable value, Context context)
throws IOException, InterruptedException {
CharSequence color = key.datum().getFavoriteColor();
if (color == null) {
color = "none";
}
context.write(new Text(color.toString()), new IntWritable(1));
}
}
public static class ColorCountReducer extends
Reducer<Text, IntWritable, AvroKey<CharSequence>, AvroValue<Integer>> {
@Override
public void reduce(Text key, Iterable<IntWritable> values,
Context context) throws IOException, InterruptedException {
int sum = 0;
for (IntWritable value : values) {
sum += value.get();
}
context.write(new AvroKey<CharSequence>(key.toString()), new AvroValue<Integer>(sum));
}
}
public int run(String[] args) throws Exception {
if (args.length != 2) {
System.err.println("Usage: MapReduceColorCount <input path> <output path>");
return -1;
}
Job job = new Job(getConf());
job.setJarByClass(MapReduceColorCount.class);
job.setJobName("Color Count");
FileInputFormat.setInputPaths(job, new Path(args[0]));
FileOutputFormat.setOutputPath(job, new Path(args[1]));
job.setInputFormatClass(AvroKeyInputFormat.class);
job.setMapperClass(ColorCountMapper.class);
AvroJob.setInputKeySchema(job, User.getClassSchema());
job.setMapOutputKeyClass(Text.class);
job.setMapOutputValueClass(IntWritable.class);
job.setOutputFormatClass(AvroKeyValueOutputFormat.class);
job.setReducerClass(ColorCountReducer.class);
AvroJob.setOutputKeySchema(job, Schema.create(Schema.Type.STRING));
AvroJob.setOutputValueSchema(job, Schema.create(Schema.Type.INT));
return (job.waitForCompletion(true) ? 0 : 1);
}
public static void main(String[] args) throws Exception {
int res = ToolRunner.run(new MapReduceColorCount(), args);
System.exit(res);
}
}
ColorCount reads in data files containing User records, defined in examples/user.avsc, and counts the number of instances of each favorite color. (This example draws inspiration from the canonical WordCount MapReduce application.) This example uses the old MapReduce API. See MapReduceAvroWordCount, found under doc/examples/mr-example/src/main/java/example/ to see the new MapReduce API example. The User schema is defined as follows:
{"namespace": "example.avro",
"type": "record",
"name": "User",
"fields": [
{"name": "name", "type": "string"},
{"name": "favorite_number", "type": ["int", "null"]},
{"name": "favorite_color", "type": ["string", "null"]}
]
}
This schema is compiled into the User class used by ColorCount via the Avro Maven plugin (see examples/mr-example/pom.xml for how this is set up).
ColorCountMapper essentially takes a User as input and extracts the User’s favorite color, emitting the key-value pair <favoriteColor, 1>. ColorCountReducer then adds up how many occurrences of a particular favorite color were emitted, and outputs the result as a Pair record. These Pairs are serialized to an Avro data file.
Running ColorCount
The ColorCount application is provided as a Maven project in the Avro docs under examples/mr-example. To build the project, including the code generation of the User schema, run:
Next, run GenerateData from examples/mr-examples to create an Avro data file, input/users.avro, containing 20 Users with favorite colors chosen randomly from a list:
mvn exec:java -q -Dexec.mainClass=example.GenerateData
Besides creating the data file, GenerateData prints the JSON representations of the Users generated to stdout, for example:
{"name": "user", "favorite_number": null, "favorite_color": "red"}
{"name": "user", "favorite_number": null, "favorite_color": "green"}
{"name": "user", "favorite_number": null, "favorite_color": "purple"}
{"name": "user", "favorite_number": null, "favorite_color": null}
...
Now we’re ready to run ColorCount. We specify our freshly-generated input folder as the input path and output as our output folder (note that MapReduce will not start a job if the output folder already exists):
mvn exec:java -q -Dexec.mainClass=example.MapredColorCount -Dexec.args="input output"
Once ColorCount completes, checking the contents of the new output directory should yield the following:
$ ls output/
part-00000.avro _SUCCESS
You can check the contents of the generated Avro file using the avro-tools jar:
$ java -jar /path/to/avro-tools-1.10.0.jar tojson output/part-00000.avro
{"value": 3, "key": "blue"}
{"value": 7, "key": "green"}
{"value": 1, "key": "none"}
{"value": 2, "key": "orange"}
{"value": 3, "key": "purple"}
{"value": 2, "key": "red"}
{"value": 2, "key": "yellow"}
Now let’s go over the ColorCount example in detail.
Mapper - org.apache.hadoop.mapred API
The easiest way to use Avro data files as input to a MapReduce job is to subclass AvroMapper. An AvroMapper defines a map function that takes an Avro datum as input and outputs a key/value pair represented as a Pair record. In the ColorCount example, ColorCountMapper is an AvroMapper that takes a User as input and outputs a Pair<CharSequence, Integer>>, where the CharSequence key is the user’s favorite color and the Integer value is 1.
public static class ColorCountMapper extends AvroMapper<User, Pair<CharSequence, Integer>> {
@Override
public void map(User user, AvroCollector<Pair<CharSequence, Integer>> collector, Reporter reporter)
throws IOException {
CharSequence color = user.getFavoriteColor();
// We need this check because the User.favorite_color field has type ["string", "null"]
if (color == null) {
color = "none";
}
collector.collect(new Pair<CharSequence, Integer>(color, 1));
}
}
In order to use our AvroMapper, we must call AvroJob.setMapperClass and AvroJob.setInputSchema.
AvroJob.setMapperClass(conf, ColorCountMapper.class);
AvroJob.setInputSchema(conf, User.getClassSchema());
Note that AvroMapper does not implement the Mapper interface. Under the hood, the specified Avro data files are deserialized into AvroWrappers containing the actual data, which are processed by a Mapper that calls the configured AvroMapper’s map function. AvroJob.setInputSchema sets up the relevant configuration parameters needed to make this happen, thus you should not need to call JobConf.setMapperClass, JobConf.setInputFormat, JobConf.setMapOutputKeyClass, JobConf.setMapOutputValueClass, or JobConf.setOutputKeyComparatorClass.
Mapper - org.apache.hadoop.mapreduce API
This document will not go into all the differences between the mapred and mapreduce APIs, however will describe the main differences. As you can see, ColorCountMapper is now a subclass of the Hadoop Mapper class and is passed an AvroKey as it’s key. Additionally, the AvroJob method calls were slightly changed.
public static class ColorCountMapper extends
Mapper<AvroKey<User>, NullWritable, Text, IntWritable> {
@Override
public void map(AvroKey<User> key, NullWritable value, Context context)
throws IOException, InterruptedException {
CharSequence color = key.datum().getFavoriteColor();
if (color == null) {
color = "none";
}
context.write(new Text(color.toString()), new IntWritable(1));
}
}
Reducer - org.apache.hadoop.mapred API
Analogously to AvroMapper, an AvroReducer defines a reducer function that takes the key/value types output by an AvroMapper (or any mapper that outputs Pairs) and outputs a key/value pair represented a Pair record. In the ColorCount example, ColorCountReducer is an AvroReducer that takes the CharSequence key representing a favorite color and the Iterable<Integer> representing the counts for that color (they should all be 1 in this example) and adds up the counts.
public static class ColorCountReducer extends AvroReducer<CharSequence, Integer,
Pair<CharSequence, Integer>> {
@Override
public void reduce(CharSequence key, Iterable<Integer> values,
AvroCollector<Pair<CharSequence, Integer>> collector,
Reporter reporter)
throws IOException {
int sum = 0;
for (Integer value : values) {
sum += value;
}
collector.collect(new Pair<CharSequence, Integer>(key, sum));
}
}
In order to use our AvroReducer, we must call AvroJob.setReducerClass and AvroJob.setOutputSchema.
AvroJob.setReducerClass(conf, ColorCountReducer.class);
AvroJob.setOutputSchema(conf, Pair.getPairSchema(Schema.create(Type.STRING),
Schema.create(Type.INT)));
Note that AvroReducer does not implement the Reducer interface. The intermediate Pairs output by the mapper are split into AvroKeys and AvroValues, which are processed by a Reducer that calls the configured AvroReducer’s reduce function. AvroJob.setOutputSchema sets up the relevant configuration parameters needed to make this happen, thus you should not need to call JobConf.setReducerClass, JobConf.setOutputFormat, JobConf.setOutputKeyClass, JobConf.setMapOutputKeyClass, JobConf.setMapOutputValueClass, or JobConf.setOutputKeyComparatorClass.
Reduce - org.apache.hadoop.mapreduce API
As before we not detail every difference between the APIs. As with the Mapper change ColorCountReducer is now a subclass of Reducer and AvroKey and AvroValue are emitted. Additionally, the AvroJob method calls were slightly changed.
public static class ColorCountReducer extends
Reducer<Text, IntWritable, AvroKey<CharSequence>, AvroValue<Integer>> {
@Override
public void reduce(Text key, Iterable<IntWritable> values,
Context context) throws IOException, InterruptedException {
int sum = 0;
for (IntWritable value : values) {
sum += value.get();
}
context.write(new AvroKey<CharSequence>(key.toString()), new AvroValue<Integer>(sum));
}
}
Learning more
The mapred API allows users to mix Avro AvroMappers and AvroReducers with non-Avro Mappers and Reducers and the mapreduce API allows users input Avro and output non-Avro or vice versa.
The mapred package has API org.apache.avro.mapred documentation as does the org.apache.avro.mapreduce package. MapReduce API (org.apache.hadoop.mapreduce). Similarily to the mapreduce package, it’s possible with the mapred API to implement your own Mappers and Reducers directly using the public classes provided in these libraries. See the AvroWordCount application, found under examples/mr-example/src/main/java/example/AvroWordCount.java in the Avro documentation, for an example of implementing a Reducer that outputs Avro data using the old MapReduce API. See the MapReduceAvroWordCount application, found under examples/mr-example/src/main/java/example/MapReduceAvroWordCount.java in the Avro documentation, for an example of implementing a Reducer that outputs Avro data using the new MapReduce API.
5.6 - IDL Language
Introduction
This document defines Avro IDL, a higher-level language for authoring Avro schemata. Before reading this document, you should have familiarity with the concepts of schemata and protocols, as well as the various primitive and complex types available in Avro.
Overview
Purpose
The aim of the Avro IDL language is to enable developers to author schemata in a way that feels more similar to common programming languages like Java, C++, or Python. Additionally, the Avro IDL language may feel more familiar for those users who have previously used the interface description languages (IDLs) in other frameworks like Thrift, Protocol Buffers, or CORBA.
Usage
Each Avro IDL file defines a single Avro Protocol, and thus generates as its output a JSON-format Avro Protocol file with extension .avpr.
To convert a .avdl file into a .avpr file, it may be processed by the idl tool. For example:
$ java -jar avro-tools.jar idl src/test/idl/input/namespaces.avdl /tmp/namespaces.avpr
$ head /tmp/namespaces.avpr
{
"protocol" : "TestNamespace",
"namespace" : "avro.test.protocol",
The idl tool can also process input to and from stdin and stdout. See idl --help for full usage information.
A Maven plugin is also provided to compile .avdl files. To use it, add something like the following to your pom.xml:
<build>
<plugins>
<plugin>
<groupId>org.apache.avro</groupId>
<artifactId>avro-maven-plugin</artifactId>
<executions>
<execution>
<goals>
<goal>idl-protocol</goal>
</goals>
</execution>
</executions>
</plugin>
</plugins>
</build>
Defining a Protocol in Avro IDL
An Avro IDL file consists of exactly one protocol definition. The minimal protocol is defined by the following code:
This is equivalent to (and generates) the following JSON protocol definition:
{
"protocol" : "MyProtocol",
"types" : [ ],
"messages" : {
}
}
The namespace of the protocol may be changed using the @namespace annotation:
@namespace("mynamespace")
protocol MyProtocol {
}
This notation is used throughout Avro IDL as a way of specifying properties for the annotated element, as will be described later in this document.
Protocols in Avro IDL can contain the following items:
- Imports of external protocol and schema files.
- Definitions of named schemata, including records, errors, enums, and fixeds.
- Definitions of RPC messages
Imports
Files may be imported in one of three formats:
-
An IDL file may be imported with a statement like:
import idl "foo.avdl";
-
A JSON protocol file may be imported with a statement like:
import protocol "foo.avpr";
-
A JSON schema file may be imported with a statement like:
import schema "foo.avsc";
Messages and types in the imported file are added to this file’s protocol.
Imported file names are resolved relative to the current IDL file.
Defining an Enumeration
Enums are defined in Avro IDL using a syntax similar to C or Java. An Avro Enum supports optional default values. In the case that a reader schema is unable to recognize a symbol written by the writer, the reader will fall back to using the defined default value. This default is only used when an incompatible symbol is read. It is not used if the enum field is missing.
Example Writer Enum Definition
enum Shapes {
SQUARE, TRIANGLE, CIRCLE, OVAL
}
Example Reader Enum Definition
enum Shapes {
SQUARE, TRIANGLE, CIRCLE
} = CIRCLE;
In the above example, the reader will use the default value of CIRCLE whenever reading data written with the OVAL symbol of the writer. Also note that, unlike the JSON format, anonymous enums cannot be defined.
Defining a Fixed Length Field
Fixed fields are defined using the following syntax:
fixed MD5(16);
This example defines a fixed-length type called MD5 which contains 16 bytes.
Defining Records and Errors
Records are defined in Avro IDL using a syntax similar to a struct definition in C:
record Employee {
string name;
boolean active = true;
long salary;
}
The above example defines a record with the name “Employee” with three fields.
To define an error, simply use the keyword error instead of record. For example:
error Kaboom {
string explanation;
int result_code = -1;
}
Each field in a record or error consists of a type and a name, optional property annotations and an optional default value.
A type reference in Avro IDL must be one of:
- A primitive type
- A logical type
- A named schema defined prior to this usage in the same Protocol
- A complex type (array, map, or union)
Primitive Types
The primitive types supported by Avro IDL are the same as those supported by Avro’s JSON format. This list includes int, long, string, boolean, float, double, null, and bytes.
Logical Types
Some of the logical types supported by Avro’s JSON format are also supported by Avro IDL. The currently supported types are:
For example:
record Job {
string jobid;
date submitDate;
time_ms submitTime;
timestamp_ms finishTime;
decimal(9,2) finishRatio;
}
References to Named Schemata
If a named schema has already been defined in the same Avro IDL file, it may be referenced by name as if it were a primitive type:
record Card {
Suit suit; // refers to the enum Card defined above
int number;
}
Default Values
Default values for fields may be optionally specified by using an equals sign after the field name followed by a JSON expression indicating the default value. This JSON is interpreted as described in the spec.
Complex Types
Arrays
Array types are written in a manner that will seem familiar to C++ or Java programmers. An array of any type t is denoted array<t>. For example, an array of strings is denoted array<string>, and a multidimensional array of Foo records would be array<array<Foo>>.
Maps
Map types are written similarly to array types. An array that contains values of type t is written map<t>. As in the JSON schema format, all maps contain string-type keys.
Unions
Union types are denoted as union { typeA, typeB, typeC, … }. For example, this record contains a string field that is optional (unioned with null):
record RecordWithUnion {
union { null, string } optionalString;
}
Note that the same restrictions apply to Avro IDL unions as apply to unions defined in the JSON format; namely, a record may not contain multiple elements of the same type.
Defining RPC Messages
The syntax to define an RPC message within a Avro IDL protocol is similar to the syntax for a method declaration within a C header file or a Java interface. To define an RPC message add which takes two arguments named foo and bar, returning an int, simply include the following definition within the protocol:
int add(int foo, int bar = 0);
Message arguments, like record fields, may specify default values.
To define a message with no response, you may use the alias void, equivalent to the Avro null type:
void logMessage(string message);
If you have previously defined an error type within the same protocol, you may declare that a message can throw this error using the syntax:
void goKaboom() throws Kaboom;
To define a one-way message, use the keyword oneway after the parameter list, for example:
void fireAndForget(string message) oneway;
Other Language Features
All Java-style comments are supported within a Avro IDL file. Any text following // on a line is ignored, as is any text between /* and */, possibly spanning multiple lines.
Comments that begin with /** are used as the documentation string for the type or field definition that follows the comment.
Escaping Identifiers
Occasionally, one will need to use a reserved language keyword as an identifier. In order to do so, backticks (`) may be used to escape the identifier. For example, to define a message with the literal name error, you may write:
This syntax is allowed anywhere an identifier is expected.
Annotations for Ordering and Namespaces
Java-style annotations may be used to add additional properties to types and fields throughout Avro IDL.
For example, to specify the sort order of a field within a record, one may use the @order annotation before the field name as follows:
record MyRecord {
string @order("ascending") myAscendingSortField;
string @order("descending") myDescendingField;
string @order("ignore") myIgnoredField;
}
A field’s type may also be preceded by annotations, e.g.:
record MyRecord {
@java-class("java.util.ArrayList") array<string> myStrings;
}
This can be used to support java classes that can be serialized/deserialized via their toString/String constructor, e.g.:
record MyRecord {
@java-class("java.math.BigDecimal") string value;
@java-key-class("java.io.File") map<string> fileStates;
array<@java-class("java.math.BigDecimal") string> weights;
}
Similarly, a @namespace annotation may be used to modify the namespace when defining a named schema. For example:
@namespace("org.apache.avro.firstNamespace")
protocol MyProto {
@namespace("org.apache.avro.someOtherNamespace")
record Foo {}
record Bar {}
}
will define a protocol in the firstNamespace namespace. The record Foo will be defined in someOtherNamespace and Bar will be defined in firstNamespace as it inherits its default from its container.
Type and field aliases are specified with the @aliases annotation as follows:
@aliases(["org.old.OldRecord", "org.ancient.AncientRecord"])
record MyRecord {
string @aliases(["oldField", "ancientField"]) myNewField;
}
Some annotations like those listed above are handled specially. All other annotations are added as properties to the protocol, message, schema or field.
Complete Example
The following is a complete example of a Avro IDL file that shows most of the above features:
/**
* An example protocol in Avro IDL
*/
@namespace("org.apache.avro.test")
protocol Simple {
@aliases(["org.foo.KindOf"])
enum Kind {
FOO,
BAR, // the bar enum value
BAZ
}
fixed MD5(16);
record TestRecord {
@order("ignore")
string name;
@order("descending")
Kind kind;
MD5 hash;
union { MD5, null} @aliases(["hash"]) nullableHash;
array<long> arrayOfLongs;
}
error TestError {
string message;
}
string hello(string greeting);
TestRecord echo(TestRecord `record`);
int add(int arg1, int arg2);
bytes echoBytes(bytes data);
void `error`() throws TestError;
void ping() oneway;
}
Additional examples may be found in the Avro source tree under the src/test/idl/input directory.
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
5.7 - SASL profile
Introduction
SASL (RFC 2222) provides a framework for authentication and security of network protocols. Each protocol that uses SASL is meant to define a SASL profile. This document provides a SASL profile for connection-based Avro RPC.
Overview
SASL negotiation proceeds as a series of message interactions over a connection between a client and server using a selected SASL mechanism. The client starts this negotiation by sending its chosen mechanism name with an initial (possibly empty) message. Negotiation proceeds with the exchange of messages until either side indicates success or failure. The content of the messages is mechanism-specific. If the negotiation succeeds, then the session can proceed over the connection, otherwise it must be abandoned.
Some mechanisms continue to process session data after negotiation (e.g., encrypting it), while some specify that further session data is transmitted unmodified.
Negotiation
Commands
Avro SASL negotiation uses four one-byte commands.
- 0: START Used in a client’s initial message.
- 1: CONTINUE Used while negotiation is ongoing.
- 2: FAIL Terminates negotiation unsuccessfully.
- 3: COMPLETE Terminates negotiation successfully.
The format of a START message is:
| 0 | 4-byte mechanism name length | mechanism name | 4-byte payload length | payload data |
The format of a CONTINUE message is:
| 1 | 4-byte payload length | payload data |
The format of a FAIL message is:
| 2 | 4-byte message length | UTF-8 message |
The format of a COMPLETE message is:
| 3 | 4-byte payload length | payload data |
Process
Negotiation is initiated by a client sending a START command containing the client’s chosen mechanism name and any mechanism-specific payload data.
The server and client then interchange some number (possibly zero) of CONTINUE messages. Each message contains payload data that is processed by the security mechanism to generate the next message.
Once either the client or server send a FAIL message then negotiation has failed. UTF-8-encoded text is included in the failure message. Once either a FAIL message has been sent or received, or any other error occurs in the negotiation, further communication on this connection must cease.
Once either the client or server send a COMPLETE message then negotiation has completed successfully. Session data may now be transmitted over the connection until it is closed by either side.
Session Data
If no SASL QOP (quality of protection) is negotiated, then all subsequent writes to/reads over this connection are written/read unmodified. In particular, messages use Avro framing, and are of the form:
| 4-byte frame length | frame data | ... | 4 zero bytes |
If a SASL QOP is negotiated, then it must be used by the connection for all subsequent messages. This is done by wrapping each non-empty frame written using the security mechanism and unwrapping each non-empty frame read. The length written in each non-empty frame is the length of the wrapped data. Complete frames must be passed to the security mechanism for unwrapping. Unwrapped data is then passed to the application as the content of the frame.
If at any point processing fails due to wrapping, unwrapping or framing errors, then all further communication on this connection must cease.
Anonymous Mechanism
The SASL anonymous mechanism (RFC 2245) is quite simple to implement. In particular, an initial anonymous request may be prefixed by the following static sequence:
| 0 | 0009 | ANONYMOUS | 0000 |
If a server uses the anonymous mechanism, it should check that the mechanism name in the start message prefixing the first request received is ‘ANONYMOUS’, then simply prefix its initial response with a COMPLETE message of:
| 3 | 0000 |
If an anonymous server recieves some other mechanism name, then it may respond with a FAIL message as simple as:
| 2 | 0000 |
Note that the anonymous mechanism need add no additional round-trip messages between client and server. The START message can be piggybacked on the initial request and the COMPLETE or FAIL message can be piggybacked on the initial response.
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
5.8 - Wiki
The Wiki page can be found here.
5.9 - FAQ
What is Avro?
Avro is a data serialization system.
Avro provides:
- Rich data structures.
- A compact, fast, binary data format.
- A container file, to store persistent data.
- Remote procedure call (RPC).
- Simple integration with dynamic languages. Code generation is not required to read or write data files nor to use or implement RPC protocols. Code generation as an optional optimization, only worth implementing for statically typed languages.
How can I get started quickly with Avro?
Check out the Quick Start Guide.
How do I statically compile a schema or protocol into generated code?
In Java
- Add the avro jar, the jackson-mapper-asl.jar and jackson-core-asl.jar to your CLASSPATH.
- Run java org.apache.avro.specific.SpecificCompiler .
This appears to be out of date, the SpecificCompiler requires two arguments, presumably an input and and output file, but it isn’t clear that this does.
Or use the Schema or Protocol Ant tasks. Avro’s build.xml provides examples of how these are used.
Lastly, you can also use the “avro-tools” jar which ships with an Avro release. Just use the “compile (schema|protocol)” command.
How are Strings represented in Java?
They use org.apache.avro.util.Utf8, not java.lang.String.
More generally, how do Avro types map to Java types?
The mappings are documented in the package javadoc for generic, specific and reflect API.
From Doug Cutting:
HDFS splits files into blocks, and mapreduce runs a map task for each block. When the task starts, it needs to be able to seek into the file to the start of the block process through the block’s end. If the file were, e.g., a gzip file, this would not be possible, since gzip files must be decompressed from the start. One cannot seek into the middle of a gzip file and start decompressing. So Hadoop’s SequenceFile places a marker periodically (~64k) in the file at record and compression boundaries, where processing can be sensibly started. Then, when a map task starts processing an HDFS block, it finds the first marker after the block’s start and continues through the first marker in the next block of the file. This requires a bit of non-local access (~0.1%). Avro’s data file uses the same method as SequenceFile.
Why isn’t every value in Avro nullable?
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 of every value stored and
corresponding computational costs to create this bit on write and
interpret it on read. These costs are wasted when values may not in
fact be null, as is the case in many datasets. In Avro such costs are
only paid when values may actually be null.
Also, allowing values to be null is a well-known source of errors. In
Avro, a value declared as non-null will always be non-null and programs
need not test for null values when processing it nor will they ever fail
for lack of such tests.
Tony Hoare calls his invention of null references his “Billion Dollar
Mistake”.
http://qconlondon.com/london-2009/presentation/Null+References:+The+Billion+Dollar+Mistake
Also note that in some programming languages not all values are permitted to be null. For example, in Java, values of type boolean, byte, short, char, int, float, long, and double may not be null.
How can I serialize directly to/from a byte array?
As pointed out in the specification, Avro data should always be stored with its schema. The Avro provided classes DataFileWriter, DataFileReader, and DataFileStream all ensure this by serializing the Schema in a container header. In some special cases, such as when implementing a new storage system or writing unit tests, you may need to write and read directly with the bare Avro serialized values. The following examples use code generated for Java from an Avro Schema, specifically the User example from the Quickstart guide.
Serializing to a byte array
This example takes a User object and returns a newly allocated byte array with the Avro serialization of that user.
ByteArrayOutputStream out = new ByteArrayOutputStream();
BinaryEncoder encoder = EncoderFactory.get().binaryEncoder(out, null);
DatumWriter<User> writer = new SpecificDatumWriter<User>(User.getClassSchema());
writer.write(user, encoder);
encoder.flush();
out.close();
byte[] serializedBytes = out.toByteArray();
Deserializing from a byte array
This example takes a byte array containing the Avro serialization of a user and returns a User object.SpecificDatumReader reader = new SpecificDatumReader(User.getClassSchema());
Decoder decoder = DecoderFactory.get().binaryDecoder(bytes, null);
User user = reader.read(null, decoder);
Note: If you are serializing or deserializing in a loop or as a method, you should be reusing objects, readers and/or writers. Check the JavaDoc to see how to reuse the objects.
6 - Apache Avro™ 1.9.2 Documentation
Introduction
Apache Avro™ is a data serialization system.
Avro provides:
- Rich data structures.
- A compact, fast, binary data format.
- A container file, to store persistent data.
- Remote procedure call (RPC).
- Simple integration with dynamic languages. Code generation is not required to read or write data files nor to use or implement RPC protocols. Code generation as an optional optimization, only worth implementing for statically typed languages.
Schemas
Avro relies on schemas. When Avro data is read, the schema used when writing it is always present. This permits each datum to be written with no per-value overheads, making serialization both fast and small. This also facilitates use with dynamic, scripting languages, since data, together with its schema, is fully self-describing.
When Avro data is stored in a file, its schema is stored with it, so that files may be processed later by any program. If the program reading the data expects a different schema this can be easily resolved, since both schemas are present.
When Avro is used in RPC, the client and server exchange schemas in the connection handshake. (This can be optimized so that, for most calls, no schemas are actually transmitted.) Since both client and server both have the other’s full schema, correspondence between same named fields, missing fields, extra fields, etc. can all be easily resolved.
Avro schemas are defined with JSON . This facilitates implementation in languages that already have JSON libraries.
Comparison with other systems
Avro provides functionality similar to systems such as Thrift, Protocol Buffers, etc. Avro differs from these systems in the following fundamental aspects.
- Dynamic typing: Avro does not require that code be generated. Data is always accompanied by a schema that permits full processing of that data without code generation, static datatypes, etc. This facilitates construction of generic data-processing systems and languages.
- Untagged data: Since the schema is present when data is read, considerably less type information need be encoded with data, resulting in smaller serialization size.
- No manually-assigned field IDs: When a schema changes, both the old and new schema are always present when processing data, so differences may be resolved symbolically, using field names.
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
6.1 - Getting Started (Java)
This is a short guide for getting started with Apache Avro™ using Java. This guide only covers using Avro for data serialization; see Patrick Hunt’s Avro RPC Quick Start for a good introduction to using Avro for RPC.
Download
Avro implementations for C, C++, C#, Java, PHP, Python, and Ruby can be downloaded from the Apache Avro™ Releases page. This guide uses Avro 1.9.2, the latest version at the time of writing. For the examples in this guide, download avro-1.9.2.jar and avro-tools-1.9.2.jar.
Alternatively, if you are using Maven, add the following dependency to your POM:
<dependency>
<groupId>org.apache.avro</groupId>
<artifactId>avro</artifactId>
<version>1.9.2</version>
</dependency>
As well as the Avro Maven plugin (for performing code generation):
<plugin>
<groupId>org.apache.avro</groupId>
<artifactId>avro-maven-plugin</artifactId>
<version>1.9.2</version>
<executions>
<execution>
<phase>generate-sources</phase>
<goals>
<goal>schema</goal>
</goals>
<configuration>
<sourceDirectory>${project.basedir}/src/main/avro/</sourceDirectory>
<outputDirectory>${project.basedir}/src/main/java/</outputDirectory>
</configuration>
</execution>
</executions>
</plugin>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-compiler-plugin</artifactId>
<configuration>
<source>1.8</source>
<target>1.8</target>
</configuration>
</plugin>
You may also build the required Avro jars from source. Building Avro is beyond the scope of this guide; see the Build Documentation page in the wiki for more information.
Defining a schema
Avro schemas are defined using JSON. Schemas are composed of primitive types (null, boolean, int, long, float, double, bytes, and string) and complex types (record, enum, array, map, union, and fixed). You can learn more about Avro schemas and types from the specification, but for now let’s start with a simple schema example, user.avsc:
{"namespace": "example.avro",
"type": "record",
"name": "User",
"fields": [
{"name": "name", "type": "string"},
{"name": "favorite_number", "type": ["int", "null"]},
{"name": "favorite_color", "type": ["string", "null"]}
]
}
This schema defines a record representing a hypothetical user. (Note that a schema file can only contain a single schema definition.) At minimum, a record definition must include its type (“type”: “record”), a name (“name”: “User”), and fields, in this case name, favorite_number, and favorite_color. We also define a namespace (“namespace”: “example.avro”), which together with the name attribute defines the “full name” of the schema (example.avro.User in this case).
Fields are defined via an array of objects, each of which defines a name and type (other attributes are optional, see the record specification for more details). The type attribute of a field is another schema object, which can be either a primitive or complex type. For example, the name field of our User schema is the primitive type string, whereas the favorite_number and favorite_color fields are both unions, represented by JSON arrays. unions are a complex type that can be any of the types listed in the array; e.g., favorite_number can either be an int or null, essentially making it an optional field.
Serializing and deserializing with code generation
Compiling the schema
Code generation allows us to automatically create classes based on our previously-defined schema. Once we have defined the relevant classes, there is no need to use the schema directly in our programs. We use the avro-tools jar to generate code as follows:
java -jar /path/to/avro-tools-1.9.2.jar compile schema <schema file> <destination>
This will generate the appropriate source files in a package based on the schema’s namespace in the provided destination folder. For instance, to generate a User class in package example.avro from the schema defined above, run
java -jar /path/to/avro-tools-1.9.2.jar compile schema user.avsc .
Note that if you using the Avro Maven plugin, there is no need to manually invoke the schema compiler; the plugin automatically performs code generation on any .avsc files present in the configured source directory.
Creating Users
Now that we’ve completed the code generation, let’s create some Users, serialize them to a data file on disk, and then read back the file and deserialize the User objects.
First let’s create some Users and set their fields.
User user1 = new User();
user1.setName("Alyssa");
user1.setFavoriteNumber(256);
// Leave favorite color null
// Alternate constructor
User user2 = new User("Ben", 7, "red");
// Construct via builder
User user3 = User.newBuilder()
.setName("Charlie")
.setFavoriteColor("blue")
.setFavoriteNumber(null)
.build();
As shown in this example, Avro objects can be created either by invoking a constructor directly or by using a builder. Unlike constructors, builders will automatically set any default values specified in the schema. Additionally, builders validate the data as it set, whereas objects constructed directly will not cause an error until the object is serialized. However, using constructors directly generally offers better performance, as builders create a copy of the datastructure before it is written.
Note that we do not set user1’s favorite color. Since that record is of type [“string”, “null”], we can either set it to a string or leave it null; it is essentially optional. Similarly, we set user3’s favorite number to null (using a builder requires setting all fields, even if they are null).
Serializing
Now let’s serialize our Users to disk.
/// Serialize user1, user2 and user3 to disk
DatumWriter<User> userDatumWriter = new SpecificDatumWriter<User>(User.class);
DataFileWriter<User> dataFileWriter = new DataFileWriter<User>(userDatumWriter);
dataFileWriter.create(user1.getSchema(), new File("users.avro"));
dataFileWriter.append(user1);
dataFileWriter.append(user2);
dataFileWriter.append(user3);
dataFileWriter.close();
We create a DatumWriter, which converts Java objects into an in-memory serialized format. The SpecificDatumWriter class is used with generated classes and extracts the schema from the specified generated type.
Next we create a DataFileWriter, which writes the serialized records, as well as the schema, to the file specified in the dataFileWriter.create call. We write our users to the file via calls to the dataFileWriter.append method. When we are done writing, we close the data file.
Deserializing
Finally, let’s deserialize the data file we just created.
// Deserialize Users from disk
DatumReader<User> userDatumReader = new SpecificDatumReader<User>(User.class);
DataFileReader<User> dataFileReader = new DataFileReader<User>(file, userDatumReader);
User user = null;
while (dataFileReader.hasNext()) {
// Reuse user object by passing it to next(). This saves us from
// allocating and garbage collecting many objects for files with
// many items.
user = dataFileReader.next(user);
System.out.println(user);
}
This snippet will output:
{"name": "Alyssa", "favorite_number": 256, "favorite_color": null}
{"name": "Ben", "favorite_number": 7, "favorite_color": "red"}
{"name": "Charlie", "favorite_number": null, "favorite_color": "blue"}
Deserializing is very similar to serializing. We create a SpecificDatumReader, analogous to the SpecificDatumWriter we used in serialization, which converts in-memory serialized items into instances of our generated class, in this case User. We pass the DatumReader and the previously created File to a DataFileReader, analogous to the DataFileWriter, which reads both the schema used by the writer as well as the data from the file on disk. The data will be read using the writer’s schema included in the file and the schema provided by the reader, in this case the User class. The writer’s schema is needed to know the order in which fields were written, while the reader’s schema is needed to know what fields are expected and how to fill in default values for fields added since the file was written. If there are differences between the two schemas, they are resolved according to the Schema Resolution specification.
Next we use the DataFileReader to iterate through the serialized Users and print the deserialized object to stdout. Note how we perform the iteration: we create a single User object which we store the current deserialized user in, and pass this record object to every call of dataFileReader.next. This is a performance optimization that allows the DataFileReader to reuse the same User object rather than allocating a new User for every iteration, which can be very expensive in terms of object allocation and garbage collection if we deserialize a large data file. While this technique is the standard way to iterate through a data file, it’s also possible to use for (User user : dataFileReader) if performance is not a concern.
Compiling and running the example code
This example code is included as a Maven project in the examples/java-example directory in the Avro docs. From this directory, execute the following commands to build and run the example:
$ mvn compile # includes code generation via Avro Maven plugin
$ mvn -q exec:java -Dexec.mainClass=example.SpecificMain
Beta feature: Generating faster code
In this release we have introduced a new approach to generating code that speeds up decoding of objects by more than 10% and encoding by more than 30% (future performance enhancements are underway). To ensure a smooth introduction of this change into production systems, this feature is controlled by a feature flag, the system property org.apache.avro.specific.use_custom_coders. In this first release, this feature is off by default. To turn it on, set the system flag to true at runtime. In the sample above, for example, you could enable the fater coders as follows:
$ mvn -q exec:java -Dexec.mainClass=example.SpecificMain \
-Dorg.apache.avro.specific.use_custom_coders=true
Note that you do not have to recompile your Avro schema to have access to this feature. The feature is compiled and built into your code, and you turn it on and off at runtime using the feature flag. As a result, you can turn it on during testing, for example, and then off in production. Or you can turn it on in production, and quickly turn it off if something breaks.
We encourage the Avro community to exercise this new feature early to help build confidence. (For those paying one-demand for compute resources in the cloud, it can lead to meaningful cost savings.) As confidence builds, we will turn this feature on by default, and eventually eliminate the feature flag (and the old code).
Serializing and deserializing without code generation
Data in Avro is always stored with its corresponding schema, meaning we can always read a serialized item regardless of whether we know the schema ahead of time. This allows us to perform serialization and deserialization without code generation.
Let’s go over the same example as in the previous section, but without using code generation: we’ll create some users, serialize them to a data file on disk, and then read back the file and deserialize the users objects.
Creating users
First, we use a Parser to read our schema definition and create a Schema object.
Schema schema = new Schema.Parser().parse(new File("user.avsc"));
Using this schema, let’s create some users.
GenericRecord user1 = new GenericData.Record(schema);
user1.put("name", "Alyssa");
user1.put("favorite_number", 256);
// Leave favorite color null
GenericRecord user2 = new GenericData.Record(schema);
user2.put("name", "Ben");
user2.put("favorite_number", 7);
user2.put("favorite_color", "red");
Since we’re not using code generation, we use GenericRecords to represent users. GenericRecord uses the schema to verify that we only specify valid fields. If we try to set a non-existent field (e.g., user1.put(“favorite_animal”, “cat”)), we’ll get an AvroRuntimeException when we run the program.
Note that we do not set user1’s favorite color. Since that record is of type [“string”, “null”], we can either set it to a string or leave it null; it is essentially optional.
Serializing
Now that we’ve created our user objects, serializing and deserializing them is almost identical to the example above which uses code generation. The main difference is that we use generic instead of specific readers and writers.
First we’ll serialize our users to a data file on disk.
// Serialize user1 and user2 to disk
File file = new File("users.avro");
DatumWriter<GenericRecord> datumWriter = new GenericDatumWriter<GenericRecord>(schema);
DataFileWriter<GenericRecord> dataFileWriter = new DataFileWriter<GenericRecord>(datumWriter);
dataFileWriter.create(schema, file);
dataFileWriter.append(user1);
dataFileWriter.append(user2);
dataFileWriter.close();
We create a DatumWriter, which converts Java objects into an in-memory serialized format. Since we are not using code generation, we create a GenericDatumWriter. It requires the schema both to determine how to write the GenericRecords and to verify that all non-nullable fields are present.
As in the code generation example, we also create a DataFileWriter, which writes the serialized records, as well as the schema, to the file specified in the dataFileWriter.create call. We write our users to the file via calls to the dataFileWriter.append method. When we are done writing, we close the data file.
Deserializing
Finally, we’ll deserialize the data file we just created.
// Deserialize users from disk
DatumReader<GenericRecord> datumReader = new GenericDatumReader<GenericRecord>(schema);
DataFileReader<GenericRecord> dataFileReader = new DataFileReader<GenericRecord>(file, datumReader);
GenericRecord user = null;
while (dataFileReader.hasNext()) {
// Reuse user object by passing it to next(). This saves us from
// allocating and garbage collecting many objects for files with
// many items.
user = dataFileReader.next(user);
System.out.println(user);
This outputs:
{"name": "Alyssa", "favorite_number": 256, "favorite_color": null}
{"name": "Ben", "favorite_number": 7, "favorite_color": "red"}
Deserializing is very similar to serializing. We create a GenericDatumReader, analogous to the GenericDatumWriter we used in serialization, which converts in-memory serialized items into GenericRecords. We pass the DatumReader and the previously created File to a DataFileReader, analogous to the DataFileWriter, which reads both the schema used by the writer as well as the data from the file on disk. The data will be read using the writer’s schema included in the file, and the reader’s schema provided to the GenericDatumReader. The writer’s schema is needed to know the order in which fields were written, while the reader’s schema is needed to know what fields are expected and how to fill in default values for fields added since the file was written. If there are differences between the two schemas, they are resolved according to the Schema Resolution specification.
Next, we use the DataFileReader to iterate through the serialized users and print the deserialized object to stdout. Note how we perform the iteration: we create a single GenericRecord object which we store the current deserialized user in, and pass this record object to every call of dataFileReader.next. This is a performance optimization that allows the DataFileReader to reuse the same record object rather than allocating a new GenericRecord for every iteration, which can be very expensive in terms of object allocation and garbage collection if we deserialize a large data file. While this technique is the standard way to iterate through a data file, it’s also possible to use for (GenericRecord user : dataFileReader) if performance is not a concern.
Compiling and running the example code
This example code is included as a Maven project in the examples/java-example directory in the Avro docs. From this directory, execute the following commands to build and run the example:
$ mvn compile
$ mvn -q exec:java -Dexec.mainClass=example.GenericMain
6.2 - Getting Started (Python)
This is a short guide for getting started with Apache Avro™ using Python. This guide only covers using Avro for data serialization; see Patrick Hunt’s Avro RPC Quick Start for a good introduction to using Avro for RPC.
Download
For Python, the easiest way to get started is to install it from PyPI. Python’s Avro API is available over PyPi.
$ python3 -m pip install avro
Avro implementations for C, C++, C#, Java, PHP, Python, and Ruby can be downloaded from the Apache Avro™ Releases page. This guide uses Avro 1.9.2, the latest version at the time of writing. Download and unzip avro-1.9.2.tar.gz, and install via python setup.py (this will probably require root privileges). Ensure that you can import avro from a Python prompt.
$ tar xvf avro-1.9.2.tar.gz
$ cd avro-1.9.2
$ sudo python setup.py install
$ python
>>> import avro # should not raise ImportError
Alternatively, you may build the Avro Python library from source. From your the root Avro directory, run the commands
$ cd lang/py/
$ ant
$ sudo python setup.py install
$ python
>>> import avro # should not raise ImportError
Defining a schema
Avro schemas are defined using JSON. Schemas are composed of primitive types (null, boolean, int, long, float, double, bytes, and string) and complex types (record, enum, array, map, union, and fixed). You can learn more about Avro schemas and types from the specification, but for now let’s start with a simple schema example, user.avsc:
{"namespace": "example.avro",
"type": "record",
"name": "User",
"fields": [
{"name": "name", "type": "string"},
{"name": "favorite_number", "type": ["int", "null"]},
{"name": "favorite_color", "type": ["string", "null"]}
]
}
This schema defines a record representing a hypothetical user. (Note that a schema file can only contain a single schema definition.) At minimum, a record definition must include its type (“type”: “record”), a name (“name”: “User”), and fields, in this case name, favorite_number, and favorite_color. We also define a namespace (“namespace”: “example.avro”), which together with the name attribute defines the “full name” of the schema (example.avro.User in this case).
Fields are defined via an array of objects, each of which defines a name and type (other attributes are optional, see the record specification for more details). The type attribute of a field is another schema object, which can be either a primitive or complex type. For example, the name field of our User schema is the primitive type string, whereas the favorite_number and favorite_color fields are both unions, represented by JSON arrays. unions are a complex type that can be any of the types listed in the array; e.g., favorite_number can either be an int or null, essentially making it an optional field.
Serializing and deserializing without code generation
Data in Avro is always stored with its corresponding schema, meaning we can always read a serialized item, regardless of whether we know the schema ahead of time. This allows us to perform serialization and deserialization without code generation. Note that the Avro Python library does not support code generation.
Try running the following code snippet, which serializes two users to a data file on disk, and then reads back and deserializes the data file:
import avro.schema
from avro.datafile import DataFileReader, DataFileWriter
from avro.io import DatumReader, DatumWriter
schema = avro.schema.parse(open("user.avsc", "rb").read())
writer = DataFileWriter(open("users.avro", "wb"), DatumWriter(), schema)
writer.append({"name": "Alyssa", "favorite_number": 256})
writer.append({"name": "Ben", "favorite_number": 7, "favorite_color": "red"})
writer.close()
reader = DataFileReader(open("users.avro", "rb"), DatumReader())
for user in reader:
print user
reader.close()
This outputs:
{u'favorite_color': None, u'favorite_number': 256, u'name': u'Alyssa'}
{u'favorite_color': u'red', u'favorite_number': 7, u'name': u'Ben'}
Do make sure that you open your files in binary mode (i.e. using the modes wb or rb respectively). Otherwise you might generate corrupt files due to automatic replacement of newline characters with the platform-specific representations.
Let’s take a closer look at what’s going on here.
schema = avro.schema.parse(open("user.avsc", "rb").read())
avro.schema.parse takes a string containing a JSON schema definition as input and outputs a avro.schema.Schema object (specifically a subclass of Schema, in this case RecordSchema). We’re passing in the contents of our user.avsc schema file here.
writer = DataFileWriter(open("users.avro", "wb"), DatumWriter(), schema)
We create a DataFileWriter, which we’ll use to write serialized items to a data file on disk. The DataFileWriter constructor takes three arguments:
- The file we’ll serialize to
- A DatumWriter, which is responsible for actually serializing the items to Avro’s binary format (DatumWriters can be used separately from DataFileWriters, e.g., to perform IPC with Avro).
- The schema we’re using. The DataFileWriter needs the schema both to write the schema to the data file, and to verify that the items we write are valid items and write the appropriate fields.
writer.append({"name": "Alyssa", "favorite_number": 256})
writer.append({"name": "Ben", "favorite_number": 7, "favorite_color": "red"})
We use DataFileWriter.append to add items to our data file. Avro records are represented as Python dicts. Since the field favorite_color has type [“int”, “null”], we are not required to specify this field, as shown in the first append. Were we to omit the required name field, an exception would be raised. Any extra entries not corresponding to a field are present in the dict are ignored.
reader = DataFileReader(open("users.avro", "rb"), DatumReader())
We open the file again, this time for reading back from disk. We use a DataFileReader and DatumReader analagous to the DataFileWriter and DatumWriter above.
for user in reader:
print user
The DataFileReader is an iterator that returns dicts corresponding to the serialized items.
6.3 - Specification
Introduction
This document defines Apache Avro. It is intended to be the authoritative specification. Implementations of Avro must adhere to this document.
Schema Declaration
A Schema is represented in JSON by one of:
- A JSON string, naming a defined type.
- A JSON object, of the form:
{"type": "typeName" ...attributes...}
where typeName is either a primitive or derived type name, as defined below. Attributes not defined in this document are permitted as metadata, but must not affect the format of serialized data.
- A JSON array, representing a union of embedded types.
Primitive Types
The set of primitive type names is:
- null: no value
- boolean: a binary value
- int: 32-bit signed integer
- long: 64-bit signed integer
- float: single precision (32-bit) IEEE 754 floating-point number
- double: double precision (64-bit) IEEE 754 floating-point number
- bytes: sequence of 8-bit unsigned bytes
- string: unicode character sequence
Primitive types have no specified attributes.
Primitive type names are also defined type names. Thus, for example, the schema “string” is equivalent to:
Complex Types
Avro supports six kinds of complex types: records, enums, arrays, maps, unions and fixed.
Records
Records use the type name “record” and support three attributes:
- name: a JSON string providing the name of the record (required).
- namespace, a JSON string that qualifies the name;
- doc: a JSON string providing documentation to the user of this schema (optional).
- aliases: a JSON array of strings, providing alternate names for this record (optional).
- fields: a JSON array, listing fields (required). Each field is a JSON object with the following attributes:
- name: a JSON string providing the name of the field (required), and
- doc: a JSON string describing this field for users (optional).
- type: a schema, as defined above
- _default: A default value for this field, used when reading instances that lack this field (optional). 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. Default values for bytes and fixed fields are JSON strings, where Unicode code points 0-255 are mapped to unsigned 8-bit byte values 0-255.
field default values
| avro type |
json type |
example |
| null |
null |
null |
| boolean |
boolean |
true |
| int,long |
integer |
1 |
| float,double |
number |
1.1 |
| bytes |
string |
"\u00FF" |
| string |
string |
"foo" |
| record |
object |
{"a": 1} |
| enum |
string |
"FOO" |
| array |
array |
[1] |
| map |
object |
{"a": 1} |
| fixed |
string |
"\u00ff" |
- order: specifies how this field impacts sort ordering of this record (optional). Valid values are “ascending” (the default), “descending”, or “ignore”. For more details on how this is used, see the sort order section below.
- aliases: a JSON array of strings, providing alternate names for this field (optional).
For example, a linked-list of 64-bit values may be defined with:
{
"type": "record",
"name": "LongList",
"aliases": ["LinkedLongs"], // old name for this
"fields" : [
{"name": "value", "type": "long"}, // each element has a long
{"name": "next", "type": ["null", "LongList"]} // optional next element
]
}
Enums
Enums use the type name “enum” and support the following attributes:
- name: a JSON string providing the name of the enum (required).
- namespace, a JSON string that qualifies the name ;
- aliases: a JSON array of strings, providing alternate names for this enum (optional).
- doc: a JSON string providing documentation to the user of this schema (optional).
- symbols: a JSON array, listing symbols, as JSON strings (required). All symbols in an enum must be unique; duplicates are prohibited. Every symbol must match the regular expression [A-Za-z_][A-Za-z0-9_]* (the same requirement as for names).
- default: A default value for this enumeration, used during resolution when the reader encounters a symbol from the writer that isn’t defined in the reader’s schema (optional). The value provided here must be a JSON string that’s a member of the symbols array. See documentation on schema resolution for how this gets used.
For example, playing card suits might be defined with:
{
"type": "enum",
"name": "Suit",
"symbols" : ["SPADES", "HEARTS", "DIAMONDS", "CLUBS"]
}
Arrays
Arrays use the type name “array” and support a single attribute:
- items: the schema of the array’s items.
For example, an array of strings is declared with:
{
"type": "array",
"items" : "string"
}
Maps
Maps use the type name “map” and support one attribute:
- values: the schema of the map’s values.
Map keys are assumed to be strings.
For example, a map from string to long is declared with:
{
"type": "map",
"values" : "long"
}
Unions
Unions, as mentioned above, are represented using JSON arrays. For example, ["null", "string"] declares a schema which may be either a null or string.
(Note that when a default value is specified for a record field whose type is a union, the type of the default value must match the first element of the union. Thus, for unions containing “null”, the “null” is usually listed first, since the default value of such unions is typically null.)
Unions may not contain more than one schema with the same type, except for the named types record, fixed and enum. For example, unions containing two array types or two map types are not permitted, but two types with different names are permitted. (Names permit efficient resolution when reading and writing unions.)
Unions may not immediately contain other unions.
Fixed
Fixed uses the type name “fixed” and supports two attributes:
- name: a string naming this fixed (required).
- namespace, a string that qualifies the name;
- aliases: a JSON array of strings, providing alternate names for this enum (optional).
- size: an integer, specifying the number of bytes per value (required).
For example, 16-byte quantity may be declared with:
{"type": "fixed", "size": 16, "name": "md5"}
Names
Record, enums and fixed are named types. Each has a fullname that is composed of two parts; a name and a namespace. Equality of names is defined on the fullname.
The name portion of a fullname, record field names, and enum symbols must:
- start with [A-Za-z_]
- subsequently contain only [A-Za-z0-9_]
A namespace is a dot-separated sequence of such names. The empty string may also be used as a namespace to indicate the null namespace. Equality of names (including field names and enum symbols) as well as fullnames is case-sensitive.
In record, enum and fixed definitions, the fullname is determined in one of the following ways:
A name and namespace are both specified. For example, one might use “name”: “X”, “namespace”: “org.foo” to indicate the fullname org.foo.X.
-
A fullname is specified. If the name specified contains a dot, then it is assumed to be a fullname, and any namespace also specified is ignored. For example, use “name”: “org.foo.X” to indicate the fullname org.foo.X.
-
A name only is specified, i.e., a name that contains no dots. In this case the namespace is taken from the most tightly enclosing schema or protocol. For example, if “name”: “X” is specified, and this occurs within a field of the record definition of org.foo.Y, then the fullname is org.foo.X. If there is no enclosing namespace then the null namespace is used.
References to previously defined names are as in the latter two cases above: if they contain a dot they are a fullname, if they do not contain a dot, the namespace is the namespace of the enclosing definition.
Primitive type names have no namespace and their names may not be defined in any namespace.
A schema or protocol may not contain multiple definitions of a fullname. Further, a name must be defined before it is used (“before” in the depth-first, left-to-right traversal of the JSON parse tree, where the types attribute of a protocol is always deemed to come “before” the messages attribute.)
Aliases
Named types and fields may have aliases. An implementation may optionally use aliases to map a writer’s schema to the reader’s. This facilitates both schema evolution as well as processing disparate datasets.
Aliases function by re-writing the writer’s schema using aliases from the reader’s schema. For example, if the writer’s schema was named “Foo” and the reader’s schema is named “Bar” and has an alias of “Foo”, then the implementation would act as though “Foo” were named “Bar” when reading. Similarly, if data was written as a record with a field named “x” and is read as a record with a field named “y” with alias “x”, then the implementation would act as though “x” were named “y” when reading.
A type alias may be specified either as a fully namespace-qualified, or relative to the namespace of the name it is an alias for. For example, if a type named “a.b” has aliases of “c” and “x.y”, then the fully qualified names of its aliases are “a.c” and “x.y”.
Data Serialization and Deserialization
Binary encoded Avro data does not include type information or field names. The benefit is that the serialized data is small, but as a result a schema must always be used in order to read Avro data correctly. The best way to ensure that the schema is structurally identical to the one used to write the data is to use the exact same schema.
Therefore, files or systems that store Avro data should always include the writer’s schema for that data. Avro-based remote procedure call (RPC) systems must also guarantee that remote recipients of data have a copy of the schema used to write that data. In general, it is advisable that any reader of Avro data should use a schema that is the same (as defined more fully in Parsing Canonical Form for Schemas) as the schema that was used to write the data in order to deserialize it correctly. Deserializing data into a newer schema is accomplished by specifying an additional schema, the results of which are described in Schema Resolution.
In general, both serialization and deserialization proceed as a depth-first, left-to-right traversal of the schema, serializing or deserializing primitive types as they are encountered. Therefore, it is possible, though not advisable, to read Avro data with a schema that does not have the same Parsing Canonical Form as the schema with which the data was written. In order for this to work, the serialized primitive values must be compatible, in order value by value, with the items in the deserialization schema. For example, int and long are always serialized the same way, so an int could be deserialized as a long. Since the compatibility of two schemas depends on both the data and the serialization format (eg. binary is more permissive than JSON because JSON includes field names, eg. a long that is too large will overflow an int), it is simpler and more reliable to use schemas with identical Parsing Canonical Form.
Encodings
Avro specifies two serialization encodings: binary and JSON. Most applications will use the binary encoding, as it is smaller and faster. But, for debugging and web-based applications, the JSON encoding may sometimes be appropriate.
Binary Encoding
Binary encoding does not include field names, self-contained information about the types of individual bytes, nor field or record separators. Therefore readers are wholly reliant on the schema used when the data was encoded.
Primitive Types
Primitive types are encoded in binary as follows:
- null is written as zero bytes.
- a boolean is written as a single byte whose value is either 0 (false) or 1 (true).
- int and long values are written using variable-length zig-zag coding. Some examples:
| value |
hex |
| 0 |
00 |
| -1 |
01 |
| 1 |
02 |
| -2 |
03 |
| 2 |
04 |
| … |
|
| -64 |
7f |
| 64 |
80 01 |
| … |
|
- a float is written as 4 bytes. The float is converted into a 32-bit integer using a method equivalent to Java’s floatToIntBits and then encoded in little-endian format.
- a double is written as 8 bytes. The double is converted into a 64-bit integer using a method equivalent to Java’s doubleToLongBits and then encoded in little-endian format.
- bytes are encoded as a long followed by that many bytes of data.
- a string is encoded as a long followed by that many bytes of UTF-8 encoded character data.
For example, the three-character string “foo” would be encoded as the long value 3 (encoded as hex 06) followed by the UTF-8 encoding of ‘f’, ‘o’, and ‘o’ (the hex bytes 66 6f 6f):
06 66 6f 6f
Complex Types
Complex types are encoded in binary as follows:
Records
A record is encoded by encoding the values of its fields in the order that they are declared. In other words, a record is encoded as just the concatenation of the encodings of its fields. Field values are encoded per their schema.
For example, the record schema
{
"type": "record",
"name": "test",
"fields" : [
{"name": "a", "type": "long"},
{"name": "b", "type": "string"}
]
}
An instance of this record whose a field has value 27 (encoded as hex 36) and whose b field has value “foo” (encoded as hex bytes 06 66 6f 6f), would be encoded simply as the concatenation of these, namely the hex byte sequence:
36 06 66 6f 6f
Enums
An enum is encoded by a int, representing the zero-based position of the symbol in the schema.
For example, consider the enum:
{"type": "enum", "name": "Foo", "symbols": ["A", "B", "C", "D"] }
This would be encoded by an int between zero and three, with zero indicating “A”, and 3 indicating “D”.
Arrays
Arrays are encoded as a series of blocks. Each block consists of a long count value, followed by that many array items. A block with count zero indicates the end of the array. Each item is encoded per the array’s item schema.
If a block’s count is negative, its absolute value is used, and the count is followed immediately by a long block size indicating the number of bytes in the block. This block size permits fast skipping through data, e.g., when projecting a record to a subset of its fields.
For example, the array schema
{"type": "array", "items": "long"}
an array containing the items 3 and 27 could be encoded as the long value 2 (encoded as hex 04) followed by long values 3 and 27 (encoded as hex 06 36) terminated by zero:
04 06 36 00
The blocked representation permits one to read and write arrays larger than can be buffered in memory, since one can start writing items without knowing the full length of the array.
Maps
Maps are encoded as a series of blocks. Each block consists of a long count value, followed by that many key/value pairs. A block with count zero indicates the end of the map. Each item is encoded per the map’s value schema.
If a block’s count is negative, its absolute value is used, and the count is followed immediately by a long block size indicating the number of bytes in the block. This block size permits fast skipping through data, e.g., when projecting a record to a subset of its fields.
The blocked representation permits one to read and write maps larger than can be buffered in memory, since one can start writing items without knowing the full length of the map.
Unions
A union is encoded by first writing a long value indicating the zero-based position within the union of the schema of its value. The value is then encoded per the indicated schema within the union.
For example, the union schema ["null","string"] would encode:
-
null as zero (the index of “null” in the union):
00
-
the string “a” as one (the index of “string” in the union, 1, encoded as hex 02), followed by the serialized string:
02 02 61
NOTE: Currently for C/C++ implementtions, the positions are practically an int, but theoretically a long. In reality, we don’t expect unions with 215M members
Fixed
Fixed instances are encoded using the number of bytes declared in the schema.
JSON Encoding
Except for unions, the JSON encoding is the same as is used to encode field default values.
The value of a union is encoded in JSON as follows:
- if its type is null, then it is encoded as a JSON null;
- otherwise it is encoded as a JSON object with one name/value pair whose name is the type’s name and whose value is the recursively encoded value. For Avro’s named types (record, fixed or enum) the user-specified name is used, for other types the type name is used.
For example, the union schema ["null","string","Foo"], where Foo is a record name, would encode:
- null as null;
- the string “a” as
{"string": "a"} and
- a Foo instance as
{"Foo": {...}}, where {...} indicates the JSON encoding of a Foo instance.
Note that the original schema is still required to correctly process JSON-encoded data. For example, the JSON encoding does not distinguish between int and long, float and double, records and maps, enums and strings, etc.
Single-object encoding
In some situations a single Avro serialized object is to be stored for a longer period of time. One very common example is storing Avro records for several weeks in an Apache Kafka topic.
In the period after a schema change this persistence system will contain records that have been written with different schemas. So the need arises to know which schema was used to write a record to support schema evolution correctly. In most cases the schema itself is too large to include in the message, so this binary wrapper format supports the use case more effectively.
Single object encoding specification
Single Avro objects are encoded as follows:
- A two-byte marker,
C3 01, to show that the message is Avro and uses this single-record format (version 1).
- The 8-byte little-endian CRC-64-AVRO fingerprint of the object’s schema.
- The Avro object encoded using Avro’s binary encoding.
Implementations use the 2-byte marker to determine whether a payload is Avro. This check helps avoid expensive lookups that resolve the schema from a fingerprint, when the message is not an encoded Avro payload.
Sort Order
Avro defines a standard sort order for data. This permits data written by one system to be efficiently sorted by another system. This can be an important optimization, as sort order comparisons are sometimes the most frequent per-object operation. Note also that Avro binary-encoded data can be efficiently ordered without deserializing it to objects.
Data items may only be compared if they have identical schemas. Pairwise comparisons are implemented recursively with a depth-first, left-to-right traversal of the schema. The first mismatch encountered determines the order of the items.
Two items with the same schema are compared according to the following rules.
- null data is always equal.
- boolean data is ordered with false before true.
- int, long, float and double data is ordered by ascending numeric value.
- bytes and fixed data are compared lexicographically by unsigned 8-bit values.
- string data is compared lexicographically by Unicode code point. Note that since UTF-8 is used as the binary encoding for strings, sorting of bytes and string binary data is identical.
- array data is compared lexicographically by element.
- enum data is ordered by the symbol’s position in the enum schema. For example, an enum whose symbols are
["z", "a"] would sort “z” values before “a” values.
- union data is first ordered by the branch within the union, and, within that, by the type of the branch. For example, an
["int", "string"] union would order all int values before all string values, with the ints and strings themselves ordered as defined above.
- record data is ordered lexicographically by field. If a field specifies that its order is:
- “ascending”, then the order of its values is unaltered.
- “descending”, then the order of its values is reversed.
- “ignore”, then its values are ignored when sorting.
- map data may not be compared. It is an error to attempt to compare data containing maps unless those maps are in an
"order":"ignore" record field.
Object Container Files
Avro includes a simple object container file format. A file has a schema, and all objects stored in the file must be written according to that schema, using binary encoding. Objects are stored in blocks that may be compressed. Syncronization markers are used between blocks to permit efficient splitting of files for MapReduce processing.
Files may include arbitrary user-specified metadata.
A file consists of:
- A file header, followed by
- one or more file data blocks.
A file header consists of:
- Four bytes, ASCII ‘O’, ‘b’, ‘j’, followed by 1.
- file metadata, including the schema.
- The 16-byte, randomly-generated sync marker for this file.
File metadata is written as if defined by the following map schema:
{"type": "map", "values": "bytes"}
All metadata properties that start with “avro.” are reserved. The following file metadata properties are currently used:
- avro.schema contains the schema of objects stored in the file, as JSON data (required).
- avro.codec the name of the compression codec used to compress blocks, as a string. Implementations are required to support the following codecs: “null” and “deflate”. If codec is absent, it is assumed to be “null”. The codecs are described with more detail below.
A file header is thus described by the following schema:
{"type": "record", "name": "org.apache.avro.file.Header",
"fields" : [
{"name": "magic", "type": {"type": "fixed", "name": "Magic", "size": 4}},
{"name": "meta", "type": {"type": "map", "values": "bytes"}},
{"name": "sync", "type": {"type": "fixed", "name": "Sync", "size": 16}}
]
}
A file data block consists of:
- A long indicating the count of objects in this block.
- A long indicating the size in bytes of the serialized objects in the current block, after any codec is applied
- The serialized objects. If a codec is specified, this is compressed by that codec.
- The file’s 16-byte sync marker.
Thus, each block’s binary data can be efficiently extracted or skipped without deserializing the contents. The combination of block size, object counts, and sync markers enable detection of corrupt blocks and help ensure data integrity.
Required Codecs
null
The “null” codec simply passes through data uncompressed.
deflate
The “deflate” codec writes the data block using the deflate algorithm as specified in RFC 1951, and typically implemented using the zlib library. Note that this format (unlike the “zlib format” in RFC 1950) does not have a checksum.
Optional Codecs
bzip2
The “bzip2” codec uses the bzip2 compression library.
snappy
The “snappy” codec uses Google’s Snappy compression library. Each compressed block is followed by the 4-byte, big-endian CRC32 checksum of the uncompressed data in the block.
Protocol Declaration
Avro protocols describe RPC interfaces. Like schemas, they are defined with JSON text.
A protocol is a JSON object with the following attributes:
- protocol, a string, the name of the protocol (required);
- namespace, an optional string that qualifies the name;
- doc, an optional string describing this protocol;
- types, an optional list of definitions of named types (records, enums, fixed and errors). An error definition is just like a record definition except it uses “error” instead of “record”. Note that forward references to named types are not permitted.
- messages, an optional JSON object whose keys are message names and whose values are objects whose attributes are described below. No two messages may have the same name.
The name and namespace qualification rules defined for schema objects apply to protocols as well.
Messages
A message has attributes:
- a doc, an optional description of the message,
- a request, a list of named, typed parameter schemas (this has the same form as the fields of a record declaration);
- a response schema;
- an optional union of declared error schemas. The effective union has “string” prepended to the declared union, to permit transmission of undeclared “system” errors. For example, if the declared error union is
["AccessError"], then the effective union is ["string", "AccessError"]. When no errors are declared, the effective error union is ["string"]. Errors are serialized using the effective union; however, a protocol’s JSON declaration contains only the declared union.
- an optional one-way boolean parameter.
A request parameter list is processed equivalently to an anonymous record. Since record field lists may vary between reader and writer, request parameters may also differ between the caller and responder, and such differences are resolved in the same manner as record field differences.
The one-way parameter may only be true when the response type is "null" and no errors are listed.
Sample Protocol
For example, one may define a simple HelloWorld protocol with:
{
"namespace": "com.acme",
"protocol": "HelloWorld",
"doc": "Protocol Greetings",
"types": [
{"name": "Greeting", "type": "record", "fields": [
{"name": "message", "type": "string"}]},
{"name": "Curse", "type": "error", "fields": [
{"name": "message", "type": "string"}]}
],
"messages": {
"hello": {
"doc": "Say hello.",
"request": [{"name": "greeting", "type": "Greeting" }],
"response": "Greeting",
"errors": ["Curse"]
}
}
}
Message Transport
Messages may be transmitted via different transport mechanisms.
To the transport, a message is an opaque byte sequence.
A transport is a system that supports:
- transmission of request messages
- receipt of corresponding response messages
Servers may send a response message back to the client corresponding to a request message. The mechanism of correspondance is transport-specific. For example, in HTTP it is implicit, since HTTP directly supports requests and responses. But a transport that multiplexes many client threads over a single socket would need to tag messages with unique identifiers.
Transports may be either stateless or stateful. In a stateless transport, messaging assumes no established connection state, while stateful transports establish connections that may be used for multiple messages. This distinction is discussed further in the handshake section below.
HTTP as Transport
When HTTP is used as a transport, each Avro message exchange is an HTTP request/response pair. All messages of an Avro protocol should share a single URL at an HTTP server. Other protocols may also use that URL. Both normal and error Avro response messages should use the 200 (OK) response code. The chunked encoding may be used for requests and responses, but, regardless the Avro request and response are the entire content of an HTTP request and response. The HTTP Content-Type of requests and responses should be specified as “avro/binary”. Requests should be made using the POST method.
HTTP is used by Avro as a stateless transport.
Message Framing
Avro messages are framed as a list of buffers.
Framing is a layer between messages and the transport. It exists to optimize certain operations.
The format of framed message data is:
- a series of buffers, where each buffer consists of:
- a four-byte, big-endian buffer length, followed by
- that many bytes of buffer data.
- A message is always terminated by a zero-length buffer.
Framing is transparent to request and response message formats (described below). Any message may be presented as a single or multiple buffers.
Framing can permit readers to more efficiently get different buffers from different sources and for writers to more efficiently store different buffers to different destinations. In particular, it can reduce the number of times large binary objects are copied. For example, if an RPC parameter consists of a megabyte of file data, that data can be copied directly to a socket from a file descriptor, and, on the other end, it could be written directly to a file descriptor, never entering user space.
A simple, recommended, framing policy is for writers to create a new segment whenever a single binary object is written that is larger than a normal output buffer. Small objects are then appended in buffers, while larger objects are written as their own buffers. When a reader then tries to read a large object the runtime can hand it an entire buffer directly, without having to copy it.
Handshake
The purpose of the handshake is to ensure that the client and the server have each other’s protocol definition, so that the client can correctly deserialize responses, and the server can correctly deserialize requests. Both clients and servers should maintain a cache of recently seen protocols, so that, in most cases, a handshake will be completed without extra round-trip network exchanges or the transmission of full protocol text.
RPC requests and responses may not be processed until a handshake has been completed. With a stateless transport, all requests and responses are prefixed by handshakes. With a stateful transport, handshakes are only attached to requests and responses until a successful handshake response has been returned over a connection. After this, request and response payloads are sent without handshakes for the lifetime of that connection.
The handshake process uses the following record schemas:
{
"type": "record",
"name": "HandshakeRequest", "namespace":"org.apache.avro.ipc",
"fields": [
{"name": "clientHash",
"type": {"type": "fixed", "name": "MD5", "size": 16}},
{"name": "clientProtocol", "type": ["null", "string"]},
{"name": "serverHash", "type": "MD5"},
{"name": "meta", "type": ["null", {"type": "map", "values": "bytes"}]}
]
}
{
"type": "record",
"name": "HandshakeResponse", "namespace": "org.apache.avro.ipc",
"fields": [
{"name": "match",
"type": {"type": "enum", "name": "HandshakeMatch",
"symbols": ["BOTH", "CLIENT", "NONE"]}},
{"name": "serverProtocol",
"type": ["null", "string"]},
{"name": "serverHash",
"type": ["null", {"type": "fixed", "name": "MD5", "size": 16}]},
{"name": "meta",
"type": ["null", {"type": "map", "values": "bytes"}]}
]
}
- A client first prefixes each request with a
HandshakeRequest containing just the hash of its protocol and of the server’s protocol (clientHash!=null, clientProtocol=null, serverHash!=null), where the hashes are 128-bit MD5 hashes of the JSON protocol text. If a client has never connected to a given server, it sends its hash as a guess of the server’s hash, otherwise it sends the hash that it previously obtained from this server.
The server responds with a HandshakeResponse containing one of:
-
match=BOTH, serverProtocol=null, serverHash=null if the client sent the valid hash of the server’s protocol and the server knows what protocol corresponds to the client’s hash. In this case, the request is complete and the response data immediately follows the HandshakeResponse.
-
match=CLIENT, serverProtocol!=null, serverHash!=null if the server has previously seen the client’s protocol, but the client sent an incorrect hash of the server’s protocol. The request is complete and the response data immediately follows the HandshakeResponse. The client must use the returned protocol to process the response and should also cache that protocol and its hash for future interactions with this server.
-
match=NONE if the server has not previously seen the client’s protocol. The serverHash and serverProtocol may also be non-null if the server’s protocol hash was incorrect.
In this case the client must then re-submit its request with its protocol text (clientHash!=null, clientProtocol!=null, serverHash!=null) and the server should respond with a successful match (match=BOTH, serverProtocol=null, serverHash=null) as above.
The meta field is reserved for future handshake enhancements.
A call consists of a request message paired with its resulting response or error message. Requests and responses contain extensible metadata, and both kinds of messages are framed as described above.
The format of a call request is:
- request metadata, a map with values of type bytes
- the message name, an Avro string, followed by
- the message parameters. Parameters are serialized according to the message’s request declaration.
When the empty string is used as a message name a server should ignore the parameters and return an empty response. A client may use this to ping a server or to perform a handshake without sending a protocol message.
When a message is declared one-way and a stateful connection has been established by a successful handshake response, no response data is sent. Otherwise the format of the call response is:
- response metadata, a map with values of type bytes
- a one-byte error flag boolean, followed by either:
- if the error flag is false, the message response, serialized per the message’s response schema.
- if the error flag is true, the error, serialized per the message’s effective error union schema.
Schema Resolution
A reader of Avro data, whether from an RPC or a file, can always parse that data because the original schema must be provided along with the data. However, the reader may be programmed to read data into a different schema. For example, if the data was written with a different version of the software than it is read, then fields may have been added or removed from records. This section specifies how such schema differences should be resolved.
We refer to the schema used to write the data as the writer’s schema, and the schema that the application expects the reader’s schema. Differences between these should be resolved as follows:
- It is an error if the two schemas do not match.
To match, one of the following must hold:
-
both schemas are arrays whose item types match
-
both schemas are maps whose value types match
-
both schemas are enums whose (unqualified) names match
-
both schemas are fixed whose sizes and (unqualified) names match
-
both schemas are records with the same (unqualified) name
-
either schema is a union
-
both schemas have same primitive type
-
the writer’s schema may be promoted to the reader’s as follows:
- int is promotable to long, float, or double
- long is promotable to float or double
- float is promotable to double
- string is promotable to bytes
- bytes is promotable to string
-
if both are records:
- the ordering of fields may be different: fields are matched by name.
- schemas for fields with the same name in both records are resolved recursively.
- if the writer’s record contains a field with a name not present in the reader’s record, the writer’s value for that field is ignored.
- if the reader’s record schema has a field that contains a default value, and writer’s schema does not have a field with the same name, then the reader should use the default value from its field.
- if the reader’s record schema has a field with no default value, and writer’s schema does not have a field with the same name, an error is signalled.
-
if both are enums:
if the writer’s symbol is not present in the reader’s enum and the reader has a default value, then that value is used, otherwise an error is signalled.
This resolution algorithm is applied recursively to the reader’s and writer’s array item schemas.
This resolution algorithm is applied recursively to the reader’s and writer’s value schemas.
The first schema in the reader’s union that matches the selected writer’s union schema is recursively resolved against it. if none match, an error is signalled.
- if reader’s is a union, but writer’s is not
The first schema in the reader’s union that matches the writer’s schema is recursively resolved against it. If none match, an error is signalled.
- if writer’s is a union, but reader’s is not
If the reader’s schema matches the selected writer’s schema, it is recursively resolved against it. If they do not match, an error is signalled.
A schema’s doc fields are ignored for the purposes of schema resolution. Hence, the doc portion of a schema may be dropped at serialization.
One of the defining characteristics of Avro is that a reader must use the schema used by the writer of the data in order to know how to read the data. This assumption results in a data format that’s compact and also amenable to many forms of schema evolution. However, the specification so far has not defined what it means for the reader to have the “same” schema as the writer. Does the schema need to be textually identical? Well, clearly adding or removing some whitespace to a JSON expression does not change its meaning. At the same time, reordering the fields of records clearly does change the meaning. So what does it mean for a reader to have “the same” schema as a writer?
Parsing Canonical Form is a transformation of a writer’s schema that let’s us define what it means for two schemas to be “the same” for the purpose of reading data written against the schema. It is called Parsing Canonical Form because the transformations strip away parts of the schema, like “doc” attributes, that are irrelevant to readers trying to parse incoming data. It is called Canonical Form because the transformations normalize the JSON text (such as the order of attributes) in a way that eliminates unimportant differences between schemas. If the Parsing Canonical Forms of two different schemas are textually equal, then those schemas are “the same” as far as any reader is concerned, i.e., there is no serialized data that would allow a reader to distinguish data generated by a writer using one of the original schemas from data generated by a writing using the other original schema. (We sketch a proof of this property in a companion document.)
The next subsection specifies the transformations that define Parsing Canonical Form. But with a well-defined canonical form, it can be convenient to go one step further, transforming these canonical forms into simple integers (“fingerprints”) that can be used to uniquely identify schemas. The subsection after next recommends some standard practices for generating such fingerprints.
Assuming an input schema (in JSON form) that’s already UTF-8 text for a valid Avro schema (including all quotes as required by JSON), the following transformations will produce its Parsing Canonical Form:
- [PRIMITIVES] Convert primitive schemas to their simple form (e.g., int instead of
{"type":"int"}).
- [FULLNAMES] Replace short names with fullnames, using applicable namespaces to do so. Then eliminate namespace attributes, which are now redundant.
- [STRIP] Keep only attributes that are relevant to parsing data, which are: type, name, fields, symbols, items, values, size. Strip all others (e.g., doc and aliases).
- [ORDER] Order the appearance of fields of JSON objects as follows: name, type, fields, symbols, items, values, size. For example, if an object has type, name, and size fields, then the name field should appear first, followed by the type and then the size fields.
- [STRINGS] For all JSON string literals in the schema text, replace any escaped characters (e.g., \uXXXX escapes) with their UTF-8 equivalents.
- [INTEGERS] Eliminate quotes around and any leading zeros in front of JSON integer literals (which appear in the size attributes of fixed schemas).
- [WHITESPACE] Eliminate all whitespace in JSON outside of string literals.
Schema Fingerprints
“[A] fingerprinting algorithm is a procedure that maps an arbitrarily large data item (such as a computer file) to a much shorter bit string, its fingerprint, that uniquely identifies the original data for all practical purposes” (quoted from Wikipedia). In the Avro context, fingerprints of Parsing Canonical Form can be useful in a number of applications; for example, to cache encoder and decoder objects, to tag data items with a short substitute for the writer’s full schema, and to quickly negotiate common-case schemas between readers and writers.
In designing fingerprinting algorithms, there is a fundamental trade-off between the length of the fingerprint and the probability of collisions. To help application designers find appropriate points within this trade-off space, while encouraging interoperability and ease of implementation, we recommend using one of the following three algorithms when fingerprinting Avro schemas:
- When applications can tolerate longer fingerprints, we recommend using the SHA-256 digest algorithm to generate 256-bit fingerprints of Parsing Canonical Forms. Most languages today have SHA-256 implementations in their libraries.
- At the opposite extreme, the smallest fingerprint we recommend is a 64-bit Rabin fingerprint. Below, we provide pseudo-code for this algorithm that can be easily translated into any programming language. 64-bit fingerprints should guarantee uniqueness for schema caches of up to a million entries (for such a cache, the chance of a collision is 3E-8). We don’t recommend shorter fingerprints, as the chances of collisions is too great (for example, with 32-bit fingerprints, a cache with as few as 100,000 schemas has a 50% chance of having a collision).
- Between these two extremes, we recommend using the MD5 message digest to generate 128-bit fingerprints. These make sense only where very large numbers of schemas are being manipulated (tens of millions); otherwise, 64-bit fingerprints should be sufficient. As with SHA-256, MD5 implementations are found in most libraries today.
These fingerprints are not meant to provide any security guarantees, even the longer SHA-256-based ones. Most Avro applications should be surrounded by security measures that prevent attackers from writing random data and otherwise interfering with the consumers of schemas. We recommend that these surrounding mechanisms be used to prevent collision and pre-image attacks (i.e., “forgery”) on schema fingerprints, rather than relying on the security properties of the fingerprints themselves.
Rabin fingerprints are cyclic redundancy checks computed using irreducible polynomials. In the style of the Appendix of RFC 1952 (pg 10), which defines the CRC-32 algorithm, here’s our definition of the 64-bit AVRO fingerprinting algorithm:
long fingerprint64(byte[] buf) {
if (FP_TABLE == null) initFPTable();
long fp = EMPTY;
for (int i = 0; i < buf.length; i++)
fp = (fp >>> 8) ^ FP_TABLE[(int)(fp ^ buf[i]) & 0xff];
return fp;
}
static long EMPTY = 0xc15d213aa4d7a795L;
static long[] FP_TABLE = null;
void initFPTable() {
FP_TABLE = new long[256];
for (int i = 0; i < 256; i++) {
long fp = i;
for (int j = 0; j < 8; j++)
fp = (fp >>> 1) ^ (EMPTY & -(fp & 1L));
FP_TABLE[i] = fp;
}
}
Readers interested in the mathematics behind this algorithm may want to read Chapter 14 of the Second Edition of Hacker’s Delight. (Unlike RFC-1952 and the book chapter, we prepend a single one bit to messages. We do this because CRCs ignore leading zero bits, which can be problematic. Our code prepends a one-bit by initializing fingerprints using EMPTY, rather than initializing using zero as in RFC-1952 and the book chapter.)
Logical Types
A logical type is an Avro primitive or complex type with extra attributes to represent a derived type. The attribute logicalType must always be present for a logical type, and is a string with the name of one of the logical types listed later in this section. Other attributes may be defined for particular logical types.
A logical type is always serialized using its underlying Avro type so that values are encoded in exactly the same way as the equivalent Avro type that does not have a logicalType attribute. Language implementations may choose to represent logical types with an appropriate native type, although this is not required.
Language implementations must ignore unknown logical types when reading, and should use the underlying Avro type. If a logical type is invalid, for example a decimal with scale greater than its precision, then implementations should ignore the logical type and use the underlying Avro type.
Decimal
The decimal logical type represents an arbitrary-precision signed decimal number of the form unscaled × 10-scale.
A decimal logical type annotates Avro bytes or fixed types. The byte array must contain the two’s-complement representation of the unscaled integer value in big-endian byte order. The scale is fixed, and is specified using an attribute.
The following attributes are supported:
- scale, a JSON integer representing the scale (optional). If not specified the scale is 0.
- precision, a JSON integer representing the (maximum) precision of decimals stored in this type (required).
For example, the following schema represents decimal numbers with a maximum precision of 4 and a scale of 2:
{
"type": "bytes",
"logicalType": "decimal",
"precision": 4,
"scale": 2
}
Precision must be a positive integer greater than zero. If the underlying type is a fixed, then the precision is limited by its size. An array of length n can store at most floor(log10(28 × n - 1 - 1)) base-10 digits of precision.
Scale must be zero or a positive integer less than or equal to the precision.
For the purposes of schema resolution, two schemas that are decimal logical types match if their scales and precisions match.
UUID
The uuid logical type represents a random generated universally unique identifier (UUID).
A uuid logical type annotates an Avro string. The string has to conform with RFC-4122
Date
The date logical type represents a date within the calendar, with no reference to a particular time zone or time of day.
A date logical type annotates an Avro int, where the int stores the number of days from the unix epoch, 1 January 1970 (ISO calendar).
Time (millisecond precision)
The time-millis logical type represents a time of day, with no reference to a particular calendar, time zone or date, with a precision of one millisecond.
A time-millis logical type annotates an Avro int, where the int stores the number of milliseconds after midnight, 00:00:00.000.
Time (microsecond precision)
The time-micros logical type represents a time of day, with no reference to a particular calendar, time zone or date, with a precision of one microsecond.
A time-micros logical type annotates an Avro long, where the long stores the number of microseconds after midnight, 00:00:00.000000.
Timestamp (millisecond precision)
The timestamp-millis logical type represents an instant on the global timeline, independent of a particular time zone or calendar, with a precision of one millisecond.
A timestamp-millis logical type annotates an Avro long, where the long stores the number of milliseconds from the unix epoch, 1 January 1970 00:00:00.000 UTC.
Timestamp (microsecond precision)
The timestamp-micros logical type represents an instant on the global timeline, independent of a particular time zone or calendar, with a precision of one microsecond.
A timestamp-micros logical type annotates an Avro long, where the long stores the number of microseconds from the unix epoch, 1 January 1970 00:00:00.000000 UTC.
Duration
The duration logical type represents an amount of time defined by a number of months, days and milliseconds. This is not equivalent to a number of milliseconds, because, depending on the moment in time from which the duration is measured, the number of days in the month and number of milliseconds in a day may differ. Other standard periods such as years, quarters, hours and minutes can be expressed through these basic periods.
A duration logical type annotates Avro fixed type of size 12, which stores three little-endian unsigned integers that represent durations at different granularities of time. The first stores a number in months, the second stores a number in days, and the third stores a number in milliseconds.
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
6.4 - Trevni
Version 0.1
DRAFT
This document is the authoritative specification of a file format. Its intent is to permit compatible, independent implementations that read and/or write files in this format.
Introduction
Data sets are often described as a table composed of rows and columns. Each record in the dataset is considered a row, with each field of the record occupying a different column. Writing records to a file one-by-one as they are created results in a row-major format, like Hadoop’s SequenceFile or Avro data files.
In many cases higher query performance may be achieved if the data is instead organized in a column-major format, where multiple values of a given column are stored adjacently. This document defines such a column-major file format for datasets.
To permit scalable, distributed query evaluation, datasets are partitioned into row groups, containing distinct collections of rows. Each row group is organized in column-major order, while row groups form a row-major partitioning of the entire dataset.
Rationale
Goals
The format is meant satisfy the following goals:
Maximize the size of row groups. Disc drives are used most efficiently when sequentially accessing data. Consider a drive that takes 10ms to seek and transfers at 100MB/second. If a 10-column dataset whose values are all the same size is split into 10MB row groups, then accessing a single column will require a sequence of seek+1MB reads, for a cost of 20ms/MB processed. If the same dataset is split into 100MB row groups then this drops to 11ms/MB processed. This effect is exaggerated for datasets with larger numbers of columns and with columns whose values are smaller than average. So we’d prefer row groups that are 100MB or greater.
Permit random access within a row group. Some queries will first examine one column, and, only when certain relatively rare criteria are met, examine other columns. Rather than iterating through selected columns of the row-group in parallel, one might iterate through one column and randomly access another. This is called support for WHERE clauses, after the SQL operator of that name.
Minimize the number of files per dataset. HDFS is a primary intended deployment platform for these files. The HDFS Namenode requires memory for each file in the filesystem, thus for a format to be HDFS-friendly it should strive to require the minimum number of distinct files.
Support co-location of columns within row-groups. Row groups are the unit of parallel operation on a column dataset. For efficient file i/o, the entirety of a row-group should ideally reside on the host that is evaluating the query in order to avoid network latencies and bottlenecks.
Data integrity. The format should permit applications to detect data corruption. Many file systems may prevent corruption, but files may be moved between filesystems and be subject to corruption at points in that process. It is best if the data in a file can be validated independently.
Extensibility. The format should permit applications to store additional annotations about a datasets in the files, such as type information, origin, etc. Some environments may have metadata stores for such information, but not all do, and files might be moved among systems with different metadata systems. The ability to keep such information within the file simplifies the coordination of such information.
Minimal overhead. The column format should not make datasets appreciably larger. Storage is a primary cost and a choice to use this format should not require additional storage.
Primary format. The column format should be usable as a primary format for datasets, not as an auxiliary, accelerated format. Applications that process a dataset in row-major order should be able to easily consume column files and applications that produce datasets in row-major order should be able to easily generate column files.
Design
To meet these goals we propose the following design.
Each row group is a separate file. All values of a column in a file are written contiguously. This maximizes the row group size, optimizing performance when querying few and small columns.
Each file occupies a single HDFS block. A larger than normal block size may be specified, e.g., ~1GB instead of the typical ~100MB. This guarantees co-location and eliminates network use when query processing can be co-located with the file. This also moderates the memory impact on the HDFS Namenode since no small files are written.
Each column in a file is written as a sequence of ~64kB compressed blocks. The sequence is prefixed by a table describing all of the blocks in the column to permit random access within the column.
Application-specific metadata may be added at the file, column, and block levels.
Checksums are included with each block, providing data integrity.
Discussion
The use of a single block per file achieves the same effect as the custom block placement policy described in the CIF paper, but while still permitting HDFS rebalancing and not increasing the number of files in the namespace.
This section formally describes the proposed column file format.
Data Model
We assume a simple data model, where a record is a set of named fields, and the value of each field is a sequence of untyped bytes. A type system may be layered on top of this, as specified in the Type Mapping section below.
Primitive Values
We define the following primitive value types:
- Signed 64-bit long values are written using a variable-length zig-zag coding, where the high-order bit in each byte determines whether subsequent bytes are present. For example:
decimal value hex bytes
| decimal value |
hex bytes |
|
| 0 |
00 |
|
| -1 |
01 |
|
| 1 |
02 |
|
| … |
|
|
| -64 |
7f |
|
| 64 |
80 01 |
|
| … |
|
|
- bytes are encoded as a long followed by that many bytes of data.
- a string is encoded as a long followed by that many bytes of UTF-8 encoded character data.
For example, the three-character string “foo” would be encoded as the long value 3 (encoded as hex 06) followed by the UTF-8 encoding of ‘f’, ‘o’, and ‘o’ (the hex bytes 66 6f 6f): 06 66 6f 6f
Type Names
The following type names are used to describe column values:
- null, requires zero bytes. Sometimes used in array columns.
- boolean, one bit, packed into bytes, little-endian;
- int, like long, but restricted to 32-bit signed values
- long 64-bit signed values, represented as above
- fixed32 32-bit values stored as four bytes, little-endian.
- fixed64 64-bit values stored as eight bytes, little-endian.
- float 32-bit IEEE floating point value, little-endian
- double 64-bit IEEE floating point value, little-endian
- string as above
- bytes as above, may be used to encapsulate more complex objects
Type names are represented as strings (UTF-8 encoded, length-prefixed).
Metadata consists of:
- A long indicating the number of metadata key/value pairs.
- For each pair, a string key and bytes value.
All metadata properties that start with “trevni.” are reserved.
The following file metadata properties are defined:
- trevni.codec the name of the default compression codec used to compress blocks, as a string. Implementations are required to support the “null” codec. Optional. If absent, it is assumed to be “null”. Codecs are described in more detail below.
- trevni.checksum the name of the checksum algorithm used in this file, as a string. Implementations are required to support the “crc-32” checksum. Optional. If absent, it is assumed to be “null”. Checksums are described in more detail below.
The following column metadata properties are defined:
- trevni.codec the name of the compression codec used to compress the blocks of this column, as a string. Implementations are required to support the “null” codec. Optional. If absent, it is assumed to be “null”. Codecs are described in more detail below.
- trevni.name the name of the column, as a string. Required.
- trevni.type the type of data in the column. One of the type names above. Required.
- trevni.values if present, indicates that the initial value of each block in this column will be stored in the block’s descriptor. Not permitted for array columns or columns that specify a parent.
- trevni.array if present, indicates that each row in this column contains a sequence of values of the named type rather than just a single value. An integer length precedes each sequence of values indicating the count of values in the sequence. If the length is negative then it indicates a sequence of zero or one lengths, where -1 indicates two zeros, -2 two ones, -3 three zeros, -4 three ones, etc.
- trevni.parent if present, the name of an array column whose lengths are also used by this column. Thus values of this column are sequences but no lengths are stored in this column.
For example, consider the following row, as JSON, where all values are primitive types, but one has multiple values.
{"id"=566, "date"=23423234234
"from"="foo@bar.com",
"to"=["bar@baz.com", "bang@foo.com"],
"content"="Hi!"}
The columns for this might be specified as:
name=id type=int
name=date type=long
name=from type=string
name=to type=string array=true
name=content type=string
If a row contains an array of records, e.g. “received” in the following:
{"id"=566, "date"=23423234234
"from"="foo@bar.com",
"to"=["bar@baz.com", "bang@foo.com"],
"content"="Hi!"
"received"=[{"date"=234234234234, "host"="192.168.0.0.1"},
{"date"=234234545645, "host"="192.168.0.0.2"}]
}
Then one can define a parent column followed by a column for each field in the record, adding the following columns:
name=received type=null array=true
name=date type=long parent=received
name=host type=string parent=received
If an array value itself contains an array, e.g. the “sigs” below:
{"id"=566, "date"=23423234234
"from"="foo@bar.com",
"to"=["bar@baz.com", "bang@foo.com"],
"content"="Hi!"
"received"=[{"date"=234234234234, "host"="192.168.0.0.1",
"sigs"=[{"algo"="weak", "value"="0af345de"}]},
{"date"=234234545645, "host"="192.168.0.0.2",
"sigs"=[]}]
}
Then a parent column may be defined that itself has a parent column.
name=sigs type=null array=true parent=received
name=algo type=string parent=sigs
name=value type=string parent=sigs
No block metadata properties are currently defined.
A file consists of:
- A file header, followed by
- one or more columns.
A file header consists of:
- Four bytes, ASCII ‘T’, ‘r’, ‘v’, followed by 0x02.
- a fixed64 indicating the number of rows in the file
- a fixed32 indicating the number of columns in the file
- file metadata.
- for each column, its column metadata
- for each column, its starting position in the file as a fixed64.
A column consists of:
- A fixed32 indicating the number of blocks in this column.
- For each block, a block descriptor
- One or more blocks.
A block descriptor consists of:
- A fixed32 indicating the number of rows in the block
- A fixed32 indicating the size in bytes of the block before the codec is applied (excluding checksum).
- A fixed32 indicating the size in bytes of the block after the codec is applied (excluding checksum).
- If this column’s metadata declares it to include values, the first value in the column, serialized according to this column’s type.
A block consists of:
- The serialized column values. If a column is an array column then value sequences are preceded by their length, as an int. If a codec is specified, the values and lengths are compressed by that codec.
- The checksum, as determined by the file metadata.
Codecs
null
The “null” codec simply passes data through uncompressed.
deflate
The “deflate” codec writes the data block using the deflate algorithm as specified in RFC 1951.
snappy
The “snappy” codec uses Google’s Snappy compression library.
Checksum algorithms
null
The “null” checksum contains zero bytes.
crc-32
Each “crc-32” checksum contains the four bytes of an ISO 3309 CRC-32 checksum of the uncompressed block data as a fixed32.
Type Mappings
We define a standard mapping for how types defined in various serialization systems are represented in a column file. Records from these systems are shredded into columns. When records are nested, a depth-first recursive walk can assign a separate column for each primitive value.
Avro
Protocol Buffers
Thrift
Implementation Notes
Some possible techniques for writing column files include:
- Use a standard ~100MB block, buffer in memory up to the block size, then flush the file directly to HDFS. A single reduce task might create multiple output files. The namenode requires memory proportional to the number of names and blocks*replication. This would increase the number of names but not blocks, so this should still be much better than a file per column.
- Spill each column to a separate local, temporary file then, when the file is closed, append these files, writing a single file to HDFS whose block size is set to be that of the entire file. This would be a bit slower than and may have trouble when the local disk is full, but it would better use HDFS namespace and further reduce seeks when processing columns whose values are small.
- Use a separate mapreduce job to convert row-major files to column-major. The map output would output a by (row#, column#, value) tuple, partitioned by row# but sorted by column# then row#. The reducer could directly write the column file. But the column file format would need to be changed to write counts, descriptors, etc. at the end of files rather than at the front.
(1) is the simplest to implement and most implementations should start with it.
References
CIF Column-Oriented Storage Techniques for MapReduce, Floratou, Patel, Shekita, & Tata, VLDB 2011.
DREMEL Dremel: Interactive Analysis of Web-Scale Datasets, Melnik, Gubarev, Long, Romer, Shivakumar, & Tolton, VLDB 2010.
6.5 - MapReduce guide
Avro provides a convenient way to represent complex data structures within a Hadoop MapReduce job. Avro data can be used as both input to and output from a MapReduce job, as well as the intermediate format. The example in this guide uses Avro data for all three, but it’s possible to mix and match; for instance, MapReduce can be used to aggregate a particular field in an Avro record.
This guide assumes basic familiarity with both Hadoop MapReduce and Avro. See the Hadoop documentation and the Avro getting started guide for introductions to these projects. This guide uses the old MapReduce API (org.apache.hadoop.mapred) and the new MapReduce API (org.apache.hadoop.mapreduce).
Setup
The code from this guide is included in the Avro docs under examples/mr-example. The example is set up as a Maven project that includes the necessary Avro and MapReduce dependencies and the Avro Maven plugin for code generation, so no external jars are needed to run the example. In particular, the POM includes the following dependencies:
<dependency>
<groupId>org.apache.avro</groupId>
<artifactId>avro</artifactId>
<version>1.9.2</version>
</dependency>
<dependency>
<groupId>org.apache.avro</groupId>
<artifactId>avro-mapred</artifactId>
<version>1.9.2</version>
</dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-client</artifactId>
<version>3.1.2</version>
</dependency>
And the following plugin:
<plugin>
<groupId>org.apache.avro</groupId>
<artifactId>avro-maven-plugin</artifactId>
<version>1.9.2</version>
<executions>
<execution>
<phase>generate-sources</phase>
<goals>
<goal>schema</goal>
</goals>
<configuration>
<sourceDirectory>${project.basedir}/../</sourceDirectory>
<outputDirectory>${project.basedir}/target/generated-sources/</outputDirectory>
</configuration>
</execution>
</executions>
</plugin>
If you do not configure the sourceDirectory and outputDirectory properties, the defaults will be used. The sourceDirectory property defaults to src/main/avro. The outputDirectory property defaults to target/generated-sources. You can change the paths to match your project layout.
Alternatively, Avro jars can be downloaded directly from the Apache Avro™ Releases page. The relevant Avro jars for this guide are avro-1.9.2.jar and avro-mapred-1.9.2.jar, as well as avro-tools-1.9.2.jar for code generation and viewing Avro data files as JSON. In addition, you will need to install Hadoop in order to use MapReduce.
Example: ColorCount
Below is a simple example of a MapReduce that uses Avro. There is an example for both the old (org.apache.hadoop.mapred) and new (org.apache.hadoop.mapreduce) APIs under examples/mr-example/src/main/java/example/. MapredColorCount is the example for the older mapred API while MapReduceColorCount is the example for the newer mapreduce API. Both examples are below, but we will detail the mapred API in our subsequent examples.
MapredColorCount:
package example;
import java.io.IOException;
import org.apache.avro.*;
import org.apache.avro.Schema.Type;
import org.apache.avro.mapred.*;
import org.apache.hadoop.conf.*;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.mapred.*;
import org.apache.hadoop.util.*;
import example.avro.User;
public class MapredColorCount extends Configured implements Tool {
public static class ColorCountMapper extends AvroMapper<User, Pair<CharSequence, Integer>> {
@Override
public void map(User user, AvroCollector<Pair<CharSequence, Integer>> collector, Reporter reporter)
throws IOException {
CharSequence color = user.getFavoriteColor();
// We need this check because the User.favorite_color field has type ["string", "null"]
if (color == null) {
color = "none";
}
collector.collect(new Pair<CharSequence, Integer>(color, 1));
}
}
public static class ColorCountReducer extends AvroReducer<CharSequence, Integer,
Pair<CharSequence, Integer>> {
@Override
public void reduce(CharSequence key, Iterable<Integer> values,
AvroCollector<Pair<CharSequence, Integer>> collector,
Reporter reporter)
throws IOException {
int sum = 0;
for (Integer value : values) {
sum += value;
}
collector.collect(new Pair<CharSequence, Integer>(key, sum));
}
}
public int run(String[] args) throws Exception {
if (args.length != 2) {
System.err.println("Usage: MapredColorCount <input path> <output path>");
return -1;
}
JobConf conf = new JobConf(getConf(), MapredColorCount.class);
conf.setJobName("colorcount");
FileInputFormat.setInputPaths(conf, new Path(args[0]));
FileOutputFormat.setOutputPath(conf, new Path(args[1]));
AvroJob.setMapperClass(conf, ColorCountMapper.class);
AvroJob.setReducerClass(conf, ColorCountReducer.class);
// Note that AvroJob.setInputSchema and AvroJob.setOutputSchema set
// relevant config options such as input/output format, map output
// classes, and output key class.
AvroJob.setInputSchema(conf, User.getClassSchema());
AvroJob.setOutputSchema(conf, Pair.getPairSchema(Schema.create(Type.STRING),
Schema.create(Type.INT)));
JobClient.runJob(conf);
return 0;
}
public static void main(String[] args) throws Exception {
int res = ToolRunner.run(new Configuration(), new MapredColorCount(), args);
System.exit(res);
}
}
MapReduceColorCount:
package example;
import java.io.IOException;
import org.apache.avro.Schema;
import org.apache.avro.mapred.AvroKey;
import org.apache.avro.mapred.AvroValue;
import org.apache.avro.mapreduce.AvroJob;
import org.apache.avro.mapreduce.AvroKeyInputFormat;
import org.apache.avro.mapreduce.AvroKeyValueOutputFormat;
import org.apache.hadoop.conf.Configured;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.IntWritable;
import org.apache.hadoop.io.NullWritable;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.mapreduce.Job;
import org.apache.hadoop.mapreduce.Mapper;
import org.apache.hadoop.mapreduce.Reducer;
import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
import org.apache.hadoop.util.Tool;
import org.apache.hadoop.util.ToolRunner;
import example.avro.User;
public class MapReduceColorCount extends Configured implements Tool {
public static class ColorCountMapper extends
Mapper<AvroKey<User>, NullWritable, Text, IntWritable> {
@Override
public void map(AvroKey<User> key, NullWritable value, Context context)
throws IOException, InterruptedException {
CharSequence color = key.datum().getFavoriteColor();
if (color == null) {
color = "none";
}
context.write(new Text(color.toString()), new IntWritable(1));
}
}
public static class ColorCountReducer extends
Reducer<Text, IntWritable, AvroKey<CharSequence>, AvroValue<Integer>> {
@Override
public void reduce(Text key, Iterable<IntWritable> values,
Context context) throws IOException, InterruptedException {
int sum = 0;
for (IntWritable value : values) {
sum += value.get();
}
context.write(new AvroKey<CharSequence>(key.toString()), new AvroValue<Integer>(sum));
}
}
public int run(String[] args) throws Exception {
if (args.length != 2) {
System.err.println("Usage: MapReduceColorCount <input path> <output path>");
return -1;
}
Job job = new Job(getConf());
job.setJarByClass(MapReduceColorCount.class);
job.setJobName("Color Count");
FileInputFormat.setInputPaths(job, new Path(args[0]));
FileOutputFormat.setOutputPath(job, new Path(args[1]));
job.setInputFormatClass(AvroKeyInputFormat.class);
job.setMapperClass(ColorCountMapper.class);
AvroJob.setInputKeySchema(job, User.getClassSchema());
job.setMapOutputKeyClass(Text.class);
job.setMapOutputValueClass(IntWritable.class);
job.setOutputFormatClass(AvroKeyValueOutputFormat.class);
job.setReducerClass(ColorCountReducer.class);
AvroJob.setOutputKeySchema(job, Schema.create(Schema.Type.STRING));
AvroJob.setOutputValueSchema(job, Schema.create(Schema.Type.INT));
return (job.waitForCompletion(true) ? 0 : 1);
}
public static void main(String[] args) throws Exception {
int res = ToolRunner.run(new MapReduceColorCount(), args);
System.exit(res);
}
}
ColorCount reads in data files containing User records, defined in examples/user.avsc, and counts the number of instances of each favorite color. (This example draws inspiration from the canonical WordCount MapReduce application.) This example uses the old MapReduce API. See MapReduceAvroWordCount, found under doc/examples/mr-example/src/main/java/example/ to see the new MapReduce API example. The User schema is defined as follows:
{"namespace": "example.avro",
"type": "record",
"name": "User",
"fields": [
{"name": "name", "type": "string"},
{"name": "favorite_number", "type": ["int", "null"]},
{"name": "favorite_color", "type": ["string", "null"]}
]
}
This schema is compiled into the User class used by ColorCount via the Avro Maven plugin (see examples/mr-example/pom.xml for how this is set up).
ColorCountMapper essentially takes a User as input and extracts the User’s favorite color, emitting the key-value pair <favoriteColor, 1>. ColorCountReducer then adds up how many occurrences of a particular favorite color were emitted, and outputs the result as a Pair record. These Pairs are serialized to an Avro data file.
Running ColorCount
The ColorCount application is provided as a Maven project in the Avro docs under examples/mr-example. To build the project, including the code generation of the User schema, run:
Next, run GenerateData from examples/mr-examples to create an Avro data file, input/users.avro, containing 20 Users with favorite colors chosen randomly from a list:
mvn exec:java -q -Dexec.mainClass=example.GenerateData
Besides creating the data file, GenerateData prints the JSON representations of the Users generated to stdout, for example:
{"name": "user", "favorite_number": null, "favorite_color": "red"}
{"name": "user", "favorite_number": null, "favorite_color": "green"}
{"name": "user", "favorite_number": null, "favorite_color": "purple"}
{"name": "user", "favorite_number": null, "favorite_color": null}
...
Now we’re ready to run ColorCount. We specify our freshly-generated input folder as the input path and output as our output folder (note that MapReduce will not start a job if the output folder already exists):
mvn exec:java -q -Dexec.mainClass=example.MapredColorCount -Dexec.args="input output"
Once ColorCount completes, checking the contents of the new output directory should yield the following:
$ ls output/
part-00000.avro _SUCCESS
You can check the contents of the generated Avro file using the avro-tools jar:
$ java -jar /path/to/avro-tools-1.9.2.jar tojson output/part-00000.avro
{"value": 3, "key": "blue"}
{"value": 7, "key": "green"}
{"value": 1, "key": "none"}
{"value": 2, "key": "orange"}
{"value": 3, "key": "purple"}
{"value": 2, "key": "red"}
{"value": 2, "key": "yellow"}
Now let’s go over the ColorCount example in detail.
Mapper - org.apache.hadoop.mapred API
The easiest way to use Avro data files as input to a MapReduce job is to subclass AvroMapper. An AvroMapper defines a map function that takes an Avro datum as input and outputs a key/value pair represented as a Pair record. In the ColorCount example, ColorCountMapper is an AvroMapper that takes a User as input and outputs a Pair<CharSequence, Integer>>, where the CharSequence key is the user’s favorite color and the Integer value is 1.
public static class ColorCountMapper extends AvroMapper<User, Pair<CharSequence, Integer>> {
@Override
public void map(User user, AvroCollector<Pair<CharSequence, Integer>> collector, Reporter reporter)
throws IOException {
CharSequence color = user.getFavoriteColor();
// We need this check because the User.favorite_color field has type ["string", "null"]
if (color == null) {
color = "none";
}
collector.collect(new Pair<CharSequence, Integer>(color, 1));
}
}
In order to use our AvroMapper, we must call AvroJob.setMapperClass and AvroJob.setInputSchema.
AvroJob.setMapperClass(conf, ColorCountMapper.class);
AvroJob.setInputSchema(conf, User.getClassSchema());
Note that AvroMapper does not implement the Mapper interface. Under the hood, the specified Avro data files are deserialized into AvroWrappers containing the actual data, which are processed by a Mapper that calls the configured AvroMapper’s map function. AvroJob.setInputSchema sets up the relevant configuration parameters needed to make this happen, thus you should not need to call JobConf.setMapperClass, JobConf.setInputFormat, JobConf.setMapOutputKeyClass, JobConf.setMapOutputValueClass, or JobConf.setOutputKeyComparatorClass.
Mapper - org.apache.hadoop.mapreduce API
This document will not go into all the differences between the mapred and mapreduce APIs, however will describe the main differences. As you can see, ColorCountMapper is now a subclass of the Hadoop Mapper class and is passed an AvroKey as it’s key. Additionally, the AvroJob method calls were slightly changed.
public static class ColorCountMapper extends
Mapper<AvroKey<User>, NullWritable, Text, IntWritable> {
@Override
public void map(AvroKey<User> key, NullWritable value, Context context)
throws IOException, InterruptedException {
CharSequence color = key.datum().getFavoriteColor();
if (color == null) {
color = "none";
}
context.write(new Text(color.toString()), new IntWritable(1));
}
}
Reducer - org.apache.hadoop.mapred API
Analogously to AvroMapper, an AvroReducer defines a reducer function that takes the key/value types output by an AvroMapper (or any mapper that outputs Pairs) and outputs a key/value pair represented a Pair record. In the ColorCount example, ColorCountReducer is an AvroReducer that takes the CharSequence key representing a favorite color and the Iterable<Integer> representing the counts for that color (they should all be 1 in this example) and adds up the counts.
public static class ColorCountReducer extends AvroReducer<CharSequence, Integer,
Pair<CharSequence, Integer>> {
@Override
public void reduce(CharSequence key, Iterable<Integer> values,
AvroCollector<Pair<CharSequence, Integer>> collector,
Reporter reporter)
throws IOException {
int sum = 0;
for (Integer value : values) {
sum += value;
}
collector.collect(new Pair<CharSequence, Integer>(key, sum));
}
}
In order to use our AvroReducer, we must call AvroJob.setReducerClass and AvroJob.setOutputSchema.
AvroJob.setReducerClass(conf, ColorCountReducer.class);
AvroJob.setOutputSchema(conf, Pair.getPairSchema(Schema.create(Type.STRING),
Schema.create(Type.INT)));
Note that AvroReducer does not implement the Reducer interface. The intermediate Pairs output by the mapper are split into AvroKeys and AvroValues, which are processed by a Reducer that calls the configured AvroReducer’s reduce function. AvroJob.setOutputSchema sets up the relevant configuration parameters needed to make this happen, thus you should not need to call JobConf.setReducerClass, JobConf.setOutputFormat, JobConf.setOutputKeyClass, JobConf.setMapOutputKeyClass, JobConf.setMapOutputValueClass, or JobConf.setOutputKeyComparatorClass.
Reduce - org.apache.hadoop.mapreduce API
As before we not detail every difference between the APIs. As with the Mapper change ColorCountReducer is now a subclass of Reducer and AvroKey and AvroValue are emitted. Additionally, the AvroJob method calls were slightly changed.
public static class ColorCountReducer extends
Reducer<Text, IntWritable, AvroKey<CharSequence>, AvroValue<Integer>> {
@Override
public void reduce(Text key, Iterable<IntWritable> values,
Context context) throws IOException, InterruptedException {
int sum = 0;
for (IntWritable value : values) {
sum += value.get();
}
context.write(new AvroKey<CharSequence>(key.toString()), new AvroValue<Integer>(sum));
}
}
Learning more
The mapred API allows users to mix Avro AvroMappers and AvroReducers with non-Avro Mappers and Reducers and the mapreduce API allows users input Avro and output non-Avro or vice versa.
The mapred package has API org.apache.avro.mapred documentation as does the org.apache.avro.mapreduce package. MapReduce API (org.apache.hadoop.mapreduce). Similarily to the mapreduce package, it’s possible with the mapred API to implement your own Mappers and Reducers directly using the public classes provided in these libraries. See the AvroWordCount application, found under examples/mr-example/src/main/java/example/AvroWordCount.java in the Avro documentation, for an example of implementing a Reducer that outputs Avro data using the old MapReduce API. See the MapReduceAvroWordCount application, found under examples/mr-example/src/main/java/example/MapReduceAvroWordCount.java in the Avro documentation, for an example of implementing a Reducer that outputs Avro data using the new MapReduce API.
6.6 - IDL Language
Introduction
This document defines Avro IDL, a higher-level language for authoring Avro schemata. Before reading this document, you should have familiarity with the concepts of schemata and protocols, as well as the various primitive and complex types available in Avro.
Overview
Purpose
The aim of the Avro IDL language is to enable developers to author schemata in a way that feels more similar to common programming languages like Java, C++, or Python. Additionally, the Avro IDL language may feel more familiar for those users who have previously used the interface description languages (IDLs) in other frameworks like Thrift, Protocol Buffers, or CORBA.
Usage
Each Avro IDL file defines a single Avro Protocol, and thus generates as its output a JSON-format Avro Protocol file with extension .avpr.
To convert a .avdl file into a .avpr file, it may be processed by the idl tool. For example:
$ java -jar avroj-tools.jar idl src/test/idl/input/namespaces.avdl /tmp/namespaces.avpr
$ head /tmp/namespaces.avpr
{
"protocol" : "TestNamespace",
"namespace" : "avro.test.protocol",
The idl tool can also process input to and from stdin and stdout. See idl --help for full usage information.
A Maven plugin is also provided to compile .avdl files. To use it, add something like the following to your pom.xml:
<build>
<plugins>
<plugin>
<groupId>org.apache.avro</groupId>
<artifactId>avro-maven-plugin</artifactId>
<executions>
<execution>
<goals>
<goal>idl-protocol</goal>
</goals>
</execution>
</executions>
</plugin>
</plugins>
</build>
Defining a Protocol in Avro IDL
An Avro IDL file consists of exactly one protocol definition. The minimal protocol is defined by the following code:
This is equivalent to (and generates) the following JSON protocol definition:
{
"protocol" : "MyProtocol",
"types" : [ ],
"messages" : {
}
}
The namespace of the protocol may be changed using the @namespace annotation:
@namespace("mynamespace")
protocol MyProtocol {
}
This notation is used throughout Avro IDL as a way of specifying properties for the annotated element, as will be described later in this document.
Protocols in Avro IDL can contain the following items:
- Imports of external protocol and schema files.
- Definitions of named schemata, including records, errors, enums, and fixeds.
- Definitions of RPC messages
Imports
Files may be imported in one of three formats:
-
An IDL file may be imported with a statement like:
import idl "foo.avdl";
-
A JSON protocol file may be imported with a statement like:
import protocol "foo.avpr";
-
A JSON schema file may be imported with a statement like:
import schema "foo.avsc";
Messages and types in the imported file are added to this file’s protocol.
Imported file names are resolved relative to the current IDL file.
Defining an Enumeration
Enums are defined in Avro IDL using a syntax similar to C or Java. An Avro Enum supports optional default values. In the case that a reader schema is unable to recognize a symbol written by the writer, the reader will fall back to using the defined default value. This default is only used when an incompatible symbol is read. It is not used if the enum field is missing.
Example Writer Enum Definition
enum Shapes {
SQUARE, TRIANGLE, CIRCLE, OVAL
}
Example Reader Enum Definition
enum Shapes {
SQUARE, TRIANGLE, CIRCLE
} = CIRCLE;
In the above example, the reader will use the default value of CIRCLE whenever reading data written with the OVAL symbol of the writer. Also note that, unlike the JSON format, anonymous enums cannot be defined.
Defining a Fixed Length Field
Fixed fields are defined using the following syntax:
fixed MD5(16);
This example defines a fixed-length type called MD5 which contains 16 bytes.
Defining Records and Errors
Records are defined in Avro IDL using a syntax similar to a struct definition in C:
record Employee {
string name;
boolean active = true;
long salary;
}
The above example defines a record with the name “Employee” with three fields.
To define an error, simply use the keyword error instead of record. For example:
error Kaboom {
string explanation;
int result_code = -1;
}
Each field in a record or error consists of a type and a name, optional property annotations and an optional default value.
A type reference in Avro IDL must be one of:
- A primitive type
- A logical type
- A named schema defined prior to this usage in the same Protocol
- A complex type (array, map, or union)
Primitive Types
The primitive types supported by Avro IDL are the same as those supported by Avro’s JSON format. This list includes int, long, string, boolean, float, double, null, and bytes.
Logical Types
Some of the logical types supported by Avro’s JSON format are also supported by Avro IDL. The currently supported types are:
For example:
record Job {
string jobid;
date submitDate;
time_ms submitTime;
timestamp_ms finishTime;
decimal(9,2) finishRatio;
}
References to Named Schemata
If a named schema has already been defined in the same Avro IDL file, it may be referenced by name as if it were a primitive type:
record Card {
Suit suit; // refers to the enum Card defined above
int number;
}
Default Values
Default values for fields may be optionally specified by using an equals sign after the field name followed by a JSON expression indicating the default value. This JSON is interpreted as described in the spec.
Complex Types
Arrays
Array types are written in a manner that will seem familiar to C++ or Java programmers. An array of any type t is denoted array<t>. For example, an array of strings is denoted array<string>, and a multidimensional array of Foo records would be array<array<Foo>>.
Maps
Map types are written similarly to array types. An array that contains values of type t is written map<t>. As in the JSON schema format, all maps contain string-type keys.
Unions
Union types are denoted as union { typeA, typeB, typeC, … }. For example, this record contains a string field that is optional (unioned with null):
record RecordWithUnion {
union { null, string } optionalString;
}
Note that the same restrictions apply to Avro IDL unions as apply to unions defined in the JSON format; namely, a record may not contain multiple elements of the same type.
Defining RPC Messages
The syntax to define an RPC message within a Avro IDL protocol is similar to the syntax for a method declaration within a C header file or a Java interface. To define an RPC message add which takes two arguments named foo and bar, returning an int, simply include the following definition within the protocol:
int add(int foo, int bar = 0);
Message arguments, like record fields, may specify default values.
To define a message with no response, you may use the alias void, equivalent to the Avro null type:
void logMessage(string message);
If you have previously defined an error type within the same protocol, you may declare that a message can throw this error using the syntax:
void goKaboom() throws Kaboom;
To define a one-way message, use the keyword oneway after the parameter list, for example:
void fireAndForget(string message) oneway;
Other Language Features
All Java-style comments are supported within a Avro IDL file. Any text following // on a line is ignored, as is any text between /* and */, possibly spanning multiple lines.
Comments that begin with /** are used as the documentation string for the type or field definition that follows the comment.
Escaping Identifiers
Occasionally, one will need to use a reserved language keyword as an identifier. In order to do so, backticks (`) may be used to escape the identifier. For example, to define a message with the literal name error, you may write:
This syntax is allowed anywhere an identifier is expected.
Annotations for Ordering and Namespaces
Java-style annotations may be used to add additional properties to types and fields throughout Avro IDL.
For example, to specify the sort order of a field within a record, one may use the @order annotation before the field name as follows:
record MyRecord {
string @order("ascending") myAscendingSortField;
string @order("descending") myDescendingField;
string @order("ignore") myIgnoredField;
}
A field’s type may also be preceded by annotations, e.g.:
record MyRecord {
@java-class("java.util.ArrayList") array<string> myStrings;
}
This can be used to support java classes that can be serialized/deserialized via their toString/String constructor, e.g.:
record MyRecord {
@java-class("java.math.BigDecimal") string value;
@java-key-class("java.io.File") map<string> fileStates;
array<@java-class("java.math.BigDecimal") string> weights;
}
Similarly, a @namespace annotation may be used to modify the namespace when defining a named schema. For example:
@namespace("org.apache.avro.firstNamespace")
protocol MyProto {
@namespace("org.apache.avro.someOtherNamespace")
record Foo {}
record Bar {}
}
will define a protocol in the firstNamespace namespace. The record Foo will be defined in someOtherNamespace and Bar will be defined in firstNamespace as it inherits its default from its container.
Type and field aliases are specified with the @aliases annotation as follows:
@aliases(["org.old.OldRecord", "org.ancient.AncientRecord"])
record MyRecord {
string @aliases(["oldField", "ancientField"]) myNewField;
}
Some annotations like those listed above are handled specially. All other annotations are added as properties to the protocol, message, schema or field.
Complete Example
The following is a complete example of a Avro IDL file that shows most of the above features:
/**
* An example protocol in Avro IDL
*/
@namespace("org.apache.avro.test")
protocol Simple {
@aliases(["org.foo.KindOf"])
enum Kind {
FOO,
BAR, // the bar enum value
BAZ
}
fixed MD5(16);
record TestRecord {
@order("ignore")
string name;
@order("descending")
Kind kind;
MD5 hash;
union { MD5, null} @aliases(["hash"]) nullableHash;
array<long> arrayOfLongs;
}
error TestError {
string message;
}
string hello(string greeting);
TestRecord echo(TestRecord `record`);
int add(int arg1, int arg2);
bytes echoBytes(bytes data);
void `error`() throws TestError;
void ping() oneway;
}
Additional examples may be found in the Avro source tree under the src/test/idl/input directory.
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
6.7 - SASL profile
Introduction
SASL (RFC 2222) provides a framework for authentication and security of network protocols. Each protocol that uses SASL is meant to define a SASL profile. This document provides a SASL profile for connection-based Avro RPC.
Overview
SASL negotiation proceeds as a series of message interactions over a connection between a client and server using a selected SASL mechanism. The client starts this negotiation by sending its chosen mechanism name with an initial (possibly empty) message. Negotiation proceeds with the exchange of messages until either side indicates success or failure. The content of the messages is mechanism-specific. If the negotiation succeeds, then the session can proceed over the connection, otherwise it must be abandoned.
Some mechanisms continue to process session data after negotiation (e.g., encrypting it), while some specify that further session data is transmitted unmodified.
Negotiation
Commands
Avro SASL negotiation uses four one-byte commands.
- 0: START Used in a client’s initial message.
- 1: CONTINUE Used while negotiation is ongoing.
- 2: FAIL Terminates negotiation unsuccessfully.
- 3: COMPLETE Terminates negotiation successfully.
The format of a START message is:
| 0 | 4-byte mechanism name length | mechanism name | 4-byte payload length | payload data |
The format of a CONTINUE message is:
| 1 | 4-byte payload length | payload data |
The format of a FAIL message is:
| 2 | 4-byte message length | UTF-8 message |
The format of a COMPLETE message is:
| 3 | 4-byte payload length | payload data |
Process
Negotiation is initiated by a client sending a START command containing the client’s chosen mechanism name and any mechanism-specific payload data.
The server and client then interchange some number (possibly zero) of CONTINUE messages. Each message contains payload data that is processed by the security mechanism to generate the next message.
Once either the client or server send a FAIL message then negotiation has failed. UTF-8-encoded text is included in the failure message. Once either a FAIL message has been sent or received, or any other error occurs in the negotiation, further communication on this connection must cease.
Once either the client or server send a COMPLETE message then negotiation has completed successfully. Session data may now be transmitted over the connection until it is closed by either side.
Session Data
If no SASL QOP (quality of protection) is negotiated, then all subsequent writes to/reads over this connection are written/read unmodified. In particular, messages use Avro framing, and are of the form:
| 4-byte frame length | frame data | ... | 4 zero bytes |
If a SASL QOP is negotiated, then it must be used by the connection for all subsequent messages. This is done by wrapping each non-empty frame written using the security mechanism and unwrapping each non-empty frame read. The length written in each non-empty frame is the length of the wrapped data. Complete frames must be passed to the security mechanism for unwrapping. Unwrapped data is then passed to the application as the content of the frame.
If at any point processing fails due to wrapping, unwrapping or framing errors, then all further communication on this connection must cease.
Anonymous Mechanism
The SASL anonymous mechanism (RFC 2245) is quite simple to implement. In particular, an initial anonymous request may be prefixed by the following static sequence:
| 0 | 0009 | ANONYMOUS | 0000 |
If a server uses the anonymous mechanism, it should check that the mechanism name in the start message prefixing the first request received is ‘ANONYMOUS’, then simply prefix its initial response with a COMPLETE message of:
| 3 | 0000 |
If an anonymous server recieves some other mechanism name, then it may respond with a FAIL message as simple as:
| 2 | 0000 |
Note that the anonymous mechanism need add no additional round-trip messages between client and server. The START message can be piggybacked on the initial request and the COMPLETE or FAIL message can be piggybacked on the initial response.
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
6.8 - Wiki
The Wiki page can be found here.
6.9 - FAQ
What is Avro?
Avro is a data serialization system.
Avro provides:
- Rich data structures.
- A compact, fast, binary data format.
- A container file, to store persistent data.
- Remote procedure call (RPC).
- Simple integration with dynamic languages. Code generation is not required to read or write data files nor to use or implement RPC protocols. Code generation as an optional optimization, only worth implementing for statically typed languages.
How can I get started quickly with Avro?
Check out the Quick Start Guide.
How do I statically compile a schema or protocol into generated code?
In Java
- Add the avro jar, the jackson-mapper-asl.jar and jackson-core-asl.jar to your CLASSPATH.
- Run java org.apache.avro.specific.SpecificCompiler .
This appears to be out of date, the SpecificCompiler requires two arguments, presumably an input and and output file, but it isn’t clear that this does.
Or use the Schema or Protocol Ant tasks. Avro’s build.xml provides examples of how these are used.
Lastly, you can also use the “avro-tools” jar which ships with an Avro release. Just use the “compile (schema|protocol)” command.
How are Strings represented in Java?
They use org.apache.avro.util.Utf8, not java.lang.String.
More generally, how do Avro types map to Java types?
The mappings are documented in the package javadoc for generic, specific and reflect API.
From Doug Cutting:
HDFS splits files into blocks, and mapreduce runs a map task for each block. When the task starts, it needs to be able to seek into the file to the start of the block process through the block’s end. If the file were, e.g., a gzip file, this would not be possible, since gzip files must be decompressed from the start. One cannot seek into the middle of a gzip file and start decompressing. So Hadoop’s SequenceFile places a marker periodically (~64k) in the file at record and compression boundaries, where processing can be sensibly started. Then, when a map task starts processing an HDFS block, it finds the first marker after the block’s start and continues through the first marker in the next block of the file. This requires a bit of non-local access (~0.1%). Avro’s data file uses the same method as SequenceFile.
Why isn’t every value in Avro nullable?
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 of every value stored and
corresponding computational costs to create this bit on write and
interpret it on read. These costs are wasted when values may not in
fact be null, as is the case in many datasets. In Avro such costs are
only paid when values may actually be null.
Also, allowing values to be null is a well-known source of errors. In
Avro, a value declared as non-null will always be non-null and programs
need not test for null values when processing it nor will they ever fail
for lack of such tests.
Tony Hoare calls his invention of null references his “Billion Dollar
Mistake”.
http://qconlondon.com/london-2009/presentation/Null+References:+The+Billion+Dollar+Mistake
Also note that in some programming languages not all values are permitted to be null. For example, in Java, values of type boolean, byte, short, char, int, float, long, and double may not be null.
How can I serialize directly to/from a byte array?
As pointed out in the specification, Avro data should always be stored with its schema. The Avro provided classes DataFileWriter, DataFileReader, and DataFileStream all ensure this by serializing the Schema in a container header. In some special cases, such as when implementing a new storage system or writing unit tests, you may need to write and read directly with the bare Avro serialized values. The following examples use code generated for Java from an Avro Schema, specifically the User example from the Quickstart guide.
Serializing to a byte array
This example takes a User object and returns a newly allocated byte array with the Avro serialization of that user.
ByteArrayOutputStream out = new ByteArrayOutputStream();
BinaryEncoder encoder = EncoderFactory.get().binaryEncoder(out, null);
DatumWriter<User> writer = new SpecificDatumWriter<User>(User.getClassSchema());
writer.write(user, encoder);
encoder.flush();
out.close();
byte[] serializedBytes = out.toByteArray();
Deserializing from a byte array
This example takes a byte array containing the Avro serialization of a user and returns a User object.SpecificDatumReader reader = new SpecificDatumReader(User.getClassSchema());
Decoder decoder = DecoderFactory.get().binaryDecoder(bytes, null);
User user = reader.read(null, decoder);
Note: If you are serializing or deserializing in a loop or as a method, you should be reusing objects, readers and/or writers. Check the JavaDoc to see how to reuse the objects.
7 - Apache Avro™ 1.9.1 Documentation
Introduction
Apache Avro™ is a data serialization system.
Avro provides:
- Rich data structures.
- A compact, fast, binary data format.
- A container file, to store persistent data.
- Remote procedure call (RPC).
- Simple integration with dynamic languages. Code generation is not required to read or write data files nor to use or implement RPC protocols. Code generation as an optional optimization, only worth implementing for statically typed languages.
Schemas
Avro relies on schemas. When Avro data is read, the schema used when writing it is always present. This permits each datum to be written with no per-value overheads, making serialization both fast and small. This also facilitates use with dynamic, scripting languages, since data, together with its schema, is fully self-describing.
When Avro data is stored in a file, its schema is stored with it, so that files may be processed later by any program. If the program reading the data expects a different schema this can be easily resolved, since both schemas are present.
When Avro is used in RPC, the client and server exchange schemas in the connection handshake. (This can be optimized so that, for most calls, no schemas are actually transmitted.) Since both client and server both have the other’s full schema, correspondence between same named fields, missing fields, extra fields, etc. can all be easily resolved.
Avro schemas are defined with JSON . This facilitates implementation in languages that already have JSON libraries.
Comparison with other systems
Avro provides functionality similar to systems such as Thrift, Protocol Buffers, etc. Avro differs from these systems in the following fundamental aspects.
- Dynamic typing: Avro does not require that code be generated. Data is always accompanied by a schema that permits full processing of that data without code generation, static datatypes, etc. This facilitates construction of generic data-processing systems and languages.
- Untagged data: Since the schema is present when data is read, considerably less type information need be encoded with data, resulting in smaller serialization size.
- No manually-assigned field IDs: When a schema changes, both the old and new schema are always present when processing data, so differences may be resolved symbolically, using field names.
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
7.1 - Getting Started (Java)
This is a short guide for getting started with Apache Avro™ using Java. This guide only covers using Avro for data serialization; see Patrick Hunt’s Avro RPC Quick Start for a good introduction to using Avro for RPC.
Download
Avro implementations for C, C++, C#, Java, PHP, Python, and Ruby can be downloaded from the Apache Avro™ Releases page. This guide uses Avro 1.9.1, the latest version at the time of writing. For the examples in this guide, download avro-1.9.1.jar and avro-tools-1.9.1.jar.
Alternatively, if you are using Maven, add the following dependency to your POM:
<dependency>
<groupId>org.apache.avro</groupId>
<artifactId>avro</artifactId>
<version>1.9.1</version>
</dependency>
As well as the Avro Maven plugin (for performing code generation):
<plugin>
<groupId>org.apache.avro</groupId>
<artifactId>avro-maven-plugin</artifactId>
<version>1.9.1</version>
<executions>
<execution>
<phase>generate-sources</phase>
<goals>
<goal>schema</goal>
</goals>
<configuration>
<sourceDirectory>${project.basedir}/src/main/avro/</sourceDirectory>
<outputDirectory>${project.basedir}/src/main/java/</outputDirectory>
</configuration>
</execution>
</executions>
</plugin>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-compiler-plugin</artifactId>
<configuration>
<source>1.8</source>
<target>1.8</target>
</configuration>
</plugin>
You may also build the required Avro jars from source. Building Avro is beyond the scope of this guide; see the Build Documentation page in the wiki for more information.
Defining a schema
Avro schemas are defined using JSON. Schemas are composed of primitive types (null, boolean, int, long, float, double, bytes, and string) and complex types (record, enum, array, map, union, and fixed). You can learn more about Avro schemas and types from the specification, but for now let’s start with a simple schema example, user.avsc:
{"namespace": "example.avro",
"type": "record",
"name": "User",
"fields": [
{"name": "name", "type": "string"},
{"name": "favorite_number", "type": ["int", "null"]},
{"name": "favorite_color", "type": ["string", "null"]}
]
}
This schema defines a record representing a hypothetical user. (Note that a schema file can only contain a single schema definition.) At minimum, a record definition must include its type (“type”: “record”), a name (“name”: “User”), and fields, in this case name, favorite_number, and favorite_color. We also define a namespace (“namespace”: “example.avro”), which together with the name attribute defines the “full name” of the schema (example.avro.User in this case).
Fields are defined via an array of objects, each of which defines a name and type (other attributes are optional, see the record specification for more details). The type attribute of a field is another schema object, which can be either a primitive or complex type. For example, the name field of our User schema is the primitive type string, whereas the favorite_number and favorite_color fields are both unions, represented by JSON arrays. unions are a complex type that can be any of the types listed in the array; e.g., favorite_number can either be an int or null, essentially making it an optional field.
Serializing and deserializing with code generation
Compiling the schema
Code generation allows us to automatically create classes based on our previously-defined schema. Once we have defined the relevant classes, there is no need to use the schema directly in our programs. We use the avro-tools jar to generate code as follows:
java -jar /path/to/avro-tools-1.9.1.jar compile schema <schema file> <destination>
This will generate the appropriate source files in a package based on the schema’s namespace in the provided destination folder. For instance, to generate a User class in package example.avro from the schema defined above, run
java -jar /path/to/avro-tools-1.9.1.jar compile schema user.avsc .
Note that if you using the Avro Maven plugin, there is no need to manually invoke the schema compiler; the plugin automatically performs code generation on any .avsc files present in the configured source directory.
Creating Users
Now that we’ve completed the code generation, let’s create some Users, serialize them to a data file on disk, and then read back the file and deserialize the User objects.
First let’s create some Users and set their fields.
User user1 = new User();
user1.setName("Alyssa");
user1.setFavoriteNumber(256);
// Leave favorite color null
// Alternate constructor
User user2 = new User("Ben", 7, "red");
// Construct via builder
User user3 = User.newBuilder()
.setName("Charlie")
.setFavoriteColor("blue")
.setFavoriteNumber(null)
.build();
As shown in this example, Avro objects can be created either by invoking a constructor directly or by using a builder. Unlike constructors, builders will automatically set any default values specified in the schema. Additionally, builders validate the data as it set, whereas objects constructed directly will not cause an error until the object is serialized. However, using constructors directly generally offers better performance, as builders create a copy of the datastructure before it is written.
Note that we do not set user1’s favorite color. Since that record is of type [“string”, “null”], we can either set it to a string or leave it null; it is essentially optional. Similarly, we set user3’s favorite number to null (using a builder requires setting all fields, even if they are null).
Serializing
Now let’s serialize our Users to disk.
/// Serialize user1, user2 and user3 to disk
DatumWriter<User> userDatumWriter = new SpecificDatumWriter<User>(User.class);
DataFileWriter<User> dataFileWriter = new DataFileWriter<User>(userDatumWriter);
dataFileWriter.create(user1.getSchema(), new File("users.avro"));
dataFileWriter.append(user1);
dataFileWriter.append(user2);
dataFileWriter.append(user3);
dataFileWriter.close();
We create a DatumWriter, which converts Java objects into an in-memory serialized format. The SpecificDatumWriter class is used with generated classes and extracts the schema from the specified generated type.
Next we create a DataFileWriter, which writes the serialized records, as well as the schema, to the file specified in the dataFileWriter.create call. We write our users to the file via calls to the dataFileWriter.append method. When we are done writing, we close the data file.
Deserializing
Finally, let’s deserialize the data file we just created.
// Deserialize Users from disk
DatumReader<User> userDatumReader = new SpecificDatumReader<User>(User.class);
DataFileReader<User> dataFileReader = new DataFileReader<User>(file, userDatumReader);
User user = null;
while (dataFileReader.hasNext()) {
// Reuse user object by passing it to next(). This saves us from
// allocating and garbage collecting many objects for files with
// many items.
user = dataFileReader.next(user);
System.out.println(user);
}
This snippet will output:
{"name": "Alyssa", "favorite_number": 256, "favorite_color": null}
{"name": "Ben", "favorite_number": 7, "favorite_color": "red"}
{"name": "Charlie", "favorite_number": null, "favorite_color": "blue"}
Deserializing is very similar to serializing. We create a SpecificDatumReader, analogous to the SpecificDatumWriter we used in serialization, which converts in-memory serialized items into instances of our generated class, in this case User. We pass the DatumReader and the previously created File to a DataFileReader, analogous to the DataFileWriter, which reads both the schema used by the writer as well as the data from the file on disk. The data will be read using the writer’s schema included in the file and the schema provided by the reader, in this case the User class. The writer’s schema is needed to know the order in which fields were written, while the reader’s schema is needed to know what fields are expected and how to fill in default values for fields added since the file was written. If there are differences between the two schemas, they are resolved according to the Schema Resolution specification.
Next we use the DataFileReader to iterate through the serialized Users and print the deserialized object to stdout. Note how we perform the iteration: we create a single User object which we store the current deserialized user in, and pass this record object to every call of dataFileReader.next. This is a performance optimization that allows the DataFileReader to reuse the same User object rather than allocating a new User for every iteration, which can be very expensive in terms of object allocation and garbage collection if we deserialize a large data file. While this technique is the standard way to iterate through a data file, it’s also possible to use for (User user : dataFileReader) if performance is not a concern.
Compiling and running the example code
This example code is included as a Maven project in the examples/java-example directory in the Avro docs. From this directory, execute the following commands to build and run the example:
$ mvn compile # includes code generation via Avro Maven plugin
$ mvn -q exec:java -Dexec.mainClass=example.SpecificMain
Beta feature: Generating faster code
In this release we have introduced a new approach to generating code that speeds up decoding of objects by more than 10% and encoding by more than 30% (future performance enhancements are underway). To ensure a smooth introduction of this change into production systems, this feature is controlled by a feature flag, the system property org.apache.avro.specific.use_custom_coders. In this first release, this feature is off by default. To turn it on, set the system flag to true at runtime. In the sample above, for example, you could enable the fater coders as follows:
$ mvn -q exec:java -Dexec.mainClass=example.SpecificMain \
-Dorg.apache.avro.specific.use_custom_coders=true
Note that you do not have to recompile your Avro schema to have access to this feature. The feature is compiled and built into your code, and you turn it on and off at runtime using the feature flag. As a result, you can turn it on during testing, for example, and then off in production. Or you can turn it on in production, and quickly turn it off if something breaks.
We encourage the Avro community to exercise this new feature early to help build confidence. (For those paying one-demand for compute resources in the cloud, it can lead to meaningful cost savings.) As confidence builds, we will turn this feature on by default, and eventually eliminate the feature flag (and the old code).
Serializing and deserializing without code generation
Data in Avro is always stored with its corresponding schema, meaning we can always read a serialized item regardless of whether we know the schema ahead of time. This allows us to perform serialization and deserialization without code generation.
Let’s go over the same example as in the previous section, but without using code generation: we’ll create some users, serialize them to a data file on disk, and then read back the file and deserialize the users objects.
Creating users
First, we use a Parser to read our schema definition and create a Schema object.
Schema schema = new Schema.Parser().parse(new File("user.avsc"));
Using this schema, let’s create some users.
GenericRecord user1 = new GenericData.Record(schema);
user1.put("name", "Alyssa");
user1.put("favorite_number", 256);
// Leave favorite color null
GenericRecord user2 = new GenericData.Record(schema);
user2.put("name", "Ben");
user2.put("favorite_number", 7);
user2.put("favorite_color", "red");
Since we’re not using code generation, we use GenericRecords to represent users. GenericRecord uses the schema to verify that we only specify valid fields. If we try to set a non-existent field (e.g., user1.put(“favorite_animal”, “cat”)), we’ll get an AvroRuntimeException when we run the program.
Note that we do not set user1’s favorite color. Since that record is of type [“string”, “null”], we can either set it to a string or leave it null; it is essentially optional.
Serializing
Now that we’ve created our user objects, serializing and deserializing them is almost identical to the example above which uses code generation. The main difference is that we use generic instead of specific readers and writers.
First we’ll serialize our users to a data file on disk.
// Serialize user1 and user2 to disk
File file = new File("users.avro");
DatumWriter<GenericRecord> datumWriter = new GenericDatumWriter<GenericRecord>(schema);
DataFileWriter<GenericRecord> dataFileWriter = new DataFileWriter<GenericRecord>(datumWriter);
dataFileWriter.create(schema, file);
dataFileWriter.append(user1);
dataFileWriter.append(user2);
dataFileWriter.close();
We create a DatumWriter, which converts Java objects into an in-memory serialized format. Since we are not using code generation, we create a GenericDatumWriter. It requires the schema both to determine how to write the GenericRecords and to verify that all non-nullable fields are present.
As in the code generation example, we also create a DataFileWriter, which writes the serialized records, as well as the schema, to the file specified in the dataFileWriter.create call. We write our users to the file via calls to the dataFileWriter.append method. When we are done writing, we close the data file.
Deserializing
Finally, we’ll deserialize the data file we just created.
// Deserialize users from disk
DatumReader<GenericRecord> datumReader = new GenericDatumReader<GenericRecord>(schema);
DataFileReader<GenericRecord> dataFileReader = new DataFileReader<GenericRecord>(file, datumReader);
GenericRecord user = null;
while (dataFileReader.hasNext()) {
// Reuse user object by passing it to next(). This saves us from
// allocating and garbage collecting many objects for files with
// many items.
user = dataFileReader.next(user);
System.out.println(user);
This outputs:
{"name": "Alyssa", "favorite_number": 256, "favorite_color": null}
{"name": "Ben", "favorite_number": 7, "favorite_color": "red"}
Deserializing is very similar to serializing. We create a GenericDatumReader, analogous to the GenericDatumWriter we used in serialization, which converts in-memory serialized items into GenericRecords. We pass the DatumReader and the previously created File to a DataFileReader, analogous to the DataFileWriter, which reads both the schema used by the writer as well as the data from the file on disk. The data will be read using the writer’s schema included in the file, and the reader’s schema provided to the GenericDatumReader. The writer’s schema is needed to know the order in which fields were written, while the reader’s schema is needed to know what fields are expected and how to fill in default values for fields added since the file was written. If there are differences between the two schemas, they are resolved according to the Schema Resolution specification.
Next, we use the DataFileReader to iterate through the serialized users and print the deserialized object to stdout. Note how we perform the iteration: we create a single GenericRecord object which we store the current deserialized user in, and pass this record object to every call of dataFileReader.next. This is a performance optimization that allows the DataFileReader to reuse the same record object rather than allocating a new GenericRecord for every iteration, which can be very expensive in terms of object allocation and garbage collection if we deserialize a large data file. While this technique is the standard way to iterate through a data file, it’s also possible to use for (GenericRecord user : dataFileReader) if performance is not a concern.
Compiling and running the example code
This example code is included as a Maven project in the examples/java-example directory in the Avro docs. From this directory, execute the following commands to build and run the example:
$ mvn compile
$ mvn -q exec:java -Dexec.mainClass=example.GenericMain
7.2 - Getting Started (Python)
This is a short guide for getting started with Apache Avro™ using Python. This guide only covers using Avro for data serialization; see Patrick Hunt’s Avro RPC Quick Start for a good introduction to using Avro for RPC.
Download
Avro implementations for C, C++, C#, Java, PHP, Python, and Ruby can be downloaded from the Apache Avro™ Releases page. This guide uses Avro 1.9.1, the latest version at the time of writing. Download and unzip avro-1.9.1.tar.gz, and install via python setup.py (this will probably require root privileges). Ensure that you can import avro from a Python prompt.
$ tar xvf avro-1.9.1.tar.gz
$ cd avro-1.9.1
$ sudo python setup.py install
$ python
>>> import avro # should not raise ImportError
Alternatively, you may build the Avro Python library from source. From your the root Avro directory, run the commands
$ cd lang/py/
$ ant
$ sudo python setup.py install
$ python
>>> import avro # should not raise ImportError
Defining a schema
Avro schemas are defined using JSON. Schemas are composed of primitive types (null, boolean, int, long, float, double, bytes, and string) and complex types (record, enum, array, map, union, and fixed). You can learn more about Avro schemas and types from the specification, but for now let’s start with a simple schema example, user.avsc:
{"namespace": "example.avro",
"type": "record",
"name": "User",
"fields": [
{"name": "name", "type": "string"},
{"name": "favorite_number", "type": ["int", "null"]},
{"name": "favorite_color", "type": ["string", "null"]}
]
}
This schema defines a record representing a hypothetical user. (Note that a schema file can only contain a single schema definition.) At minimum, a record definition must include its type (“type”: “record”), a name (“name”: “User”), and fields, in this case name, favorite_number, and favorite_color. We also define a namespace (“namespace”: “example.avro”), which together with the name attribute defines the “full name” of the schema (example.avro.User in this case).
Fields are defined via an array of objects, each of which defines a name and type (other attributes are optional, see the record specification for more details). The type attribute of a field is another schema object, which can be either a primitive or complex type. For example, the name field of our User schema is the primitive type string, whereas the favorite_number and favorite_color fields are both unions, represented by JSON arrays. unions are a complex type that can be any of the types listed in the array; e.g., favorite_number can either be an int or null, essentially making it an optional field.
Serializing and deserializing without code generation
Data in Avro is always stored with its corresponding schema, meaning we can always read a serialized item, regardless of whether we know the schema ahead of time. This allows us to perform serialization and deserialization without code generation. Note that the Avro Python library does not support code generation.
Try running the following code snippet, which serializes two users to a data file on disk, and then reads back and deserializes the data file:
import avro.schema
from avro.datafile import DataFileReader, DataFileWriter
from avro.io import DatumReader, DatumWriter
schema = avro.schema.parse(open("user.avsc", "rb").read())
writer = DataFileWriter(open("users.avro", "wb"), DatumWriter(), schema)
writer.append({"name": "Alyssa", "favorite_number": 256})
writer.append({"name": "Ben", "favorite_number": 7, "favorite_color": "red"})
writer.close()
reader = DataFileReader(open("users.avro", "rb"), DatumReader())
for user in reader:
print user
reader.close()
This outputs:
{u'favorite_color': None, u'favorite_number': 256, u'name': u'Alyssa'}
{u'favorite_color': u'red', u'favorite_number': 7, u'name': u'Ben'}
Do make sure that you open your files in binary mode (i.e. using the modes wb or rb respectively). Otherwise you might generate corrupt files due to automatic replacement of newline characters with the platform-specific representations.
Let’s take a closer look at what’s going on here.
schema = avro.schema.parse(open("user.avsc", "rb").read())
avro.schema.parse takes a string containing a JSON schema definition as input and outputs a avro.schema.Schema object (specifically a subclass of Schema, in this case RecordSchema). We’re passing in the contents of our user.avsc schema file here.
writer = DataFileWriter(open("users.avro", "wb"), DatumWriter(), schema)
We create a DataFileWriter, which we’ll use to write serialized items to a data file on disk. The DataFileWriter constructor takes three arguments:
- The file we’ll serialize to
- A DatumWriter, which is responsible for actually serializing the items to Avro’s binary format (DatumWriters can be used separately from DataFileWriters, e.g., to perform IPC Avro TODO: is this true??).
- The schema we’re using. The DataFileWriter needs the schema both to write the schema to the data file, and to verify that the items we write are valid items and write the appropriate fields.
writer.append({"name": "Alyssa", "favorite_number": 256})
writer.append({"name": "Ben", "favorite_number": 7, "favorite_color": "red"})
We use DataFileWriter.append to add items to our data file. Avro records are represented as Python dicts. Since the field favorite_color has type [“int”, “null”], we are not required to specify this field, as shown in the first append. Were we to omit the required name field, an exception would be raised. Any extra entries not corresponding to a field are present in the dict are ignored.
reader = DataFileReader(open("users.avro", "rb"), DatumReader())
We open the file again, this time for reading back from disk. We use a DataFileReader and DatumReader analagous to the DataFileWriter and DatumWriter above.
for user in reader:
print user
The DataFileReader is an iterator that returns dicts corresponding to the serialized items.
7.3 - Specification
Introduction
This document defines Apache Avro. It is intended to be the authoritative specification. Implementations of Avro must adhere to this document.
Schema Declaration
A Schema is represented in JSON by one of:
- A JSON string, naming a defined type.
- A JSON object, of the form:
{"type": "typeName" ...attributes...}
where typeName is either a primitive or derived type name, as defined below. Attributes not defined in this document are permitted as metadata, but must not affect the format of serialized data.
- A JSON array, representing a union of embedded types.
Primitive Types
The set of primitive type names is:
- null: no value
- boolean: a binary value
- int: 32-bit signed integer
- long: 64-bit signed integer
- float: single precision (32-bit) IEEE 754 floating-point number
- double: double precision (64-bit) IEEE 754 floating-point number
- bytes: sequence of 8-bit unsigned bytes
- string: unicode character sequence
Primitive types have no specified attributes.
Primitive type names are also defined type names. Thus, for example, the schema “string” is equivalent to:
Complex Types
Avro supports six kinds of complex types: records, enums, arrays, maps, unions and fixed.
Records
Records use the type name “record” and support three attributes:
- name: a JSON string providing the name of the record (required).
- namespace, a JSON string that qualifies the name;
- doc: a JSON string providing documentation to the user of this schema (optional).
- aliases: a JSON array of strings, providing alternate names for this record (optional).
- fields: a JSON array, listing fields (required). Each field is a JSON object with the following attributes:
- name: a JSON string providing the name of the field (required), and
- doc: a JSON string describing this field for users (optional).
- type: a schema, as defined above
- _default: A default value for this field, used when reading instances that lack this field (optional). 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. Default values for bytes and fixed fields are JSON strings, where Unicode code points 0-255 are mapped to unsigned 8-bit byte values 0-255.
field default values
| avro type |
json type |
example |
| null |
null |
null |
| boolean |
boolean |
true |
| int,long |
integer |
1 |
| float,double |
number |
1.1 |
| bytes |
string |
"\u00FF" |
| string |
string |
"foo" |
| record |
object |
{"a": 1} |
| enum |
string |
"FOO" |
| array |
array |
[1] |
| map |
object |
{"a": 1} |
| fixed |
string |
"\u00ff" |
- order: specifies how this field impacts sort ordering of this record (optional). Valid values are “ascending” (the default), “descending”, or “ignore”. For more details on how this is used, see the sort order section below.
- aliases: a JSON array of strings, providing alternate names for this field (optional).
For example, a linked-list of 64-bit values may be defined with:
{
"type": "record",
"name": "LongList",
"aliases": ["LinkedLongs"], // old name for this
"fields" : [
{"name": "value", "type": "long"}, // each element has a long
{"name": "next", "type": ["null", "LongList"]} // optional next element
]
}
Enums
Enums use the type name “enum” and support the following attributes:
- name: a JSON string providing the name of the enum (required).
- namespace, a JSON string that qualifies the name ;
- aliases: a JSON array of strings, providing alternate names for this enum (optional).
- doc: a JSON string providing documentation to the user of this schema (optional).
- symbols: a JSON array, listing symbols, as JSON strings (required). All symbols in an enum must be unique; duplicates are prohibited. Every symbol must match the regular expression [A-Za-z_][A-Za-z0-9_]* (the same requirement as for names).
- default: A default value for this enumeration, used during resolution when the reader encounters a symbol from the writer that isn’t defined in the reader’s schema (optional). The value provided here must be a JSON string that’s a member of the symbols array. See documentation on schema resolution for how this gets used.
For example, playing card suits might be defined with:
{
"type": "enum",
"name": "Suit",
"symbols" : ["SPADES", "HEARTS", "DIAMONDS", "CLUBS"]
}
Arrays
Arrays use the type name “array” and support a single attribute:
- items: the schema of the array’s items.
For example, an array of strings is declared with:
{
"type": "array",
"items" : "string"
}
Maps
Maps use the type name “map” and support one attribute:
- values: the schema of the map’s values.
Map keys are assumed to be strings.
For example, a map from string to long is declared with:
{
"type": "map",
"values" : "long"
}
Unions
Unions, as mentioned above, are represented using JSON arrays. For example, ["null", "string"] declares a schema which may be either a null or string.
(Note that when a default value is specified for a record field whose type is a union, the type of the default value must match the first element of the union. Thus, for unions containing “null”, the “null” is usually listed first, since the default value of such unions is typically null.)
Unions may not contain more than one schema with the same type, except for the named types record, fixed and enum. For example, unions containing two array types or two map types are not permitted, but two types with different names are permitted. (Names permit efficient resolution when reading and writing unions.)
Unions may not immediately contain other unions.
Fixed
Fixed uses the type name “fixed” and supports two attributes:
- name: a string naming this fixed (required).
- namespace, a string that qualifies the name;
- aliases: a JSON array of strings, providing alternate names for this enum (optional).
- size: an integer, specifying the number of bytes per value (required).
For example, 16-byte quantity may be declared with:
{"type": "fixed", "size": 16, "name": "md5"}
Names
Record, enums and fixed are named types. Each has a fullname that is composed of two parts; a name and a namespace. Equality of names is defined on the fullname.
The name portion of a fullname, record field names, and enum symbols must:
- start with [A-Za-z_]
- subsequently contain only [A-Za-z0-9_]
A namespace is a dot-separated sequence of such names. The empty string may also be used as a namespace to indicate the null namespace. Equality of names (including field names and enum symbols) as well as fullnames is case-sensitive.
In record, enum and fixed definitions, the fullname is determined in one of the following ways:
A name and namespace are both specified. For example, one might use “name”: “X”, “namespace”: “org.foo” to indicate the fullname org.foo.X.
-
A fullname is specified. If the name specified contains a dot, then it is assumed to be a fullname, and any namespace also specified is ignored. For example, use “name”: “org.foo.X” to indicate the fullname org.foo.X.
-
A name only is specified, i.e., a name that contains no dots. In this case the namespace is taken from the most tightly enclosing schema or protocol. For example, if “name”: “X” is specified, and this occurs within a field of the record definition of org.foo.Y, then the fullname is org.foo.X. If there is no enclosing namespace then the null namespace is used.
References to previously defined names are as in the latter two cases above: if they contain a dot they are a fullname, if they do not contain a dot, the namespace is the namespace of the enclosing definition.
Primitive type names have no namespace and their names may not be defined in any namespace.
A schema or protocol may not contain multiple definitions of a fullname. Further, a name must be defined before it is used (“before” in the depth-first, left-to-right traversal of the JSON parse tree, where the types attribute of a protocol is always deemed to come “before” the messages attribute.)
Aliases
Named types and fields may have aliases. An implementation may optionally use aliases to map a writer’s schema to the reader’s. This facilitates both schema evolution as well as processing disparate datasets.
Aliases function by re-writing the writer’s schema using aliases from the reader’s schema. For example, if the writer’s schema was named “Foo” and the reader’s schema is named “Bar” and has an alias of “Foo”, then the implementation would act as though “Foo” were named “Bar” when reading. Similarly, if data was written as a record with a field named “x” and is read as a record with a field named “y” with alias “x”, then the implementation would act as though “x” were named “y” when reading.
A type alias may be specified either as a fully namespace-qualified, or relative to the namespace of the name it is an alias for. For example, if a type named “a.b” has aliases of “c” and “x.y”, then the fully qualified names of its aliases are “a.c” and “x.y”.
Data Serialization and Deserialization
Binary encoded Avro data does not include type information or field names. The benefit is that the serialized data is small, but as a result a schema must always be used in order to read Avro data correctly. The best way to ensure that the schema is structurally identical to the one used to write the data is to use the exact same schema.
Therefore, files or systems that store Avro data should always include the writer’s schema for that data. Avro-based remote procedure call (RPC) systems must also guarantee that remote recipients of data have a copy of the schema used to write that data. In general, it is advisable that any reader of Avro data should use a schema that is the same (as defined more fully in Parsing Canonical Form for Schemas) as the schema that was used to write the data in order to deserialize it correctly. Deserializing data into a newer schema is accomplished by specifying an additional schema, the results of which are described in Schema Resolution.
In general, both serialization and deserialization proceed as a depth-first, left-to-right traversal of the schema, serializing or deserializing primitive types as they are encountered. Therefore, it is possible, though not advisable, to read Avro data with a schema that does not have the same Parsing Canonical Form as the schema with which the data was written. In order for this to work, the serialized primitive values must be compatible, in order value by value, with the items in the deserialization schema. For example, int and long are always serialized the same way, so an int could be deserialized as a long. Since the compatibility of two schemas depends on both the data and the serialization format (eg. binary is more permissive than JSON because JSON includes field names, eg. a long that is too large will overflow an int), it is simpler and more reliable to use schemas with identical Parsing Canonical Form.
Encodings
Avro specifies two serialization encodings: binary and JSON. Most applications will use the binary encoding, as it is smaller and faster. But, for debugging and web-based applications, the JSON encoding may sometimes be appropriate.
Binary Encoding
Binary encoding does not include field names, self-contained information about the types of individual bytes, nor field or record separators. Therefore readers are wholly reliant on the schema used when the data was encoded.
Primitive Types
Primitive types are encoded in binary as follows:
- null is written as zero bytes.
- a boolean is written as a single byte whose value is either 0 (false) or 1 (true).
- int and long values are written using variable-length zig-zag coding. Some examples:
| value |
hex |
| 0 |
00 |
| -1 |
01 |
| 1 |
02 |
| -2 |
03 |
| 2 |
04 |
| … |
|
| -64 |
7f |
| 64 |
80 01 |
| … |
|
- a float is written as 4 bytes. The float is converted into a 32-bit integer using a method equivalent to Java’s floatToIntBits and then encoded in little-endian format.
- a double is written as 8 bytes. The double is converted into a 64-bit integer using a method equivalent to Java’s doubleToLongBits and then encoded in little-endian format.
- bytes are encoded as a long followed by that many bytes of data.
- a string is encoded as a long followed by that many bytes of UTF-8 encoded character data.
For example, the three-character string “foo” would be encoded as the long value 3 (encoded as hex 06) followed by the UTF-8 encoding of ‘f’, ‘o’, and ‘o’ (the hex bytes 66 6f 6f):
06 66 6f 6f
Complex Types
Complex types are encoded in binary as follows:
Records
A record is encoded by encoding the values of its fields in the order that they are declared. In other words, a record is encoded as just the concatenation of the encodings of its fields. Field values are encoded per their schema.
For example, the record schema
{
"type": "record",
"name": "test",
"fields" : [
{"name": "a", "type": "long"},
{"name": "b", "type": "string"}
]
}
An instance of this record whose a field has value 27 (encoded as hex 36) and whose b field has value “foo” (encoded as hex bytes 06 66 6f 6f), would be encoded simply as the concatenation of these, namely the hex byte sequence:
36 06 66 6f 6f
Enums
An enum is encoded by a int, representing the zero-based position of the symbol in the schema.
For example, consider the enum:
{"type": "enum", "name": "Foo", "symbols": ["A", "B", "C", "D"] }
This would be encoded by an int between zero and three, with zero indicating “A”, and 3 indicating “D”.
Arrays
Arrays are encoded as a series of blocks. Each block consists of a long count value, followed by that many array items. A block with count zero indicates the end of the array. Each item is encoded per the array’s item schema.
If a block’s count is negative, its absolute value is used, and the count is followed immediately by a long block size indicating the number of bytes in the block. This block size permits fast skipping through data, e.g., when projecting a record to a subset of its fields.
For example, the array schema
{"type": "array", "items": "long"}
an array containing the items 3 and 27 could be encoded as the long value 2 (encoded as hex 04) followed by long values 3 and 27 (encoded as hex 06 36) terminated by zero:
04 06 36 00
The blocked representation permits one to read and write arrays larger than can be buffered in memory, since one can start writing items without knowing the full length of the array.
Maps
Maps are encoded as a series of blocks. Each block consists of a long count value, followed by that many key/value pairs. A block with count zero indicates the end of the map. Each item is encoded per the map’s value schema.
If a block’s count is negative, its absolute value is used, and the count is followed immediately by a long block size indicating the number of bytes in the block. This block size permits fast skipping through data, e.g., when projecting a record to a subset of its fields.
The blocked representation permits one to read and write maps larger than can be buffered in memory, since one can start writing items without knowing the full length of the map.
Unions
A union is encoded by first writing a long value indicating the zero-based position within the union of the schema of its value. The value is then encoded per the indicated schema within the union.
For example, the union schema ["null","string"] would encode:
-
null as zero (the index of “null” in the union):
00
-
the string “a” as one (the index of “string” in the union, 1, encoded as hex 02), followed by the serialized string:
02 02 61
NOTE: Currently for C/C++ implementtions, the positions are practically an int, but theoretically a long. In reality, we don’t expect unions with 215M members
Fixed
Fixed instances are encoded using the number of bytes declared in the schema.
JSON Encoding
Except for unions, the JSON encoding is the same as is used to encode field default values.
The value of a union is encoded in JSON as follows:
- if its type is null, then it is encoded as a JSON null;
- otherwise it is encoded as a JSON object with one name/value pair whose name is the type’s name and whose value is the recursively encoded value. For Avro’s named types (record, fixed or enum) the user-specified name is used, for other types the type name is used.
For example, the union schema ["null","string","Foo"], where Foo is a record name, would encode:
- null as null;
- the string “a” as
{"string": "a"} and
- a Foo instance as
{"Foo": {...}}, where {...} indicates the JSON encoding of a Foo instance.
Note that the original schema is still required to correctly process JSON-encoded data. For example, the JSON encoding does not distinguish between int and long, float and double, records and maps, enums and strings, etc.
Single-object encoding
In some situations a single Avro serialized object is to be stored for a longer period of time. One very common example is storing Avro records for several weeks in an Apache Kafka topic.
In the period after a schema change this persistence system will contain records that have been written with different schemas. So the need arises to know which schema was used to write a record to support schema evolution correctly. In most cases the schema itself is too large to include in the message, so this binary wrapper format supports the use case more effectively.
Single object encoding specification
Single Avro objects are encoded as follows:
- A two-byte marker,
C3 01, to show that the message is Avro and uses this single-record format (version 1).
- The 8-byte little-endian CRC-64-AVRO fingerprint of the object’s schema.
- The Avro object encoded using Avro’s binary encoding.
Implementations use the 2-byte marker to determine whether a payload is Avro. This check helps avoid expensive lookups that resolve the schema from a fingerprint, when the message is not an encoded Avro payload.
Sort Order
Avro defines a standard sort order for data. This permits data written by one system to be efficiently sorted by another system. This can be an important optimization, as sort order comparisons are sometimes the most frequent per-object operation. Note also that Avro binary-encoded data can be efficiently ordered without deserializing it to objects.
Data items may only be compared if they have identical schemas. Pairwise comparisons are implemented recursively with a depth-first, left-to-right traversal of the schema. The first mismatch encountered determines the order of the items.
Two items with the same schema are compared according to the following rules.
- null data is always equal.
- boolean data is ordered with false before true.
- int, long, float and double data is ordered by ascending numeric value.
- bytes and fixed data are compared lexicographically by unsigned 8-bit values.
- string data is compared lexicographically by Unicode code point. Note that since UTF-8 is used as the binary encoding for strings, sorting of bytes and string binary data is identical.
- array data is compared lexicographically by element.
- enum data is ordered by the symbol’s position in the enum schema. For example, an enum whose symbols are
["z", "a"] would sort “z” values before “a” values.
- union data is first ordered by the branch within the union, and, within that, by the type of the branch. For example, an
["int", "string"] union would order all int values before all string values, with the ints and strings themselves ordered as defined above.
- record data is ordered lexicographically by field. If a field specifies that its order is:
- “ascending”, then the order of its values is unaltered.
- “descending”, then the order of its values is reversed.
- “ignore”, then its values are ignored when sorting.
- map data may not be compared. It is an error to attempt to compare data containing maps unless those maps are in an
"order":"ignore" record field.
Object Container Files
Avro includes a simple object container file format. A file has a schema, and all objects stored in the file must be written according to that schema, using binary encoding. Objects are stored in blocks that may be compressed. Syncronization markers are used between blocks to permit efficient splitting of files for MapReduce processing.
Files may include arbitrary user-specified metadata.
A file consists of:
- A file header, followed by
- one or more file data blocks.
A file header consists of:
- Four bytes, ASCII ‘O’, ‘b’, ‘j’, followed by 1.
- file metadata, including the schema.
- The 16-byte, randomly-generated sync marker for this file.
File metadata is written as if defined by the following map schema:
{"type": "map", "values": "bytes"}
All metadata properties that start with “avro.” are reserved. The following file metadata properties are currently used:
- avro.schema contains the schema of objects stored in the file, as JSON data (required).
- avro.codec the name of the compression codec used to compress blocks, as a string. Implementations are required to support the following codecs: “null” and “deflate”. If codec is absent, it is assumed to be “null”. The codecs are described with more detail below.
A file header is thus described by the following schema:
{"type": "record", "name": "org.apache.avro.file.Header",
"fields" : [
{"name": "magic", "type": {"type": "fixed", "name": "Magic", "size": 4}},
{"name": "meta", "type": {"type": "map", "values": "bytes"}},
{"name": "sync", "type": {"type": "fixed", "name": "Sync", "size": 16}}
]
}
A file data block consists of:
- A long indicating the count of objects in this block.
- A long indicating the size in bytes of the serialized objects in the current block, after any codec is applied
- The serialized objects. If a codec is specified, this is compressed by that codec.
- The file’s 16-byte sync marker.
Thus, each block’s binary data can be efficiently extracted or skipped without deserializing the contents. The combination of block size, object counts, and sync markers enable detection of corrupt blocks and help ensure data integrity.
Required Codecs
null
The “null” codec simply passes through data uncompressed.
deflate
The “deflate” codec writes the data block using the deflate algorithm as specified in RFC 1951, and typically implemented using the zlib library. Note that this format (unlike the “zlib format” in RFC 1950) does not have a checksum.
Optional Codecs
bzip2
The “bzip2” codec uses the bzip2 compression library.
snappy
The “snappy” codec uses Google’s Snappy compression library. Each compressed block is followed by the 4-byte, big-endian CRC32 checksum of the uncompressed data in the block.
Protocol Declaration
Avro protocols describe RPC interfaces. Like schemas, they are defined with JSON text.
A protocol is a JSON object with the following attributes:
- protocol, a string, the name of the protocol (required);
- namespace, an optional string that qualifies the name;
- doc, an optional string describing this protocol;
- types, an optional list of definitions of named types (records, enums, fixed and errors). An error definition is just like a record definition except it uses “error” instead of “record”. Note that forward references to named types are not permitted.
- messages, an optional JSON object whose keys are message names and whose values are objects whose attributes are described below. No two messages may have the same name.
The name and namespace qualification rules defined for schema objects apply to protocols as well.
Messages
A message has attributes:
- a doc, an optional description of the message,
- a request, a list of named, typed parameter schemas (this has the same form as the fields of a record declaration);
- a response schema;
- an optional union of declared error schemas. The effective union has “string” prepended to the declared union, to permit transmission of undeclared “system” errors. For example, if the declared error union is
["AccessError"], then the effective union is ["string", "AccessError"]. When no errors are declared, the effective error union is ["string"]. Errors are serialized using the effective union; however, a protocol’s JSON declaration contains only the declared union.
- an optional one-way boolean parameter.
A request parameter list is processed equivalently to an anonymous record. Since record field lists may vary between reader and writer, request parameters may also differ between the caller and responder, and such differences are resolved in the same manner as record field differences.
The one-way parameter may only be true when the response type is "null" and no errors are listed.
Sample Protocol
For example, one may define a simple HelloWorld protocol with:
{
"namespace": "com.acme",
"protocol": "HelloWorld",
"doc": "Protocol Greetings",
"types": [
{"name": "Greeting", "type": "record", "fields": [
{"name": "message", "type": "string"}]},
{"name": "Curse", "type": "error", "fields": [
{"name": "message", "type": "string"}]}
],
"messages": {
"hello": {
"doc": "Say hello.",
"request": [{"name": "greeting", "type": "Greeting" }],
"response": "Greeting",
"errors": ["Curse"]
}
}
}
Message Transport
Messages may be transmitted via different transport mechanisms.
To the transport, a message is an opaque byte sequence.
A transport is a system that supports:
- transmission of request messages
- receipt of corresponding response messages
Servers may send a response message back to the client corresponding to a request message. The mechanism of correspondance is transport-specific. For example, in HTTP it is implicit, since HTTP directly supports requests and responses. But a transport that multiplexes many client threads over a single socket would need to tag messages with unique identifiers.
Transports may be either stateless or stateful. In a stateless transport, messaging assumes no established connection state, while stateful transports establish connections that may be used for multiple messages. This distinction is discussed further in the handshake section below.
HTTP as Transport
When HTTP is used as a transport, each Avro message exchange is an HTTP request/response pair. All messages of an Avro protocol should share a single URL at an HTTP server. Other protocols may also use that URL. Both normal and error Avro response messages should use the 200 (OK) response code. The chunked encoding may be used for requests and responses, but, regardless the Avro request and response are the entire content of an HTTP request and response. The HTTP Content-Type of requests and responses should be specified as “avro/binary”. Requests should be made using the POST method.
HTTP is used by Avro as a stateless transport.
Message Framing
Avro messages are framed as a list of buffers.
Framing is a layer between messages and the transport. It exists to optimize certain operations.
The format of framed message data is:
- a series of buffers, where each buffer consists of:
- a four-byte, big-endian buffer length, followed by
- that many bytes of buffer data.
- A message is always terminated by a zero-length buffer.
Framing is transparent to request and response message formats (described below). Any message may be presented as a single or multiple buffers.
Framing can permit readers to more efficiently get different buffers from different sources and for writers to more efficiently store different buffers to different destinations. In particular, it can reduce the number of times large binary objects are copied. For example, if an RPC parameter consists of a megabyte of file data, that data can be copied directly to a socket from a file descriptor, and, on the other end, it could be written directly to a file descriptor, never entering user space.
A simple, recommended, framing policy is for writers to create a new segment whenever a single binary object is written that is larger than a normal output buffer. Small objects are then appended in buffers, while larger objects are written as their own buffers. When a reader then tries to read a large object the runtime can hand it an entire buffer directly, without having to copy it.
Handshake
The purpose of the handshake is to ensure that the client and the server have each other’s protocol definition, so that the client can correctly deserialize responses, and the server can correctly deserialize requests. Both clients and servers should maintain a cache of recently seen protocols, so that, in most cases, a handshake will be completed without extra round-trip network exchanges or the transmission of full protocol text.
RPC requests and responses may not be processed until a handshake has been completed. With a stateless transport, all requests and responses are prefixed by handshakes. With a stateful transport, handshakes are only attached to requests and responses until a successful handshake response has been returned over a connection. After this, request and response payloads are sent without handshakes for the lifetime of that connection.
The handshake process uses the following record schemas:
{
"type": "record",
"name": "HandshakeRequest", "namespace":"org.apache.avro.ipc",
"fields": [
{"name": "clientHash",
"type": {"type": "fixed", "name": "MD5", "size": 16}},
{"name": "clientProtocol", "type": ["null", "string"]},
{"name": "serverHash", "type": "MD5"},
{"name": "meta", "type": ["null", {"type": "map", "values": "bytes"}]}
]
}
{
"type": "record",
"name": "HandshakeResponse", "namespace": "org.apache.avro.ipc",
"fields": [
{"name": "match",
"type": {"type": "enum", "name": "HandshakeMatch",
"symbols": ["BOTH", "CLIENT", "NONE"]}},
{"name": "serverProtocol",
"type": ["null", "string"]},
{"name": "serverHash",
"type": ["null", {"type": "fixed", "name": "MD5", "size": 16}]},
{"name": "meta",
"type": ["null", {"type": "map", "values": "bytes"}]}
]
}
- A client first prefixes each request with a
HandshakeRequest containing just the hash of its protocol and of the server’s protocol (clientHash!=null, clientProtocol=null, serverHash!=null), where the hashes are 128-bit MD5 hashes of the JSON protocol text. If a client has never connected to a given server, it sends its hash as a guess of the server’s hash, otherwise it sends the hash that it previously obtained from this server.
The server responds with a HandshakeResponse containing one of:
-
match=BOTH, serverProtocol=null, serverHash=null if the client sent the valid hash of the server’s protocol and the server knows what protocol corresponds to the client’s hash. In this case, the request is complete and the response data immediately follows the HandshakeResponse.
-
match=CLIENT, serverProtocol!=null, serverHash!=null if the server has previously seen the client’s protocol, but the client sent an incorrect hash of the server’s protocol. The request is complete and the response data immediately follows the HandshakeResponse. The client must use the returned protocol to process the response and should also cache that protocol and its hash for future interactions with this server.
-
match=NONE if the server has not previously seen the client’s protocol. The serverHash and serverProtocol may also be non-null if the server’s protocol hash was incorrect.
In this case the client must then re-submit its request with its protocol text (clientHash!=null, clientProtocol!=null, serverHash!=null) and the server should respond with a successful match (match=BOTH, serverProtocol=null, serverHash=null) as above.
The meta field is reserved for future handshake enhancements.
A call consists of a request message paired with its resulting response or error message. Requests and responses contain extensible metadata, and both kinds of messages are framed as described above.
The format of a call request is:
- request metadata, a map with values of type bytes
- the message name, an Avro string, followed by
- the message parameters. Parameters are serialized according to the message’s request declaration.
When the empty string is used as a message name a server should ignore the parameters and return an empty response. A client may use this to ping a server or to perform a handshake without sending a protocol message.
When a message is declared one-way and a stateful connection has been established by a successful handshake response, no response data is sent. Otherwise the format of the call response is:
- response metadata, a map with values of type bytes
- a one-byte error flag boolean, followed by either:
- if the error flag is false, the message response, serialized per the message’s response schema.
- if the error flag is true, the error, serialized per the message’s effective error union schema.
Schema Resolution
A reader of Avro data, whether from an RPC or a file, can always parse that data because the original schema must be provided along with the data. However, the reader may be programmed to read data into a different schema. For example, if the data was written with a different version of the software than it is read, then fields may have been added or removed from records. This section specifies how such schema differences should be resolved.
We refer to the schema used to write the data as the writer’s schema, and the schema that the application expects the reader’s schema. Differences between these should be resolved as follows:
- It is an error if the two schemas do not match.
To match, one of the following must hold:
-
both schemas are arrays whose item types match
-
both schemas are maps whose value types match
-
both schemas are enums whose (unqualified) names match
-
both schemas are fixed whose sizes and (unqualified) names match
-
both schemas are records with the same (unqualified) name
-
either schema is a union
-
both schemas have same primitive type
-
the writer’s schema may be promoted to the reader’s as follows:
- int is promotable to long, float, or double
- long is promotable to float or double
- float is promotable to double
- string is promotable to bytes
- bytes is promotable to string
-
if both are records:
- the ordering of fields may be different: fields are matched by name.
- schemas for fields with the same name in both records are resolved recursively.
- if the writer’s record contains a field with a name not present in the reader’s record, the writer’s value for that field is ignored.
- if the reader’s record schema has a field that contains a default value, and writer’s schema does not have a field with the same name, then the reader should use the default value from its field.
- if the reader’s record schema has a field with no default value, and writer’s schema does not have a field with the same name, an error is signalled.
-
if both are enums:
if the writer’s symbol is not present in the reader’s enum and the reader has a default value, then that value is used, otherwise an error is signalled.
This resolution algorithm is applied recursively to the reader’s and writer’s array item schemas.
This resolution algorithm is applied recursively to the reader’s and writer’s value schemas.
The first schema in the reader’s union that matches the selected writer’s union schema is recursively resolved against it. if none match, an error is signalled.
- if reader’s is a union, but writer’s is not
The first schema in the reader’s union that matches the writer’s schema is recursively resolved against it. If none match, an error is signalled.
- if writer’s is a union, but reader’s is not
If the reader’s schema matches the selected writer’s schema, it is recursively resolved against it. If they do not match, an error is signalled.
A schema’s doc fields are ignored for the purposes of schema resolution. Hence, the doc portion of a schema may be dropped at serialization.
One of the defining characteristics of Avro is that a reader must use the schema used by the writer of the data in order to know how to read the data. This assumption results in a data format that’s compact and also amenable to many forms of schema evolution. However, the specification so far has not defined what it means for the reader to have the “same” schema as the writer. Does the schema need to be textually identical? Well, clearly adding or removing some whitespace to a JSON expression does not change its meaning. At the same time, reordering the fields of records clearly does change the meaning. So what does it mean for a reader to have “the same” schema as a writer?
Parsing Canonical Form is a transformation of a writer’s schema that let’s us define what it means for two schemas to be “the same” for the purpose of reading data written against the schema. It is called Parsing Canonical Form because the transformations strip away parts of the schema, like “doc” attributes, that are irrelevant to readers trying to parse incoming data. It is called Canonical Form because the transformations normalize the JSON text (such as the order of attributes) in a way that eliminates unimportant differences between schemas. If the Parsing Canonical Forms of two different schemas are textually equal, then those schemas are “the same” as far as any reader is concerned, i.e., there is no serialized data that would allow a reader to distinguish data generated by a writer using one of the original schemas from data generated by a writing using the other original schema. (We sketch a proof of this property in a companion document.)
The next subsection specifies the transformations that define Parsing Canonical Form. But with a well-defined canonical form, it can be convenient to go one step further, transforming these canonical forms into simple integers (“fingerprints”) that can be used to uniquely identify schemas. The subsection after next recommends some standard practices for generating such fingerprints.
Assuming an input schema (in JSON form) that’s already UTF-8 text for a valid Avro schema (including all quotes as required by JSON), the following transformations will produce its Parsing Canonical Form:
- [PRIMITIVES] Convert primitive schemas to their simple form (e.g., int instead of
{"type":"int"}).
- [FULLNAMES] Replace short names with fullnames, using applicable namespaces to do so. Then eliminate namespace attributes, which are now redundant.
- [STRIP] Keep only attributes that are relevant to parsing data, which are: type, name, fields, symbols, items, values, size. Strip all others (e.g., doc and aliases).
- [ORDER] Order the appearance of fields of JSON objects as follows: name, type, fields, symbols, items, values, size. For example, if an object has type, name, and size fields, then the name field should appear first, followed by the type and then the size fields.
- [STRINGS] For all JSON string literals in the schema text, replace any escaped characters (e.g., \uXXXX escapes) with their UTF-8 equivalents.
- [INTEGERS] Eliminate quotes around and any leading zeros in front of JSON integer literals (which appear in the size attributes of fixed schemas).
- [WHITESPACE] Eliminate all whitespace in JSON outside of string literals.
Schema Fingerprints
“[A] fingerprinting algorithm is a procedure that maps an arbitrarily large data item (such as a computer file) to a much shorter bit string, its fingerprint, that uniquely identifies the original data for all practical purposes” (quoted from Wikipedia). In the Avro context, fingerprints of Parsing Canonical Form can be useful in a number of applications; for example, to cache encoder and decoder objects, to tag data items with a short substitute for the writer’s full schema, and to quickly negotiate common-case schemas between readers and writers.
In designing fingerprinting algorithms, there is a fundamental trade-off between the length of the fingerprint and the probability of collisions. To help application designers find appropriate points within this trade-off space, while encouraging interoperability and ease of implementation, we recommend using one of the following three algorithms when fingerprinting Avro schemas:
- When applications can tolerate longer fingerprints, we recommend using the SHA-256 digest algorithm to generate 256-bit fingerprints of Parsing Canonical Forms. Most languages today have SHA-256 implementations in their libraries.
- At the opposite extreme, the smallest fingerprint we recommend is a 64-bit Rabin fingerprint. Below, we provide pseudo-code for this algorithm that can be easily translated into any programming language. 64-bit fingerprints should guarantee uniqueness for schema caches of up to a million entries (for such a cache, the chance of a collision is 3E-8). We don’t recommend shorter fingerprints, as the chances of collisions is too great (for example, with 32-bit fingerprints, a cache with as few as 100,000 schemas has a 50% chance of having a collision).
- Between these two extremes, we recommend using the MD5 message digest to generate 128-bit fingerprints. These make sense only where very large numbers of schemas are being manipulated (tens of millions); otherwise, 64-bit fingerprints should be sufficient. As with SHA-256, MD5 implementations are found in most libraries today.
These fingerprints are not meant to provide any security guarantees, even the longer SHA-256-based ones. Most Avro applications should be surrounded by security measures that prevent attackers from writing random data and otherwise interfering with the consumers of schemas. We recommend that these surrounding mechanisms be used to prevent collision and pre-image attacks (i.e., “forgery”) on schema fingerprints, rather than relying on the security properties of the fingerprints themselves.
Rabin fingerprints are cyclic redundancy checks computed using irreducible polynomials. In the style of the Appendix of RFC 1952 (pg 10), which defines the CRC-32 algorithm, here’s our definition of the 64-bit AVRO fingerprinting algorithm:
long fingerprint64(byte[] buf) {
if (FP_TABLE == null) initFPTable();
long fp = EMPTY;
for (int i = 0; i < buf.length; i++)
fp = (fp >>> 8) ^ FP_TABLE[(int)(fp ^ buf[i]) & 0xff];
return fp;
}
static long EMPTY = 0xc15d213aa4d7a795L;
static long[] FP_TABLE = null;
void initFPTable() {
FP_TABLE = new long[256];
for (int i = 0; i < 256; i++) {
long fp = i;
for (int j = 0; j < 8; j++)
fp = (fp >>> 1) ^ (EMPTY & -(fp & 1L));
FP_TABLE[i] = fp;
}
}
Readers interested in the mathematics behind this algorithm may want to read this book chapter. (Unlike RFC-1952 and the book chapter, we prepend a single one bit to messages. We do this because CRCs ignore leading zero bits, which can be problematic. Our code prepends a one-bit by initializing fingerprints using EMPTY, rather than initializing using zero as in RFC-1952 and the book chapter.)
Logical Types
A logical type is an Avro primitive or complex type with extra attributes to represent a derived type. The attribute logicalType must always be present for a logical type, and is a string with the name of one of the logical types listed later in this section. Other attributes may be defined for particular logical types.
A logical type is always serialized using its underlying Avro type so that values are encoded in exactly the same way as the equivalent Avro type that does not have a logicalType attribute. Language implementations may choose to represent logical types with an appropriate native type, although this is not required.
Language implementations must ignore unknown logical types when reading, and should use the underlying Avro type. If a logical type is invalid, for example a decimal with scale greater than its precision, then implementations should ignore the logical type and use the underlying Avro type.
Decimal
The decimal logical type represents an arbitrary-precision signed decimal number of the form unscaled × 10-scale.
A decimal logical type annotates Avro bytes or fixed types. The byte array must contain the two’s-complement representation of the unscaled integer value in big-endian byte order. The scale is fixed, and is specified using an attribute.
The following attributes are supported:
- scale, a JSON integer representing the scale (optional). If not specified the scale is 0.
- precision, a JSON integer representing the (maximum) precision of decimals stored in this type (required).
For example, the following schema represents decimal numbers with a maximum precision of 4 and a scale of 2:
{
"type": "bytes",
"logicalType": "decimal",
"precision": 4,
"scale": 2
}
Precision must be a positive integer greater than zero. If the underlying type is a fixed, then the precision is limited by its size. An array of length n can store at most floor(log10(28 × n - 1 - 1)) base-10 digits of precision.
Scale must be zero or a positive integer less than or equal to the precision.
For the purposes of schema resolution, two schemas that are decimal logical types match if their scales and precisions match.
UUID
The uuid logical type represents a random generated universally unique identifier (UUID).
A uuid logical type annotates an Avro string. The string has to conform with RFC-4122
Date
The date logical type represents a date within the calendar, with no reference to a particular time zone or time of day.
A date logical type annotates an Avro int, where the int stores the number of days from the unix epoch, 1 January 1970 (ISO calendar).
Time (millisecond precision)
The time-millis logical type represents a time of day, with no reference to a particular calendar, time zone or date, with a precision of one millisecond.
A time-millis logical type annotates an Avro int, where the int stores the number of milliseconds after midnight, 00:00:00.000.
Time (microsecond precision)
The time-micros logical type represents a time of day, with no reference to a particular calendar, time zone or date, with a precision of one microsecond.
A time-micros logical type annotates an Avro long, where the long stores the number of microseconds after midnight, 00:00:00.000000.
Timestamp (millisecond precision)
The timestamp-millis logical type represents an instant on the global timeline, independent of a particular time zone or calendar, with a precision of one millisecond.
A timestamp-millis logical type annotates an Avro long, where the long stores the number of milliseconds from the unix epoch, 1 January 1970 00:00:00.000 UTC.
Timestamp (microsecond precision)
The timestamp-micros logical type represents an instant on the global timeline, independent of a particular time zone or calendar, with a precision of one microsecond.
A timestamp-micros logical type annotates an Avro long, where the long stores the number of microseconds from the unix epoch, 1 January 1970 00:00:00.000000 UTC.
Duration
The duration logical type represents an amount of time defined by a number of months, days and milliseconds. This is not equivalent to a number of milliseconds, because, depending on the moment in time from which the duration is measured, the number of days in the month and number of milliseconds in a day may differ. Other standard periods such as years, quarters, hours and minutes can be expressed through these basic periods.
A duration logical type annotates Avro fixed type of size 12, which stores three little-endian unsigned integers that represent durations at different granularities of time. The first stores a number in months, the second stores a number in days, and the third stores a number in milliseconds.
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
7.4 - Trevni
Version 0.1
DRAFT
This document is the authoritative specification of a file format. Its intent is to permit compatible, independent implementations that read and/or write files in this format.
Introduction
Data sets are often described as a table composed of rows and columns. Each record in the dataset is considered a row, with each field of the record occupying a different column. Writing records to a file one-by-one as they are created results in a row-major format, like Hadoop’s SequenceFile or Avro data files.
In many cases higher query performance may be achieved if the data is instead organized in a column-major format, where multiple values of a given column are stored adjacently. This document defines such a column-major file format for datasets.
To permit scalable, distributed query evaluation, datasets are partitioned into row groups, containing distinct collections of rows. Each row group is organized in column-major order, while row groups form a row-major partitioning of the entire dataset.
Rationale
Goals
The format is meant satisfy the following goals:
Maximize the size of row groups. Disc drives are used most efficiently when sequentially accessing data. Consider a drive that takes 10ms to seek and transfers at 100MB/second. If a 10-column dataset whose values are all the same size is split into 10MB row groups, then accessing a single column will require a sequence of seek+1MB reads, for a cost of 20ms/MB processed. If the same dataset is split into 100MB row groups then this drops to 11ms/MB processed. This effect is exaggerated for datasets with larger numbers of columns and with columns whose values are smaller than average. So we’d prefer row groups that are 100MB or greater.
Permit random access within a row group. Some queries will first examine one column, and, only when certain relatively rare criteria are met, examine other columns. Rather than iterating through selected columns of the row-group in parallel, one might iterate through one column and randomly access another. This is called support for WHERE clauses, after the SQL operator of that name.
Minimize the number of files per dataset. HDFS is a primary intended deployment platform for these files. The HDFS Namenode requires memory for each file in the filesystem, thus for a format to be HDFS-friendly it should strive to require the minimum number of distinct files.
Support co-location of columns within row-groups. Row groups are the unit of parallel operation on a column dataset. For efficient file i/o, the entirety of a row-group should ideally reside on the host that is evaluating the query in order to avoid network latencies and bottlenecks.
Data integrity. The format should permit applications to detect data corruption. Many file systems may prevent corruption, but files may be moved between filesystems and be subject to corruption at points in that process. It is best if the data in a file can be validated independently.
Extensibility. The format should permit applications to store additional annotations about a datasets in the files, such as type information, origin, etc. Some environments may have metadata stores for such information, but not all do, and files might be moved among systems with different metadata systems. The ability to keep such information within the file simplifies the coordination of such information.
Minimal overhead. The column format should not make datasets appreciably larger. Storage is a primary cost and a choice to use this format should not require additional storage.
Primary format. The column format should be usable as a primary format for datasets, not as an auxiliary, accelerated format. Applications that process a dataset in row-major order should be able to easily consume column files and applications that produce datasets in row-major order should be able to easily generate column files.
Design
To meet these goals we propose the following design.
Each row group is a separate file. All values of a column in a file are written contiguously. This maximizes the row group size, optimizing performance when querying few and small columns.
Each file occupies a single HDFS block. A larger than normal block size may be specified, e.g., ~1GB instead of the typical ~100MB. This guarantees co-location and eliminates network use when query processing can be co-located with the file. This also moderates the memory impact on the HDFS Namenode since no small files are written.
Each column in a file is written as a sequence of ~64kB compressed blocks. The sequence is prefixed by a table describing all of the blocks in the column to permit random access within the column.
Application-specific metadata may be added at the file, column, and block levels.
Checksums are included with each block, providing data integrity.
Discussion
The use of a single block per file achieves the same effect as the custom block placement policy described in the CIF paper, but while still permitting HDFS rebalancing and not increasing the number of files in the namespace.
This section formally describes the proposed column file format.
Data Model
We assume a simple data model, where a record is a set of named fields, and the value of each field is a sequence of untyped bytes. A type system may be layered on top of this, as specified in the Type Mapping section below.
Primitive Values
We define the following primitive value types:
- Signed 64-bit long values are written using a variable-length zig-zag coding, where the high-order bit in each byte determines whether subsequent bytes are present. For example:
decimal value hex bytes
| decimal value |
hex bytes |
|
| 0 |
00 |
|
| -1 |
01 |
|
| 1 |
02 |
|
| … |
|
|
| -64 |
7f |
|
| 64 |
80 01 |
|
| … |
|
|
- bytes are encoded as a long followed by that many bytes of data.
- a string is encoded as a long followed by that many bytes of UTF-8 encoded character data.
For example, the three-character string “foo” would be encoded as the long value 3 (encoded as hex 06) followed by the UTF-8 encoding of ‘f’, ‘o’, and ‘o’ (the hex bytes 66 6f 6f): 06 66 6f 6f
Type Names
The following type names are used to describe column values:
- null, requires zero bytes. Sometimes used in array columns.
- boolean, one bit, packed into bytes, little-endian;
- int, like long, but restricted to 32-bit signed values
- long 64-bit signed values, represented as above
- fixed32 32-bit values stored as four bytes, little-endian.
- fixed64 64-bit values stored as eight bytes, little-endian.
- float 32-bit IEEE floating point value, little-endian
- double 64-bit IEEE floating point value, little-endian
- string as above
- bytes as above, may be used to encapsulate more complex objects
Type names are represented as strings (UTF-8 encoded, length-prefixed).
Metadata consists of:
- A long indicating the number of metadata key/value pairs.
- For each pair, a string key and bytes value.
All metadata properties that start with “trevni.” are reserved.
The following file metadata properties are defined:
- trevni.codec the name of the default compression codec used to compress blocks, as a string. Implementations are required to support the “null” codec. Optional. If absent, it is assumed to be “null”. Codecs are described in more detail below.
- trevni.checksum the name of the checksum algorithm used in this file, as a string. Implementations are required to support the “crc-32” checksum. Optional. If absent, it is assumed to be “null”. Checksums are described in more detail below.
The following column metadata properties are defined:
- trevni.codec the name of the compression codec used to compress the blocks of this column, as a string. Implementations are required to support the “null” codec. Optional. If absent, it is assumed to be “null”. Codecs are described in more detail below.
- trevni.name the name of the column, as a string. Required.
- trevni.type the type of data in the column. One of the type names above. Required.
- trevni.values if present, indicates that the initial value of each block in this column will be stored in the block’s descriptor. Not permitted for array columns or columns that specify a parent.
- trevni.array if present, indicates that each row in this column contains a sequence of values of the named type rather than just a single value. An integer length precedes each sequence of values indicating the count of values in the sequence. If the length is negative then it indicates a sequence of zero or one lengths, where -1 indicates two zeros, -2 two ones, -3 three zeros, -4 three ones, etc.
- trevni.parent if present, the name of an array column whose lengths are also used by this column. Thus values of this column are sequences but no lengths are stored in this column.
For example, consider the following row, as JSON, where all values are primitive types, but one has multiple values.
{"id"=566, "date"=23423234234
"from"="foo@bar.com",
"to"=["bar@baz.com", "bang@foo.com"],
"content"="Hi!"}
The columns for this might be specified as:
name=id type=int
name=date type=long
name=from type=string
name=to type=string array=true
name=content type=string
If a row contains an array of records, e.g. “received” in the following:
{"id"=566, "date"=23423234234
"from"="foo@bar.com",
"to"=["bar@baz.com", "bang@foo.com"],
"content"="Hi!"
"received"=[{"date"=234234234234, "host"="192.168.0.0.1"},
{"date"=234234545645, "host"="192.168.0.0.2"}]
}
Then one can define a parent column followed by a column for each field in the record, adding the following columns:
name=received type=null array=true
name=date type=long parent=received
name=host type=string parent=received
If an array value itself contains an array, e.g. the “sigs” below:
{"id"=566, "date"=23423234234
"from"="foo@bar.com",
"to"=["bar@baz.com", "bang@foo.com"],
"content"="Hi!"
"received"=[{"date"=234234234234, "host"="192.168.0.0.1",
"sigs"=[{"algo"="weak", "value"="0af345de"}]},
{"date"=234234545645, "host"="192.168.0.0.2",
"sigs"=[]}]
}
Then a parent column may be defined that itself has a parent column.
name=sigs type=null array=true parent=received
name=algo type=string parent=sigs
name=value type=string parent=sigs
No block metadata properties are currently defined.
A file consists of:
- A file header, followed by
- one or more columns.
A file header consists of:
- Four bytes, ASCII ‘T’, ‘r’, ‘v’, followed by 0x02.
- a fixed64 indicating the number of rows in the file
- a fixed32 indicating the number of columns in the file
- file metadata.
- for each column, its column metadata
- for each column, its starting position in the file as a fixed64.
A column consists of:
- A fixed32 indicating the number of blocks in this column.
- For each block, a block descriptor
- One or more blocks.
A block descriptor consists of:
- A fixed32 indicating the number of rows in the block
- A fixed32 indicating the size in bytes of the block before the codec is applied (excluding checksum).
- A fixed32 indicating the size in bytes of the block after the codec is applied (excluding checksum).
- If this column’s metadata declares it to include values, the first value in the column, serialized according to this column’s type.
A block consists of:
- The serialized column values. If a column is an array column then value sequences are preceded by their length, as an int. If a codec is specified, the values and lengths are compressed by that codec.
- The checksum, as determined by the file metadata.
Codecs
null
The “null” codec simply passes data through uncompressed.
deflate
The “deflate” codec writes the data block using the deflate algorithm as specified in RFC 1951.
snappy
The “snappy” codec uses Google’s Snappy compression library.
Checksum algorithms
null
The “null” checksum contains zero bytes.
crc-32
Each “crc-32” checksum contains the four bytes of an ISO 3309 CRC-32 checksum of the uncompressed block data as a fixed32.
Type Mappings
We define a standard mapping for how types defined in various serialization systems are represented in a column file. Records from these systems are shredded into columns. When records are nested, a depth-first recursive walk can assign a separate column for each primitive value.
Avro
Protocol Buffers
Thrift
Implementation Notes
Some possible techniques for writing column files include:
- Use a standard ~100MB block, buffer in memory up to the block size, then flush the file directly to HDFS. A single reduce task might create multiple output files. The namenode requires memory proportional to the number of names and blocks*replication. This would increase the number of names but not blocks, so this should still be much better than a file per column.
- Spill each column to a separate local, temporary file then, when the file is closed, append these files, writing a single file to HDFS whose block size is set to be that of the entire file. This would be a bit slower than and may have trouble when the local disk is full, but it would better use HDFS namespace and further reduce seeks when processing columns whose values are small.
- Use a separate mapreduce job to convert row-major files to column-major. The map output would output a by (row#, column#, value) tuple, partitioned by row# but sorted by column# then row#. The reducer could directly write the column file. But the column file format would need to be changed to write counts, descriptors, etc. at the end of files rather than at the front.
(1) is the simplest to implement and most implementations should start with it.
References
CIF Column-Oriented Storage Techniques for MapReduce, Floratou, Patel, Shekita, & Tata, VLDB 2011.
DREMEL Dremel: Interactive Analysis of Web-Scale Datasets, Melnik, Gubarev, Long, Romer, Shivakumar, & Tolton, VLDB 2010.
7.5 - MapReduce guide
Avro provides a convenient way to represent complex data structures within a Hadoop MapReduce job. Avro data can be used as both input to and output from a MapReduce job, as well as the intermediate format. The example in this guide uses Avro data for all three, but it’s possible to mix and match; for instance, MapReduce can be used to aggregate a particular field in an Avro record.
This guide assumes basic familiarity with both Hadoop MapReduce and Avro. See the Hadoop documentation and the Avro getting started guide for introductions to these projects. This guide uses the old MapReduce API (org.apache.hadoop.mapred) and the new MapReduce API (org.apache.hadoop.mapreduce).
Setup
The code from this guide is included in the Avro docs under examples/mr-example. The example is set up as a Maven project that includes the necessary Avro and MapReduce dependencies and the Avro Maven plugin for code generation, so no external jars are needed to run the example. In particular, the POM includes the following dependencies:
<dependency>
<groupId>org.apache.avro</groupId>
<artifactId>avro</artifactId>
<version>1.9.1</version>
</dependency>
<dependency>
<groupId>org.apache.avro</groupId>
<artifactId>avro-mapred</artifactId>
<version>1.9.1</version>
</dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-client</artifactId>
<version>3.1.2</version>
</dependency>
And the following plugin:
<plugin>
<groupId>org.apache.avro</groupId>
<artifactId>avro-maven-plugin</artifactId>
<version>1.9.1</version>
<executions>
<execution>
<phase>generate-sources</phase>
<goals>
<goal>schema</goal>
</goals>
<configuration>
<sourceDirectory>${project.basedir}/../</sourceDirectory>
<outputDirectory>${project.basedir}/target/generated-sources/</outputDirectory>
</configuration>
</execution>
</executions>
</plugin>
If you do not configure the sourceDirectory and outputDirectory properties, the defaults will be used. The sourceDirectory property defaults to src/main/avro. The outputDirectory property defaults to target/generated-sources. You can change the paths to match your project layout.
Alternatively, Avro jars can be downloaded directly from the Apache Avro™ Releases page. The relevant Avro jars for this guide are avro-1.9.1.jar and avro-mapred-1.9.1.jar, as well as avro-tools-1.9.1.jar for code generation and viewing Avro data files as JSON. In addition, you will need to install Hadoop in order to use MapReduce.
Example: ColorCount
Below is a simple example of a MapReduce that uses Avro. There is an example for both the old (org.apache.hadoop.mapred) and new (org.apache.hadoop.mapreduce) APIs under examples/mr-example/src/main/java/example/. MapredColorCount is the example for the older mapred API while MapReduceColorCount is the example for the newer mapreduce API. Both examples are below, but we will detail the mapred API in our subsequent examples.
MapredColorCount:
package example;
import java.io.IOException;
import org.apache.avro.*;
import org.apache.avro.Schema.Type;
import org.apache.avro.mapred.*;
import org.apache.hadoop.conf.*;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.mapred.*;
import org.apache.hadoop.util.*;
import example.avro.User;
public class MapredColorCount extends Configured implements Tool {
public static class ColorCountMapper extends AvroMapper<User, Pair<CharSequence, Integer>> {
@Override
public void map(User user, AvroCollector<Pair<CharSequence, Integer>> collector, Reporter reporter)
throws IOException {
CharSequence color = user.getFavoriteColor();
// We need this check because the User.favorite_color field has type ["string", "null"]
if (color == null) {
color = "none";
}
collector.collect(new Pair<CharSequence, Integer>(color, 1));
}
}
public static class ColorCountReducer extends AvroReducer<CharSequence, Integer,
Pair<CharSequence, Integer>> {
@Override
public void reduce(CharSequence key, Iterable<Integer> values,
AvroCollector<Pair<CharSequence, Integer>> collector,
Reporter reporter)
throws IOException {
int sum = 0;
for (Integer value : values) {
sum += value;
}
collector.collect(new Pair<CharSequence, Integer>(key, sum));
}
}
public int run(String[] args) throws Exception {
if (args.length != 2) {
System.err.println("Usage: MapredColorCount <input path> <output path>");
return -1;
}
JobConf conf = new JobConf(getConf(), MapredColorCount.class);
conf.setJobName("colorcount");
FileInputFormat.setInputPaths(conf, new Path(args[0]));
FileOutputFormat.setOutputPath(conf, new Path(args[1]));
AvroJob.setMapperClass(conf, ColorCountMapper.class);
AvroJob.setReducerClass(conf, ColorCountReducer.class);
// Note that AvroJob.setInputSchema and AvroJob.setOutputSchema set
// relevant config options such as input/output format, map output
// classes, and output key class.
AvroJob.setInputSchema(conf, User.getClassSchema());
AvroJob.setOutputSchema(conf, Pair.getPairSchema(Schema.create(Type.STRING),
Schema.create(Type.INT)));
JobClient.runJob(conf);
return 0;
}
public static void main(String[] args) throws Exception {
int res = ToolRunner.run(new Configuration(), new MapredColorCount(), args);
System.exit(res);
}
}
MapReduceColorCount:
package example;
import java.io.IOException;
import org.apache.avro.Schema;
import org.apache.avro.mapred.AvroKey;
import org.apache.avro.mapred.AvroValue;
import org.apache.avro.mapreduce.AvroJob;
import org.apache.avro.mapreduce.AvroKeyInputFormat;
import org.apache.avro.mapreduce.AvroKeyValueOutputFormat;
import org.apache.hadoop.conf.Configured;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.IntWritable;
import org.apache.hadoop.io.NullWritable;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.mapreduce.Job;
import org.apache.hadoop.mapreduce.Mapper;
import org.apache.hadoop.mapreduce.Reducer;
import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
import org.apache.hadoop.util.Tool;
import org.apache.hadoop.util.ToolRunner;
import example.avro.User;
public class MapReduceColorCount extends Configured implements Tool {
public static class ColorCountMapper extends
Mapper<AvroKey<User>, NullWritable, Text, IntWritable> {
@Override
public void map(AvroKey<User> key, NullWritable value, Context context)
throws IOException, InterruptedException {
CharSequence color = key.datum().getFavoriteColor();
if (color == null) {
color = "none";
}
context.write(new Text(color.toString()), new IntWritable(1));
}
}
public static class ColorCountReducer extends
Reducer<Text, IntWritable, AvroKey<CharSequence>, AvroValue<Integer>> {
@Override
public void reduce(Text key, Iterable<IntWritable> values,
Context context) throws IOException, InterruptedException {
int sum = 0;
for (IntWritable value : values) {
sum += value.get();
}
context.write(new AvroKey<CharSequence>(key.toString()), new AvroValue<Integer>(sum));
}
}
public int run(String[] args) throws Exception {
if (args.length != 2) {
System.err.println("Usage: MapReduceColorCount <input path> <output path>");
return -1;
}
Job job = new Job(getConf());
job.setJarByClass(MapReduceColorCount.class);
job.setJobName("Color Count");
FileInputFormat.setInputPaths(job, new Path(args[0]));
FileOutputFormat.setOutputPath(job, new Path(args[1]));
job.setInputFormatClass(AvroKeyInputFormat.class);
job.setMapperClass(ColorCountMapper.class);
AvroJob.setInputKeySchema(job, User.getClassSchema());
job.setMapOutputKeyClass(Text.class);
job.setMapOutputValueClass(IntWritable.class);
job.setOutputFormatClass(AvroKeyValueOutputFormat.class);
job.setReducerClass(ColorCountReducer.class);
AvroJob.setOutputKeySchema(job, Schema.create(Schema.Type.STRING));
AvroJob.setOutputValueSchema(job, Schema.create(Schema.Type.INT));
return (job.waitForCompletion(true) ? 0 : 1);
}
public static void main(String[] args) throws Exception {
int res = ToolRunner.run(new MapReduceColorCount(), args);
System.exit(res);
}
}
ColorCount reads in data files containing User records, defined in examples/user.avsc, and counts the number of instances of each favorite color. (This example draws inspiration from the canonical WordCount MapReduce application.) This example uses the old MapReduce API. See MapReduceAvroWordCount, found under doc/examples/mr-example/src/main/java/example/ to see the new MapReduce API example. The User schema is defined as follows:
{"namespace": "example.avro",
"type": "record",
"name": "User",
"fields": [
{"name": "name", "type": "string"},
{"name": "favorite_number", "type": ["int", "null"]},
{"name": "favorite_color", "type": ["string", "null"]}
]
}
This schema is compiled into the User class used by ColorCount via the Avro Maven plugin (see examples/mr-example/pom.xml for how this is set up).
ColorCountMapper essentially takes a User as input and extracts the User’s favorite color, emitting the key-value pair <favoriteColor, 1>. ColorCountReducer then adds up how many occurrences of a particular favorite color were emitted, and outputs the result as a Pair record. These Pairs are serialized to an Avro data file.
Running ColorCount
The ColorCount application is provided as a Maven project in the Avro docs under examples/mr-example. To build the project, including the code generation of the User schema, run:
Next, run GenerateData from examples/mr-examples to create an Avro data file, input/users.avro, containing 20 Users with favorite colors chosen randomly from a list:
mvn exec:java -q -Dexec.mainClass=example.GenerateData
Besides creating the data file, GenerateData prints the JSON representations of the Users generated to stdout, for example:
{"name": "user", "favorite_number": null, "favorite_color": "red"}
{"name": "user", "favorite_number": null, "favorite_color": "green"}
{"name": "user", "favorite_number": null, "favorite_color": "purple"}
{"name": "user", "favorite_number": null, "favorite_color": null}
...
Now we’re ready to run ColorCount. We specify our freshly-generated input folder as the input path and output as our output folder (note that MapReduce will not start a job if the output folder already exists):
mvn exec:java -q -Dexec.mainClass=example.MapredColorCount -Dexec.args="input output"
Once ColorCount completes, checking the contents of the new output directory should yield the following:
$ ls output/
part-00000.avro _SUCCESS
You can check the contents of the generated Avro file using the avro-tools jar:
$ java -jar /path/to/avro-tools-1.9.1.jar tojson output/part-00000.avro
{"value": 3, "key": "blue"}
{"value": 7, "key": "green"}
{"value": 1, "key": "none"}
{"value": 2, "key": "orange"}
{"value": 3, "key": "purple"}
{"value": 2, "key": "red"}
{"value": 2, "key": "yellow"}
Now let’s go over the ColorCount example in detail.
Mapper - org.apache.hadoop.mapred API
The easiest way to use Avro data files as input to a MapReduce job is to subclass AvroMapper. An AvroMapper defines a map function that takes an Avro datum as input and outputs a key/value pair represented as a Pair record. In the ColorCount example, ColorCountMapper is an AvroMapper that takes a User as input and outputs a Pair<CharSequence, Integer>>, where the CharSequence key is the user’s favorite color and the Integer value is 1.
public static class ColorCountMapper extends AvroMapper<User, Pair<CharSequence, Integer>> {
@Override
public void map(User user, AvroCollector<Pair<CharSequence, Integer>> collector, Reporter reporter)
throws IOException {
CharSequence color = user.getFavoriteColor();
// We need this check because the User.favorite_color field has type ["string", "null"]
if (color == null) {
color = "none";
}
collector.collect(new Pair<CharSequence, Integer>(color, 1));
}
}
In order to use our AvroMapper, we must call AvroJob.setMapperClass and AvroJob.setInputSchema.
AvroJob.setMapperClass(conf, ColorCountMapper.class);
AvroJob.setInputSchema(conf, User.getClassSchema());
Note that AvroMapper does not implement the Mapper interface. Under the hood, the specified Avro data files are deserialized into AvroWrappers containing the actual data, which are processed by a Mapper that calls the configured AvroMapper’s map function. AvroJob.setInputSchema sets up the relevant configuration parameters needed to make this happen, thus you should not need to call JobConf.setMapperClass, JobConf.setInputFormat, JobConf.setMapOutputKeyClass, JobConf.setMapOutputValueClass, or JobConf.setOutputKeyComparatorClass.
Mapper - org.apache.hadoop.mapreduce API
This document will not go into all the differences between the mapred and mapreduce APIs, however will describe the main differences. As you can see, ColorCountMapper is now a subclass of the Hadoop Mapper class and is passed an AvroKey as it’s key. Additionally, the AvroJob method calls were slightly changed.
public static class ColorCountMapper extends
Mapper<AvroKey<User>, NullWritable, Text, IntWritable> {
@Override
public void map(AvroKey<User> key, NullWritable value, Context context)
throws IOException, InterruptedException {
CharSequence color = key.datum().getFavoriteColor();
if (color == null) {
color = "none";
}
context.write(new Text(color.toString()), new IntWritable(1));
}
}
Reducer - org.apache.hadoop.mapred API
Analogously to AvroMapper, an AvroReducer defines a reducer function that takes the key/value types output by an AvroMapper (or any mapper that outputs Pairs) and outputs a key/value pair represented a Pair record. In the ColorCount example, ColorCountReducer is an AvroReducer that takes the CharSequence key representing a favorite color and the Iterable<Integer> representing the counts for that color (they should all be 1 in this example) and adds up the counts.
public static class ColorCountReducer extends AvroReducer<CharSequence, Integer,
Pair<CharSequence, Integer>> {
@Override
public void reduce(CharSequence key, Iterable<Integer> values,
AvroCollector<Pair<CharSequence, Integer>> collector,
Reporter reporter)
throws IOException {
int sum = 0;
for (Integer value : values) {
sum += value;
}
collector.collect(new Pair<CharSequence, Integer>(key, sum));
}
}
In order to use our AvroReducer, we must call AvroJob.setReducerClass and AvroJob.setOutputSchema.
AvroJob.setReducerClass(conf, ColorCountReducer.class);
AvroJob.setOutputSchema(conf, Pair.getPairSchema(Schema.create(Type.STRING),
Schema.create(Type.INT)));
Note that AvroReducer does not implement the Reducer interface. The intermediate Pairs output by the mapper are split into AvroKeys and AvroValues, which are processed by a Reducer that calls the configured AvroReducer’s reduce function. AvroJob.setOutputSchema sets up the relevant configuration parameters needed to make this happen, thus you should not need to call JobConf.setReducerClass, JobConf.setOutputFormat, JobConf.setOutputKeyClass, JobConf.setMapOutputKeyClass, JobConf.setMapOutputValueClass, or JobConf.setOutputKeyComparatorClass.
Reduce - org.apache.hadoop.mapreduce API
As before we not detail every difference between the APIs. As with the Mapper change ColorCountReducer is now a subclass of Reducer and AvroKey and AvroValue are emitted. Additionally, the AvroJob method calls were slightly changed.
public static class ColorCountReducer extends
Reducer<Text, IntWritable, AvroKey<CharSequence>, AvroValue<Integer>> {
@Override
public void reduce(Text key, Iterable<IntWritable> values,
Context context) throws IOException, InterruptedException {
int sum = 0;
for (IntWritable value : values) {
sum += value.get();
}
context.write(new AvroKey<CharSequence>(key.toString()), new AvroValue<Integer>(sum));
}
}
Learning more
The mapred API allows users to mix Avro AvroMappers and AvroReducers with non-Avro Mappers and Reducers and the mapreduce API allows users input Avro and output non-Avro or vice versa.
The mapred package has API org.apache.avro.mapred documentation as does the org.apache.avro.mapreduce package. MapReduce API (org.apache.hadoop.mapreduce). Similarily to the mapreduce package, it’s possible with the mapred API to implement your own Mappers and Reducers directly using the public classes provided in these libraries. See the AvroWordCount application, found under examples/mr-example/src/main/java/example/AvroWordCount.java in the Avro documentation, for an example of implementing a Reducer that outputs Avro data using the old MapReduce API. See the MapReduceAvroWordCount application, found under examples/mr-example/src/main/java/example/MapReduceAvroWordCount.java in the Avro documentation, for an example of implementing a Reducer that outputs Avro data using the new MapReduce API.
7.6 - IDL Language
Introduction
This document defines Avro IDL, a higher-level language for authoring Avro schemata. Before reading this document, you should have familiarity with the concepts of schemata and protocols, as well as the various primitive and complex types available in Avro.
Overview
Purpose
The aim of the Avro IDL language is to enable developers to author schemata in a way that feels more similar to common programming languages like Java, C++, or Python. Additionally, the Avro IDL language may feel more familiar for those users who have previously used the interface description languages (IDLs) in other frameworks like Thrift, Protocol Buffers, or CORBA.
Usage
Each Avro IDL file defines a single Avro Protocol, and thus generates as its output a JSON-format Avro Protocol file with extension .avpr.
To convert a .avdl file into a .avpr file, it may be processed by the idl tool. For example:
$ java -jar avroj-tools.jar idl src/test/idl/input/namespaces.avdl /tmp/namespaces.avpr
$ head /tmp/namespaces.avpr
{
"protocol" : "TestNamespace",
"namespace" : "avro.test.protocol",
The idl tool can also process input to and from stdin and stdout. See idl --help for full usage information.
A Maven plugin is also provided to compile .avdl files. To use it, add something like the following to your pom.xml:
<build>
<plugins>
<plugin>
<groupId>org.apache.avro</groupId>
<artifactId>avro-maven-plugin</artifactId>
<executions>
<execution>
<goals>
<goal>idl-protocol</goal>
</goals>
</execution>
</executions>
</plugin>
</plugins>
</build>
Defining a Protocol in Avro IDL
An Avro IDL file consists of exactly one protocol definition. The minimal protocol is defined by the following code:
This is equivalent to (and generates) the following JSON protocol definition:
{
"protocol" : "MyProtocol",
"types" : [ ],
"messages" : {
}
}
The namespace of the protocol may be changed using the @namespace annotation:
@namespace("mynamespace")
protocol MyProtocol {
}
This notation is used throughout Avro IDL as a way of specifying properties for the annotated element, as will be described later in this document.
Protocols in Avro IDL can contain the following items:
- Imports of external protocol and schema files.
- Definitions of named schemata, including records, errors, enums, and fixeds.
- Definitions of RPC messages
Imports
Files may be imported in one of three formats:
-
An IDL file may be imported with a statement like:
import idl "foo.avdl";
-
A JSON protocol file may be imported with a statement like:
import protocol "foo.avpr";
-
A JSON schema file may be imported with a statement like:
import schema "foo.avsc";
Messages and types in the imported file are added to this file’s protocol.
Imported file names are resolved relative to the current IDL file.
Defining an Enumeration
Enums are defined in Avro IDL using a syntax similar to C or Java. An Avro Enum supports optional default values. In the case that a reader schema is unable to recognize a symbol written by the writer, the reader will fall back to using the defined default value. This default is only used when an incompatible symbol is read. It is not used if the enum field is missing.
Example Writer Enum Definition
enum Shapes {
SQUARE, TRIANGLE, CIRCLE, OVAL
}
Example Reader Enum Definition
enum Shapes {
SQUARE, TRIANGLE, CIRCLE
} = CIRCLE;
In the above example, the reader will use the default value of CIRCLE whenever reading data written with the OVAL symbol of the writer. Also note that, unlike the JSON format, anonymous enums cannot be defined.
Defining a Fixed Length Field
Fixed fields are defined using the following syntax:
fixed MD5(16);
This example defines a fixed-length type called MD5 which contains 16 bytes.
Defining Records and Errors
Records are defined in Avro IDL using a syntax similar to a struct definition in C:
record Employee {
string name;
boolean active = true;
long salary;
}
The above example defines a record with the name “Employee” with three fields.
To define an error, simply use the keyword error instead of record. For example:
error Kaboom {
string explanation;
int result_code = -1;
}
Each field in a record or error consists of a type and a name, optional property annotations and an optional default value.
A type reference in Avro IDL must be one of:
- A primitive type
- A logical type
- A named schema defined prior to this usage in the same Protocol
- A complex type (array, map, or union)
Primitive Types
The primitive types supported by Avro IDL are the same as those supported by Avro’s JSON format. This list includes int, long, string, boolean, float, double, null, and bytes.
Logical Types
Some of the logical types supported by Avro’s JSON format are also supported by Avro IDL. The currently supported types are:
For example:
record Job {
string jobid;
date submitDate;
time_ms submitTime;
timestamp_ms finishTime;
decimal(9,2) finishRatio;
}
References to Named Schemata
If a named schema has already been defined in the same Avro IDL file, it may be referenced by name as if it were a primitive type:
record Card {
Suit suit; // refers to the enum Card defined above
int number;
}
Default Values
Default values for fields may be optionally specified by using an equals sign after the field name followed by a JSON expression indicating the default value. This JSON is interpreted as described in the spec.
Complex Types
Arrays
Array types are written in a manner that will seem familiar to C++ or Java programmers. An array of any type t is denoted array<t>. For example, an array of strings is denoted array<string>, and a multidimensional array of Foo records would be array<array<Foo>>.
Maps
Map types are written similarly to array types. An array that contains values of type t is written map<t>. As in the JSON schema format, all maps contain string-type keys.
Unions
Union types are denoted as union { typeA, typeB, typeC, … }. For example, this record contains a string field that is optional (unioned with null):
record RecordWithUnion {
union { null, string } optionalString;
}
Note that the same restrictions apply to Avro IDL unions as apply to unions defined in the JSON format; namely, a record may not contain multiple elements of the same type.
Defining RPC Messages
The syntax to define an RPC message within a Avro IDL protocol is similar to the syntax for a method declaration within a C header file or a Java interface. To define an RPC message add which takes two arguments named foo and bar, returning an int, simply include the following definition within the protocol:
int add(int foo, int bar = 0);
Message arguments, like record fields, may specify default values.
To define a message with no response, you may use the alias void, equivalent to the Avro null type:
void logMessage(string message);
If you have previously defined an error type within the same protocol, you may declare that a message can throw this error using the syntax:
void goKaboom() throws Kaboom;
To define a one-way message, use the keyword oneway after the parameter list, for example:
void fireAndForget(string message) oneway;
Other Language Features
All Java-style comments are supported within a Avro IDL file. Any text following // on a line is ignored, as is any text between /* and */, possibly spanning multiple lines.
Comments that begin with /** are used as the documentation string for the type or field definition that follows the comment.
Escaping Identifiers
Occasionally, one will need to use a reserved language keyword as an identifier. In order to do so, backticks (`) may be used to escape the identifier. For example, to define a message with the literal name error, you may write:
This syntax is allowed anywhere an identifier is expected.
Annotations for Ordering and Namespaces
Java-style annotations may be used to add additional properties to types and fields throughout Avro IDL.
For example, to specify the sort order of a field within a record, one may use the @order annotation before the field name as follows:
record MyRecord {
string @order("ascending") myAscendingSortField;
string @order("descending") myDescendingField;
string @order("ignore") myIgnoredField;
}
A field’s type may also be preceded by annotations, e.g.:
record MyRecord {
@java-class("java.util.ArrayList") array<string> myStrings;
}
This can be used to support java classes that can be serialized/deserialized via their toString/String constructor, e.g.:
record MyRecord {
@java-class("java.math.BigDecimal") string value;
@java-key-class("java.io.File") map<string> fileStates;
array<@java-class("java.math.BigDecimal") string> weights;
}
Similarly, a @namespace annotation may be used to modify the namespace when defining a named schema. For example:
@namespace("org.apache.avro.firstNamespace")
protocol MyProto {
@namespace("org.apache.avro.someOtherNamespace")
record Foo {}
record Bar {}
}
will define a protocol in the firstNamespace namespace. The record Foo will be defined in someOtherNamespace and Bar will be defined in firstNamespace as it inherits its default from its container.
Type and field aliases are specified with the @aliases annotation as follows:
@aliases(["org.old.OldRecord", "org.ancient.AncientRecord"])
record MyRecord {
string @aliases(["oldField", "ancientField"]) myNewField;
}
Some annotations like those listed above are handled specially. All other annotations are added as properties to the protocol, message, schema or field.
Complete Example
The following is a complete example of a Avro IDL file that shows most of the above features:
/**
* An example protocol in Avro IDL
*/
@namespace("org.apache.avro.test")
protocol Simple {
@aliases(["org.foo.KindOf"])
enum Kind {
FOO,
BAR, // the bar enum value
BAZ
}
fixed MD5(16);
record TestRecord {
@order("ignore")
string name;
@order("descending")
Kind kind;
MD5 hash;
union { MD5, null} @aliases(["hash"]) nullableHash;
array<long> arrayOfLongs;
}
error TestError {
string message;
}
string hello(string greeting);
TestRecord echo(TestRecord `record`);
int add(int arg1, int arg2);
bytes echoBytes(bytes data);
void `error`() throws TestError;
void ping() oneway;
}
Additional examples may be found in the Avro source tree under the src/test/idl/input directory.
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
7.7 - SASL profile
Introduction
SASL (RFC 2222) provides a framework for authentication and security of network protocols. Each protocol that uses SASL is meant to define a SASL profile. This document provides a SASL profile for connection-based Avro RPC.
Overview
SASL negotiation proceeds as a series of message interactions over a connection between a client and server using a selected SASL mechanism. The client starts this negotiation by sending its chosen mechanism name with an initial (possibly empty) message. Negotiation proceeds with the exchange of messages until either side indicates success or failure. The content of the messages is mechanism-specific. If the negotiation succeeds, then the session can proceed over the connection, otherwise it must be abandoned.
Some mechanisms continue to process session data after negotiation (e.g., encrypting it), while some specify that further session data is transmitted unmodified.
Negotiation
Commands
Avro SASL negotiation uses four one-byte commands.
- 0: START Used in a client’s initial message.
- 1: CONTINUE Used while negotiation is ongoing.
- 2: FAIL Terminates negotiation unsuccessfully.
- 3: COMPLETE Terminates negotiation successfully.
The format of a START message is:
| 0 | 4-byte mechanism name length | mechanism name | 4-byte payload length | payload data |
The format of a CONTINUE message is:
| 1 | 4-byte payload length | payload data |
The format of a FAIL message is:
| 2 | 4-byte message length | UTF-8 message |
The format of a COMPLETE message is:
| 3 | 4-byte payload length | payload data |
Process
Negotiation is initiated by a client sending a START command containing the client’s chosen mechanism name and any mechanism-specific payload data.
The server and client then interchange some number (possibly zero) of CONTINUE messages. Each message contains payload data that is processed by the security mechanism to generate the next message.
Once either the client or server send a FAIL message then negotiation has failed. UTF-8-encoded text is included in the failure message. Once either a FAIL message has been sent or received, or any other error occurs in the negotiation, further communication on this connection must cease.
Once either the client or server send a COMPLETE message then negotiation has completed successfully. Session data may now be transmitted over the connection until it is closed by either side.
Session Data
If no SASL QOP (quality of protection) is negotiated, then all subsequent writes to/reads over this connection are written/read unmodified. In particular, messages use Avro framing, and are of the form:
| 4-byte frame length | frame data | ... | 4 zero bytes |
If a SASL QOP is negotiated, then it must be used by the connection for all subsequent messages. This is done by wrapping each non-empty frame written using the security mechanism and unwrapping each non-empty frame read. The length written in each non-empty frame is the length of the wrapped data. Complete frames must be passed to the security mechanism for unwrapping. Unwrapped data is then passed to the application as the content of the frame.
If at any point processing fails due to wrapping, unwrapping or framing errors, then all further communication on this connection must cease.
Anonymous Mechanism
The SASL anonymous mechanism (RFC 2245) is quite simple to implement. In particular, an initial anonymous request may be prefixed by the following static sequence:
| 0 | 0009 | ANONYMOUS | 0000 |
If a server uses the anonymous mechanism, it should check that the mechanism name in the start message prefixing the first request received is ‘ANONYMOUS’, then simply prefix its initial response with a COMPLETE message of:
| 3 | 0000 |
If an anonymous server recieves some other mechanism name, then it may respond with a FAIL message as simple as:
| 2 | 0000 |
Note that the anonymous mechanism need add no additional round-trip messages between client and server. The START message can be piggybacked on the initial request and the COMPLETE or FAIL message can be piggybacked on the initial response.
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
7.8 - Wiki
The Wiki page can be found here.
7.9 - FAQ
What is Avro?
Avro is a data serialization system.
Avro provides:
- Rich data structures.
- A compact, fast, binary data format.
- A container file, to store persistent data.
- Remote procedure call (RPC).
- Simple integration with dynamic languages. Code generation is not required to read or write data files nor to use or implement RPC protocols. Code generation as an optional optimization, only worth implementing for statically typed languages.
How can I get started quickly with Avro?
Check out the Quick Start Guide.
How do I statically compile a schema or protocol into generated code?
In Java
- Add the avro jar, the jackson-mapper-asl.jar and jackson-core-asl.jar to your CLASSPATH.
- Run java org.apache.avro.specific.SpecificCompiler .
This appears to be out of date, the SpecificCompiler requires two arguments, presumably an input and and output file, but it isn’t clear that this does.
Or use the Schema or Protocol Ant tasks. Avro’s build.xml provides examples of how these are used.
Lastly, you can also use the “avro-tools” jar which ships with an Avro release. Just use the “compile (schema|protocol)” command.
How are Strings represented in Java?
They use org.apache.avro.util.Utf8, not java.lang.String.
More generally, how do Avro types map to Java types?
The mappings are documented in the package javadoc for generic, specific and reflect API.
From Doug Cutting:
HDFS splits files into blocks, and mapreduce runs a map task for each block. When the task starts, it needs to be able to seek into the file to the start of the block process through the block’s end. If the file were, e.g., a gzip file, this would not be possible, since gzip files must be decompressed from the start. One cannot seek into the middle of a gzip file and start decompressing. So Hadoop’s SequenceFile places a marker periodically (~64k) in the file at record and compression boundaries, where processing can be sensibly started. Then, when a map task starts processing an HDFS block, it finds the first marker after the block’s start and continues through the first marker in the next block of the file. This requires a bit of non-local access (~0.1%). Avro’s data file uses the same method as SequenceFile.
Why isn’t every value in Avro nullable?
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 of every value stored and
corresponding computational costs to create this bit on write and
interpret it on read. These costs are wasted when values may not in
fact be null, as is the case in many datasets. In Avro such costs are
only paid when values may actually be null.
Also, allowing values to be null is a well-known source of errors. In
Avro, a value declared as non-null will always be non-null and programs
need not test for null values when processing it nor will they ever fail
for lack of such tests.
Tony Hoare calls his invention of null references his “Billion Dollar
Mistake”.
http://qconlondon.com/london-2009/presentation/Null+References:+The+Billion+Dollar+Mistake
Also note that in some programming languages not all values are permitted to be null. For example, in Java, values of type boolean, byte, short, char, int, float, long, and double may not be null.
How can I serialize directly to/from a byte array?
As pointed out in the specification, Avro data should always be stored with its schema. The Avro provided classes DataFileWriter, DataFileReader, and DataFileStream all ensure this by serializing the Schema in a container header. In some special cases, such as when implementing a new storage system or writing unit tests, you may need to write and read directly with the bare Avro serialized values. The following examples use code generated for Java from an Avro Schema, specifically the User example from the Quickstart guide.
Serializing to a byte array
This example takes a User object and returns a newly allocated byte array with the Avro serialization of that user.
ByteArrayOutputStream out = new ByteArrayOutputStream();
BinaryEncoder encoder = EncoderFactory.get().binaryEncoder(out, null);
DatumWriter<User> writer = new SpecificDatumWriter<User>(User.getClassSchema());
writer.write(user, encoder);
encoder.flush();
out.close();
byte[] serializedBytes = out.toByteArray();
Deserializing from a byte array
This example takes a byte array containing the Avro serialization of a user and returns a User object.SpecificDatumReader reader = new SpecificDatumReader(User.getClassSchema());
Decoder decoder = DecoderFactory.get().binaryDecoder(bytes, null);
User user = reader.read(null, decoder);
Note: If you are serializing or deserializing in a loop or as a method, you should be reusing objects, readers and/or writers. Check the JavaDoc to see how to reuse the objects.
8 - Apache Avro™ 1.9.0 Documentation
Introduction
Apache Avro™ is a data serialization system.
Avro provides:
- Rich data structures.
- A compact, fast, binary data format.
- A container file, to store persistent data.
- Remote procedure call (RPC).
- Simple integration with dynamic languages. Code generation is not required to read or write data files nor to use or implement RPC protocols. Code generation as an optional optimization, only worth implementing for statically typed languages.
Schemas
Avro relies on schemas. When Avro data is read, the schema used when writing it is always present. This permits each datum to be written with no per-value overheads, making serialization both fast and small. This also facilitates use with dynamic, scripting languages, since data, together with its schema, is fully self-describing.
When Avro data is stored in a file, its schema is stored with it, so that files may be processed later by any program. If the program reading the data expects a different schema this can be easily resolved, since both schemas are present.
When Avro is used in RPC, the client and server exchange schemas in the connection handshake. (This can be optimized so that, for most calls, no schemas are actually transmitted.) Since both client and server both have the other’s full schema, correspondence between same named fields, missing fields, extra fields, etc. can all be easily resolved.
Avro schemas are defined with JSON . This facilitates implementation in languages that already have JSON libraries.
Comparison with other systems
Avro provides functionality similar to systems such as Thrift, Protocol Buffers, etc. Avro differs from these systems in the following fundamental aspects.
- Dynamic typing: Avro does not require that code be generated. Data is always accompanied by a schema that permits full processing of that data without code generation, static datatypes, etc. This facilitates construction of generic data-processing systems and languages.
- Untagged data: Since the schema is present when data is read, considerably less type information need be encoded with data, resulting in smaller serialization size.
- No manually-assigned field IDs: When a schema changes, both the old and new schema are always present when processing data, so differences may be resolved symbolically, using field names.
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
8.1 - Getting Started (Java)
This is a short guide for getting started with Apache Avro™ using Java. This guide only covers using Avro for data serialization; see Patrick Hunt’s Avro RPC Quick Start for a good introduction to using Avro for RPC.
Download
Avro implementations for C, C++, C#, Java, PHP, Python, and Ruby can be downloaded from the Apache Avro™ Releases page. This guide uses Avro 1.9.0, the latest version at the time of writing. For the examples in this guide, download avro-1.9.0.jar and avro-tools-1.9.0.jar.
Alternatively, if you are using Maven, add the following dependency to your POM:
<dependency>
<groupId>org.apache.avro</groupId>
<artifactId>avro</artifactId>
<version>1.9.0</version>
</dependency>
As well as the Avro Maven plugin (for performing code generation):
<plugin>
<groupId>org.apache.avro</groupId>
<artifactId>avro-maven-plugin</artifactId>
<version>1.9.0</version>
<executions>
<execution>
<phase>generate-sources</phase>
<goals>
<goal>schema</goal>
</goals>
<configuration>
<sourceDirectory>${project.basedir}/src/main/avro/</sourceDirectory>
<outputDirectory>${project.basedir}/src/main/java/</outputDirectory>
</configuration>
</execution>
</executions>
</plugin>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-compiler-plugin</artifactId>
<configuration>
<source>1.8</source>
<target>1.8</target>
</configuration>
</plugin>
You may also build the required Avro jars from source. Building Avro is beyond the scope of this guide; see the Build Documentation page in the wiki for more information.
Defining a schema
Avro schemas are defined using JSON. Schemas are composed of primitive types (null, boolean, int, long, float, double, bytes, and string) and complex types (record, enum, array, map, union, and fixed). You can learn more about Avro schemas and types from the specification, but for now let’s start with a simple schema example, user.avsc:
{"namespace": "example.avro",
"type": "record",
"name": "User",
"fields": [
{"name": "name", "type": "string"},
{"name": "favorite_number", "type": ["int", "null"]},
{"name": "favorite_color", "type": ["string", "null"]}
]
}
This schema defines a record representing a hypothetical user. (Note that a schema file can only contain a single schema definition.) At minimum, a record definition must include its type (“type”: “record”), a name (“name”: “User”), and fields, in this case name, favorite_number, and favorite_color. We also define a namespace (“namespace”: “example.avro”), which together with the name attribute defines the “full name” of the schema (example.avro.User in this case).
Fields are defined via an array of objects, each of which defines a name and type (other attributes are optional, see the record specification for more details). The type attribute of a field is another schema object, which can be either a primitive or complex type. For example, the name field of our User schema is the primitive type string, whereas the favorite_number and favorite_color fields are both unions, represented by JSON arrays. unions are a complex type that can be any of the types listed in the array; e.g., favorite_number can either be an int or null, essentially making it an optional field.
Serializing and deserializing with code generation
Compiling the schema
Code generation allows us to automatically create classes based on our previously-defined schema. Once we have defined the relevant classes, there is no need to use the schema directly in our programs. We use the avro-tools jar to generate code as follows:
java -jar /path/to/avro-tools-1.9.0.jar compile schema <schema file> <destination>
This will generate the appropriate source files in a package based on the schema’s namespace in the provided destination folder. For instance, to generate a User class in package example.avro from the schema defined above, run
java -jar /path/to/avro-tools-1.9.0.jar compile schema user.avsc .
Note that if you using the Avro Maven plugin, there is no need to manually invoke the schema compiler; the plugin automatically performs code generation on any .avsc files present in the configured source directory.
Creating Users
Now that we’ve completed the code generation, let’s create some Users, serialize them to a data file on disk, and then read back the file and deserialize the User objects.
First let’s create some Users and set their fields.
User user1 = new User();
user1.setName("Alyssa");
user1.setFavoriteNumber(256);
// Leave favorite color null
// Alternate constructor
User user2 = new User("Ben", 7, "red");
// Construct via builder
User user3 = User.newBuilder()
.setName("Charlie")
.setFavoriteColor("blue")
.setFavoriteNumber(null)
.build();
As shown in this example, Avro objects can be created either by invoking a constructor directly or by using a builder. Unlike constructors, builders will automatically set any default values specified in the schema. Additionally, builders validate the data as it set, whereas objects constructed directly will not cause an error until the object is serialized. However, using constructors directly generally offers better performance, as builders create a copy of the datastructure before it is written.
Note that we do not set user1’s favorite color. Since that record is of type [“string”, “null”], we can either set it to a string or leave it null; it is essentially optional. Similarly, we set user3’s favorite number to null (using a builder requires setting all fields, even if they are null).
Serializing
Now let’s serialize our Users to disk.
/// Serialize user1, user2 and user3 to disk
DatumWriter<User> userDatumWriter = new SpecificDatumWriter<User>(User.class);
DataFileWriter<User> dataFileWriter = new DataFileWriter<User>(userDatumWriter);
dataFileWriter.create(user1.getSchema(), new File("users.avro"));
dataFileWriter.append(user1);
dataFileWriter.append(user2);
dataFileWriter.append(user3);
dataFileWriter.close();
We create a DatumWriter, which converts Java objects into an in-memory serialized format. The SpecificDatumWriter class is used with generated classes and extracts the schema from the specified generated type.
Next we create a DataFileWriter, which writes the serialized records, as well as the schema, to the file specified in the dataFileWriter.create call. We write our users to the file via calls to the dataFileWriter.append method. When we are done writing, we close the data file.
Deserializing
Finally, let’s deserialize the data file we just created.
// Deserialize Users from disk
DatumReader<User> userDatumReader = new SpecificDatumReader<User>(User.class);
DataFileReader<User> dataFileReader = new DataFileReader<User>(file, userDatumReader);
User user = null;
while (dataFileReader.hasNext()) {
// Reuse user object by passing it to next(). This saves us from
// allocating and garbage collecting many objects for files with
// many items.
user = dataFileReader.next(user);
System.out.println(user);
}
This snippet will output:
{"name": "Alyssa", "favorite_number": 256, "favorite_color": null}
{"name": "Ben", "favorite_number": 7, "favorite_color": "red"}
{"name": "Charlie", "favorite_number": null, "favorite_color": "blue"}
Deserializing is very similar to serializing. We create a SpecificDatumReader, analogous to the SpecificDatumWriter we used in serialization, which converts in-memory serialized items into instances of our generated class, in this case User. We pass the DatumReader and the previously created File to a DataFileReader, analogous to the DataFileWriter, which reads both the schema used by the writer as well as the data from the file on disk. The data will be read using the writer’s schema included in the file and the schema provided by the reader, in this case the User class. The writer’s schema is needed to know the order in which fields were written, while the reader’s schema is needed to know what fields are expected and how to fill in default values for fields added since the file was written. If there are differences between the two schemas, they are resolved according to the Schema Resolution specification.
Next we use the DataFileReader to iterate through the serialized Users and print the deserialized object to stdout. Note how we perform the iteration: we create a single User object which we store the current deserialized user in, and pass this record object to every call of dataFileReader.next. This is a performance optimization that allows the DataFileReader to reuse the same User object rather than allocating a new User for every iteration, which can be very expensive in terms of object allocation and garbage collection if we deserialize a large data file. While this technique is the standard way to iterate through a data file, it’s also possible to use for (User user : dataFileReader) if performance is not a concern.
Compiling and running the example code
This example code is included as a Maven project in the examples/java-example directory in the Avro docs. From this directory, execute the following commands to build and run the example:
$ mvn compile # includes code generation via Avro Maven plugin
$ mvn -q exec:java -Dexec.mainClass=example.SpecificMain
Beta feature: Generating faster code
In this release we have introduced a new approach to generating code that speeds up decoding of objects by more than 10% and encoding by more than 30% (future performance enhancements are underway). To ensure a smooth introduction of this change into production systems, this feature is controlled by a feature flag, the system property org.apache.avro.specific.use_custom_coders. In this first release, this feature is off by default. To turn it on, set the system flag to true at runtime. In the sample above, for example, you could enable the fater coders as follows:
$ mvn -q exec:java -Dexec.mainClass=example.SpecificMain \
-Dorg.apache.avro.specific.use_custom_coders=true
Note that you do not have to recompile your Avro schema to have access to this feature. The feature is compiled and built into your code, and you turn it on and off at runtime using the feature flag. As a result, you can turn it on during testing, for example, and then off in production. Or you can turn it on in production, and quickly turn it off if something breaks.
We encourage the Avro community to exercise this new feature early to help build confidence. (For those paying one-demand for compute resources in the cloud, it can lead to meaningful cost savings.) As confidence builds, we will turn this feature on by default, and eventually eliminate the feature flag (and the old code).
Serializing and deserializing without code generation
Data in Avro is always stored with its corresponding schema, meaning we can always read a serialized item regardless of whether we know the schema ahead of time. This allows us to perform serialization and deserialization without code generation.
Let’s go over the same example as in the previous section, but without using code generation: we’ll create some users, serialize them to a data file on disk, and then read back the file and deserialize the users objects.
Creating users
First, we use a Parser to read our schema definition and create a Schema object.
Schema schema = new Schema.Parser().parse(new File("user.avsc"));
Using this schema, let’s create some users.
GenericRecord user1 = new GenericData.Record(schema);
user1.put("name", "Alyssa");
user1.put("favorite_number", 256);
// Leave favorite color null
GenericRecord user2 = new GenericData.Record(schema);
user2.put("name", "Ben");
user2.put("favorite_number", 7);
user2.put("favorite_color", "red");
Since we’re not using code generation, we use GenericRecords to represent users. GenericRecord uses the schema to verify that we only specify valid fields. If we try to set a non-existent field (e.g., user1.put(“favorite_animal”, “cat”)), we’ll get an AvroRuntimeException when we run the program.
Note that we do not set user1’s favorite color. Since that record is of type [“string”, “null”], we can either set it to a string or leave it null; it is essentially optional.
Serializing
Now that we’ve created our user objects, serializing and deserializing them is almost identical to the example above which uses code generation. The main difference is that we use generic instead of specific readers and writers.
First we’ll serialize our users to a data file on disk.
// Serialize user1 and user2 to disk
File file = new File("users.avro");
DatumWriter<GenericRecord> datumWriter = new GenericDatumWriter<GenericRecord>(schema);
DataFileWriter<GenericRecord> dataFileWriter = new DataFileWriter<GenericRecord>(datumWriter);
dataFileWriter.create(schema, file);
dataFileWriter.append(user1);
dataFileWriter.append(user2);
dataFileWriter.close();
We create a DatumWriter, which converts Java objects into an in-memory serialized format. Since we are not using code generation, we create a GenericDatumWriter. It requires the schema both to determine how to write the GenericRecords and to verify that all non-nullable fields are present.
As in the code generation example, we also create a DataFileWriter, which writes the serialized records, as well as the schema, to the file specified in the dataFileWriter.create call. We write our users to the file via calls to the dataFileWriter.append method. When we are done writing, we close the data file.
Deserializing
Finally, we’ll deserialize the data file we just created.
// Deserialize users from disk
DatumReader<GenericRecord> datumReader = new GenericDatumReader<GenericRecord>(schema);
DataFileReader<GenericRecord> dataFileReader = new DataFileReader<GenericRecord>(file, datumReader);
GenericRecord user = null;
while (dataFileReader.hasNext()) {
// Reuse user object by passing it to next(). This saves us from
// allocating and garbage collecting many objects for files with
// many items.
user = dataFileReader.next(user);
System.out.println(user);
This outputs:
{"name": "Alyssa", "favorite_number": 256, "favorite_color": null}
{"name": "Ben", "favorite_number": 7, "favorite_color": "red"}
Deserializing is very similar to serializing. We create a GenericDatumReader, analogous to the GenericDatumWriter we used in serialization, which converts in-memory serialized items into GenericRecords. We pass the DatumReader and the previously created File to a DataFileReader, analogous to the DataFileWriter, which reads both the schema used by the writer as well as the data from the file on disk. The data will be read using the writer’s schema included in the file, and the reader’s schema provided to the GenericDatumReader. The writer’s schema is needed to know the order in which fields were written, while the reader’s schema is needed to know what fields are expected and how to fill in default values for fields added since the file was written. If there are differences between the two schemas, they are resolved according to the Schema Resolution specification.
Next, we use the DataFileReader to iterate through the serialized users and print the deserialized object to stdout. Note how we perform the iteration: we create a single GenericRecord object which we store the current deserialized user in, and pass this record object to every call of dataFileReader.next. This is a performance optimization that allows the DataFileReader to reuse the same record object rather than allocating a new GenericRecord for every iteration, which can be very expensive in terms of object allocation and garbage collection if we deserialize a large data file. While this technique is the standard way to iterate through a data file, it’s also possible to use for (GenericRecord user : dataFileReader) if performance is not a concern.
Compiling and running the example code
This example code is included as a Maven project in the examples/java-example directory in the Avro docs. From this directory, execute the following commands to build and run the example:
$ mvn compile
$ mvn -q exec:java -Dexec.mainClass=example.GenericMain
8.2 - Getting Started (Python)
This is a short guide for getting started with Apache Avro™ using Python. This guide only covers using Avro for data serialization; see Patrick Hunt’s Avro RPC Quick Start for a good introduction to using Avro for RPC.
Download
Avro implementations for C, C++, C#, Java, PHP, Python, and Ruby can be downloaded from the Apache Avro™ Releases page. This guide uses Avro 1.9.0, the latest version at the time of writing. Download and unzip avro-1.9.0.tar.gz, and install via python setup.py (this will probably require root privileges). Ensure that you can import avro from a Python prompt.
$ tar xvf avro-1.9.0.tar.gz
$ cd avro-1.9.0
$ sudo python setup.py install
$ python
>>> import avro # should not raise ImportError
Alternatively, you may build the Avro Python library from source. From your the root Avro directory, run the commands
$ cd lang/py/
$ ant
$ sudo python setup.py install
$ python
>>> import avro # should not raise ImportError
Defining a schema
Avro schemas are defined using JSON. Schemas are composed of primitive types (null, boolean, int, long, float, double, bytes, and string) and complex types (record, enum, array, map, union, and fixed). You can learn more about Avro schemas and types from the specification, but for now let’s start with a simple schema example, user.avsc:
{"namespace": "example.avro",
"type": "record",
"name": "User",
"fields": [
{"name": "name", "type": "string"},
{"name": "favorite_number", "type": ["int", "null"]},
{"name": "favorite_color", "type": ["string", "null"]}
]
}
This schema defines a record representing a hypothetical user. (Note that a schema file can only contain a single schema definition.) At minimum, a record definition must include its type (“type”: “record”), a name (“name”: “User”), and fields, in this case name, favorite_number, and favorite_color. We also define a namespace (“namespace”: “example.avro”), which together with the name attribute defines the “full name” of the schema (example.avro.User in this case).
Fields are defined via an array of objects, each of which defines a name and type (other attributes are optional, see the record specification for more details). The type attribute of a field is another schema object, which can be either a primitive or complex type. For example, the name field of our User schema is the primitive type string, whereas the favorite_number and favorite_color fields are both unions, represented by JSON arrays. unions are a complex type that can be any of the types listed in the array; e.g., favorite_number can either be an int or null, essentially making it an optional field.
Serializing and deserializing without code generation
Data in Avro is always stored with its corresponding schema, meaning we can always read a serialized item, regardless of whether we know the schema ahead of time. This allows us to perform serialization and deserialization without code generation. Note that the Avro Python library does not support code generation.
Try running the following code snippet, which serializes two users to a data file on disk, and then reads back and deserializes the data file:
import avro.schema
from avro.datafile import DataFileReader, DataFileWriter
from avro.io import DatumReader, DatumWriter
schema = avro.schema.parse(open("user.avsc", "rb").read())
writer = DataFileWriter(open("users.avro", "wb"), DatumWriter(), schema)
writer.append({"name": "Alyssa", "favorite_number": 256})
writer.append({"name": "Ben", "favorite_number": 7, "favorite_color": "red"})
writer.close()
reader = DataFileReader(open("users.avro", "rb"), DatumReader())
for user in reader:
print user
reader.close()
This outputs:
{u'favorite_color': None, u'favorite_number': 256, u'name': u'Alyssa'}
{u'favorite_color': u'red', u'favorite_number': 7, u'name': u'Ben'}
Do make sure that you open your files in binary mode (i.e. using the modes wb or rb respectively). Otherwise you might generate corrupt files due to automatic replacement of newline characters with the platform-specific representations.
Let’s take a closer look at what’s going on here.
schema = avro.schema.parse(open("user.avsc", "rb").read())
avro.schema.parse takes a string containing a JSON schema definition as input and outputs a avro.schema.Schema object (specifically a subclass of Schema, in this case RecordSchema). We’re passing in the contents of our user.avsc schema file here.
writer = DataFileWriter(open("users.avro", "wb"), DatumWriter(), schema)
We create a DataFileWriter, which we’ll use to write serialized items to a data file on disk. The DataFileWriter constructor takes three arguments:
- The file we’ll serialize to
- A DatumWriter, which is responsible for actually serializing the items to Avro’s binary format (DatumWriters can be used separately from DataFileWriters, e.g., to perform IPC Avro TODO: is this true??).
- The schema we’re using. The DataFileWriter needs the schema both to write the schema to the data file, and to verify that the items we write are valid items and write the appropriate fields.
writer.append({"name": "Alyssa", "favorite_number": 256})
writer.append({"name": "Ben", "favorite_number": 7, "favorite_color": "red"})
We use DataFileWriter.append to add items to our data file. Avro records are represented as Python dicts. Since the field favorite_color has type [“int”, “null”], we are not required to specify this field, as shown in the first append. Were we to omit the required name field, an exception would be raised. Any extra entries not corresponding to a field are present in the dict are ignored.
reader = DataFileReader(open("users.avro", "rb"), DatumReader())
We open the file again, this time for reading back from disk. We use a DataFileReader and DatumReader analagous to the DataFileWriter and DatumWriter above.
for user in reader:
print user
The DataFileReader is an iterator that returns dicts corresponding to the serialized items.
8.3 - Specification
Introduction
This document defines Apache Avro. It is intended to be the authoritative specification. Implementations of Avro must adhere to this document.
Schema Declaration
A Schema is represented in JSON by one of:
- A JSON string, naming a defined type.
- A JSON object, of the form:
{"type": "typeName" ...attributes...}
where typeName is either a primitive or derived type name, as defined below. Attributes not defined in this document are permitted as metadata, but must not affect the format of serialized data.
- A JSON array, representing a union of embedded types.
Primitive Types
The set of primitive type names is:
- null: no value
- boolean: a binary value
- int: 32-bit signed integer
- long: 64-bit signed integer
- float: single precision (32-bit) IEEE 754 floating-point number
- double: double precision (64-bit) IEEE 754 floating-point number
- bytes: sequence of 8-bit unsigned bytes
- string: unicode character sequence
Primitive types have no specified attributes.
Primitive type names are also defined type names. Thus, for example, the schema “string” is equivalent to:
Complex Types
Avro supports six kinds of complex types: records, enums, arrays, maps, unions and fixed.
Records
Records use the type name “record” and support three attributes:
- name: a JSON string providing the name of the record (required).
- namespace, a JSON string that qualifies the name;
- doc: a JSON string providing documentation to the user of this schema (optional).
- aliases: a JSON array of strings, providing alternate names for this record (optional).
- fields: a JSON array, listing fields (required). Each field is a JSON object with the following attributes:
- name: a JSON string providing the name of the field (required), and
- doc: a JSON string describing this field for users (optional).
- type: a schema, as defined above
- _default: A default value for this field, used when reading instances that lack this field (optional). 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. Default values for bytes and fixed fields are JSON strings, where Unicode code points 0-255 are mapped to unsigned 8-bit byte values 0-255.
field default values
| avro type |
json type |
example |
| null |
null |
null |
| boolean |
boolean |
true |
| int,long |
integer |
1 |
| float,double |
number |
1.1 |
| bytes |
string |
"\u00FF" |
| string |
string |
"foo" |
| record |
object |
{"a": 1} |
| enum |
string |
"FOO" |
| array |
array |
[1] |
| map |
object |
{"a": 1} |
| fixed |
string |
"\u00ff" |
- order: specifies how this field impacts sort ordering of this record (optional). Valid values are “ascending” (the default), “descending”, or “ignore”. For more details on how this is used, see the sort order section below.
- aliases: a JSON array of strings, providing alternate names for this field (optional).
For example, a linked-list of 64-bit values may be defined with:
{
"type": "record",
"name": "LongList",
"aliases": ["LinkedLongs"], // old name for this
"fields" : [
{"name": "value", "type": "long"}, // each element has a long
{"name": "next", "type": ["null", "LongList"]} // optional next element
]
}
Enums
Enums use the type name “enum” and support the following attributes:
- name: a JSON string providing the name of the enum (required).
- namespace, a JSON string that qualifies the name ;
- aliases: a JSON array of strings, providing alternate names for this enum (optional).
- doc: a JSON string providing documentation to the user of this schema (optional).
- symbols: a JSON array, listing symbols, as JSON strings (required). All symbols in an enum must be unique; duplicates are prohibited. Every symbol must match the regular expression [A-Za-z_][A-Za-z0-9_]* (the same requirement as for names).
- default: A default value for this enumeration, used during resolution when the reader encounters a symbol from the writer that isn’t defined in the reader’s schema (optional). The value provided here must be a JSON string that’s a member of the symbols array. See documentation on schema resolution for how this gets used.
For example, playing card suits might be defined with:
{
"type": "enum",
"name": "Suit",
"symbols" : ["SPADES", "HEARTS", "DIAMONDS", "CLUBS"]
}
Arrays
Arrays use the type name “array” and support a single attribute:
- items: the schema of the array’s items.
For example, an array of strings is declared with:
{
"type": "array",
"items" : "string"
}
Maps
Maps use the type name “map” and support one attribute:
- values: the schema of the map’s values.
Map keys are assumed to be strings.
For example, a map from string to long is declared with:
{
"type": "map",
"values" : "long"
}
Unions
Unions, as mentioned above, are represented using JSON arrays. For example, ["null", "string"] declares a schema which may be either a null or string.
(Note that when a default value is specified for a record field whose type is a union, the type of the default value must match the first element of the union. Thus, for unions containing “null”, the “null” is usually listed first, since the default value of such unions is typically null.)
Unions may not contain more than one schema with the same type, except for the named types record, fixed and enum. For example, unions containing two array types or two map types are not permitted, but two types with different names are permitted. (Names permit efficient resolution when reading and writing unions.)
Unions may not immediately contain other unions.
Fixed
Fixed uses the type name “fixed” and supports two attributes:
- name: a string naming this fixed (required).
- namespace, a string that qualifies the name;
- aliases: a JSON array of strings, providing alternate names for this enum (optional).
- size: an integer, specifying the number of bytes per value (required).
For example, 16-byte quantity may be declared with:
{"type": "fixed", "size": 16, "name": "md5"}
Names
Record, enums and fixed are named types. Each has a fullname that is composed of two parts; a name and a namespace. Equality of names is defined on the fullname.
The name portion of a fullname, record field names, and enum symbols must:
- start with [A-Za-z_]
- subsequently contain only [A-Za-z0-9_]
A namespace is a dot-separated sequence of such names. The empty string may also be used as a namespace to indicate the null namespace. Equality of names (including field names and enum symbols) as well as fullnames is case-sensitive.
In record, enum and fixed definitions, the fullname is determined in one of the following ways:
A name and namespace are both specified. For example, one might use “name”: “X”, “namespace”: “org.foo” to indicate the fullname org.foo.X.
-
A fullname is specified. If the name specified contains a dot, then it is assumed to be a fullname, and any namespace also specified is ignored. For example, use “name”: “org.foo.X” to indicate the fullname org.foo.X.
-
A name only is specified, i.e., a name that contains no dots. In this case the namespace is taken from the most tightly enclosing schema or protocol. For example, if “name”: “X” is specified, and this occurs within a field of the record definition of org.foo.Y, then the fullname is org.foo.X. If there is no enclosing namespace then the null namespace is used.
References to previously defined names are as in the latter two cases above: if they contain a dot they are a fullname, if they do not contain a dot, the namespace is the namespace of the enclosing definition.
Primitive type names have no namespace and their names may not be defined in any namespace.
A schema or protocol may not contain multiple definitions of a fullname. Further, a name must be defined before it is used (“before” in the depth-first, left-to-right traversal of the JSON parse tree, where the types attribute of a protocol is always deemed to come “before” the messages attribute.)
Aliases
Named types and fields may have aliases. An implementation may optionally use aliases to map a writer’s schema to the reader’s. This facilitates both schema evolution as well as processing disparate datasets.
Aliases function by re-writing the writer’s schema using aliases from the reader’s schema. For example, if the writer’s schema was named “Foo” and the reader’s schema is named “Bar” and has an alias of “Foo”, then the implementation would act as though “Foo” were named “Bar” when reading. Similarly, if data was written as a record with a field named “x” and is read as a record with a field named “y” with alias “x”, then the implementation would act as though “x” were named “y” when reading.
A type alias may be specified either as a fully namespace-qualified, or relative to the namespace of the name it is an alias for. For example, if a type named “a.b” has aliases of “c” and “x.y”, then the fully qualified names of its aliases are “a.c” and “x.y”.
Data Serialization and Deserialization
Binary encoded Avro data does not include type information or field names. The benefit is that the serialized data is small, but as a result a schema must always be used in order to read Avro data correctly. The best way to ensure that the schema is structurally identical to the one used to write the data is to use the exact same schema.
Therefore, files or systems that store Avro data should always include the writer’s schema for that data. Avro-based remote procedure call (RPC) systems must also guarantee that remote recipients of data have a copy of the schema used to write that data. In general, it is advisable that any reader of Avro data should use a schema that is the same (as defined more fully in Parsing Canonical Form for Schemas) as the schema that was used to write the data in order to deserialize it correctly. Deserializing data into a newer schema is accomplished by specifying an additional schema, the results of which are described in Schema Resolution.
In general, both serialization and deserialization proceed as a depth-first, left-to-right traversal of the schema, serializing or deserializing primitive types as they are encountered. Therefore, it is possible, though not advisable, to read Avro data with a schema that does not have the same Parsing Canonical Form as the schema with which the data was written. In order for this to work, the serialized primitive values must be compatible, in order value by value, with the items in the deserialization schema. For example, int and long are always serialized the same way, so an int could be deserialized as a long. Since the compatibility of two schemas depends on both the data and the serialization format (eg. binary is more permissive than JSON because JSON includes field names, eg. a long that is too large will overflow an int), it is simpler and more reliable to use schemas with identical Parsing Canonical Form.
Encodings
Avro specifies two serialization encodings: binary and JSON. Most applications will use the binary encoding, as it is smaller and faster. But, for debugging and web-based applications, the JSON encoding may sometimes be appropriate.
Binary Encoding
Binary encoding does not include field names, self-contained information about the types of individual bytes, nor field or record separators. Therefore readers are wholly reliant on the schema used when the data was encoded.
Primitive Types
Primitive types are encoded in binary as follows:
- null is written as zero bytes.
- a boolean is written as a single byte whose value is either 0 (false) or 1 (true).
- int and long values are written using variable-length zig-zag coding. Some examples:
| value |
hex |
| 0 |
00 |
| -1 |
01 |
| 1 |
02 |
| -2 |
03 |
| 2 |
04 |
| … |
|
| -64 |
7f |
| 64 |
80 01 |
| … |
|
- a float is written as 4 bytes. The float is converted into a 32-bit integer using a method equivalent to Java’s floatToIntBits and then encoded in little-endian format.
- a double is written as 8 bytes. The double is converted into a 64-bit integer using a method equivalent to Java’s doubleToLongBits and then encoded in little-endian format.
- bytes are encoded as a long followed by that many bytes of data.
- a string is encoded as a long followed by that many bytes of UTF-8 encoded character data.
For example, the three-character string “foo” would be encoded as the long value 3 (encoded as hex 06) followed by the UTF-8 encoding of ‘f’, ‘o’, and ‘o’ (the hex bytes 66 6f 6f):
06 66 6f 6f
Complex Types
Complex types are encoded in binary as follows:
Records
A record is encoded by encoding the values of its fields in the order that they are declared. In other words, a record is encoded as just the concatenation of the encodings of its fields. Field values are encoded per their schema.
For example, the record schema
{
"type": "record",
"name": "test",
"fields" : [
{"name": "a", "type": "long"},
{"name": "b", "type": "string"}
]
}
An instance of this record whose a field has value 27 (encoded as hex 36) and whose b field has value “foo” (encoded as hex bytes 06 66 6f 6f), would be encoded simply as the concatenation of these, namely the hex byte sequence:
36 06 66 6f 6f
Enums
An enum is encoded by a int, representing the zero-based position of the symbol in the schema.
For example, consider the enum:
{"type": "enum", "name": "Foo", "symbols": ["A", "B", "C", "D"] }
This would be encoded by an int between zero and three, with zero indicating “A”, and 3 indicating “D”.
Arrays
Arrays are encoded as a series of blocks. Each block consists of a long count value, followed by that many array items. A block with count zero indicates the end of the array. Each item is encoded per the array’s item schema.
If a block’s count is negative, its absolute value is used, and the count is followed immediately by a long block size indicating the number of bytes in the block. This block size permits fast skipping through data, e.g., when projecting a record to a subset of its fields.
For example, the array schema
{"type": "array", "items": "long"}
an array containing the items 3 and 27 could be encoded as the long value 2 (encoded as hex 04) followed by long values 3 and 27 (encoded as hex 06 36) terminated by zero:
04 06 36 00
The blocked representation permits one to read and write arrays larger than can be buffered in memory, since one can start writing items without knowing the full length of the array.
Maps
Maps are encoded as a series of blocks. Each block consists of a long count value, followed by that many key/value pairs. A block with count zero indicates the end of the map. Each item is encoded per the map’s value schema.
If a block’s count is negative, its absolute value is used, and the count is followed immediately by a long block size indicating the number of bytes in the block. This block size permits fast skipping through data, e.g., when projecting a record to a subset of its fields.
The blocked representation permits one to read and write maps larger than can be buffered in memory, since one can start writing items without knowing the full length of the map.
Unions
A union is encoded by first writing a long value indicating the zero-based position within the union of the schema of its value. The value is then encoded per the indicated schema within the union.
For example, the union schema ["null","string"] would encode:
-
null as zero (the index of “null” in the union):
00
-
the string “a” as one (the index of “string” in the union, 1, encoded as hex 02), followed by the serialized string:
02 02 61
NOTE: Currently for C/C++ implementtions, the positions are practically an int, but theoretically a long. In reality, we don’t expect unions with 215M members
Fixed
Fixed instances are encoded using the number of bytes declared in the schema.
JSON Encoding
Except for unions, the JSON encoding is the same as is used to encode field default values.
The value of a union is encoded in JSON as follows:
- if its type is null, then it is encoded as a JSON null;
- otherwise it is encoded as a JSON object with one name/value pair whose name is the type’s name and whose value is the recursively encoded value. For Avro’s named types (record, fixed or enum) the user-specified name is used, for other types the type name is used.
For example, the union schema ["null","string","Foo"], where Foo is a record name, would encode:
- null as null;
- the string “a” as
{"string": "a"} and
- a Foo instance as
{"Foo": {...}}, where {...} indicates the JSON encoding of a Foo instance.
Note that the original schema is still required to correctly process JSON-encoded data. For example, the JSON encoding does not distinguish between int and long, float and double, records and maps, enums and strings, etc.
Single-object encoding
In some situations a single Avro serialized object is to be stored for a longer period of time. One very common example is storing Avro records for several weeks in an Apache Kafka topic.
In the period after a schema change this persistence system will contain records that have been written with different schemas. So the need arises to know which schema was used to write a record to support schema evolution correctly. In most cases the schema itself is too large to include in the message, so this binary wrapper format supports the use case more effectively.
Single object encoding specification
Single Avro objects are encoded as follows:
- A two-byte marker,
C3 01, to show that the message is Avro and uses this single-record format (version 1).
- The 8-byte little-endian CRC-64-AVRO fingerprint of the object’s schema.
- The Avro object encoded using Avro’s binary encoding.
Implementations use the 2-byte marker to determine whether a payload is Avro. This check helps avoid expensive lookups that resolve the schema from a fingerprint, when the message is not an encoded Avro payload.
Sort Order
Avro defines a standard sort order for data. This permits data written by one system to be efficiently sorted by another system. This can be an important optimization, as sort order comparisons are sometimes the most frequent per-object operation. Note also that Avro binary-encoded data can be efficiently ordered without deserializing it to objects.
Data items may only be compared if they have identical schemas. Pairwise comparisons are implemented recursively with a depth-first, left-to-right traversal of the schema. The first mismatch encountered determines the order of the items.
Two items with the same schema are compared according to the following rules.
- null data is always equal.
- boolean data is ordered with false before true.
- int, long, float and double data is ordered by ascending numeric value.
- bytes and fixed data are compared lexicographically by unsigned 8-bit values.
- string data is compared lexicographically by Unicode code point. Note that since UTF-8 is used as the binary encoding for strings, sorting of bytes and string binary data is identical.
- array data is compared lexicographically by element.
- enum data is ordered by the symbol’s position in the enum schema. For example, an enum whose symbols are
["z", "a"] would sort “z” values before “a” values.
- union data is first ordered by the branch within the union, and, within that, by the type of the branch. For example, an
["int", "string"] union would order all int values before all string values, with the ints and strings themselves ordered as defined above.
- record data is ordered lexicographically by field. If a field specifies that its order is:
- “ascending”, then the order of its values is unaltered.
- “descending”, then the order of its values is reversed.
- “ignore”, then its values are ignored when sorting.
- map data may not be compared. It is an error to attempt to compare data containing maps unless those maps are in an
"order":"ignore" record field.
Object Container Files
Avro includes a simple object container file format. A file has a schema, and all objects stored in the file must be written according to that schema, using binary encoding. Objects are stored in blocks that may be compressed. Syncronization markers are used between blocks to permit efficient splitting of files for MapReduce processing.
Files may include arbitrary user-specified metadata.
A file consists of:
- A file header, followed by
- one or more file data blocks.
A file header consists of:
- Four bytes, ASCII ‘O’, ‘b’, ‘j’, followed by 1.
- file metadata, including the schema.
- The 16-byte, randomly-generated sync marker for this file.
File metadata is written as if defined by the following map schema:
{"type": "map", "values": "bytes"}
All metadata properties that start with “avro.” are reserved. The following file metadata properties are currently used:
- avro.schema contains the schema of objects stored in the file, as JSON data (required).
- avro.codec the name of the compression codec used to compress blocks, as a string. Implementations are required to support the following codecs: “null” and “deflate”. If codec is absent, it is assumed to be “null”. The codecs are described with more detail below.
A file header is thus described by the following schema:
{"type": "record", "name": "org.apache.avro.file.Header",
"fields" : [
{"name": "magic", "type": {"type": "fixed", "name": "Magic", "size": 4}},
{"name": "meta", "type": {"type": "map", "values": "bytes"}},
{"name": "sync", "type": {"type": "fixed", "name": "Sync", "size": 16}}
]
}
A file data block consists of:
- A long indicating the count of objects in this block.
- A long indicating the size in bytes of the serialized objects in the current block, after any codec is applied
- The serialized objects. If a codec is specified, this is compressed by that codec.
- The file’s 16-byte sync marker.
Thus, each block’s binary data can be efficiently extracted or skipped without deserializing the contents. The combination of block size, object counts, and sync markers enable detection of corrupt blocks and help ensure data integrity.
Required Codecs
null
The “null” codec simply passes through data uncompressed.
deflate
The “deflate” codec writes the data block using the deflate algorithm as specified in RFC 1951, and typically implemented using the zlib library. Note that this format (unlike the “zlib format” in RFC 1950) does not have a checksum.
Optional Codecs
bzip2
The “bzip2” codec uses the bzip2 compression library.
snappy
The “snappy” codec uses Google’s Snappy compression library. Each compressed block is followed by the 4-byte, big-endian CRC32 checksum of the uncompressed data in the block.
Protocol Declaration
Avro protocols describe RPC interfaces. Like schemas, they are defined with JSON text.
A protocol is a JSON object with the following attributes:
- protocol, a string, the name of the protocol (required);
- namespace, an optional string that qualifies the name;
- doc, an optional string describing this protocol;
- types, an optional list of definitions of named types (records, enums, fixed and errors). An error definition is just like a record definition except it uses “error” instead of “record”. Note that forward references to named types are not permitted.
- messages, an optional JSON object whose keys are message names and whose values are objects whose attributes are described below. No two messages may have the same name.
The name and namespace qualification rules defined for schema objects apply to protocols as well.
Messages
A message has attributes:
- a doc, an optional description of the message,
- a request, a list of named, typed parameter schemas (this has the same form as the fields of a record declaration);
- a response schema;
- an optional union of declared error schemas. The effective union has “string” prepended to the declared union, to permit transmission of undeclared “system” errors. For example, if the declared error union is
["AccessError"], then the effective union is ["string", "AccessError"]. When no errors are declared, the effective error union is ["string"]. Errors are serialized using the effective union; however, a protocol’s JSON declaration contains only the declared union.
- an optional one-way boolean parameter.
A request parameter list is processed equivalently to an anonymous record. Since record field lists may vary between reader and writer, request parameters may also differ between the caller and responder, and such differences are resolved in the same manner as record field differences.
The one-way parameter may only be true when the response type is "null" and no errors are listed.
Sample Protocol
For example, one may define a simple HelloWorld protocol with:
{
"namespace": "com.acme",
"protocol": "HelloWorld",
"doc": "Protocol Greetings",
"types": [
{"name": "Greeting", "type": "record", "fields": [
{"name": "message", "type": "string"}]},
{"name": "Curse", "type": "error", "fields": [
{"name": "message", "type": "string"}]}
],
"messages": {
"hello": {
"doc": "Say hello.",
"request": [{"name": "greeting", "type": "Greeting" }],
"response": "Greeting",
"errors": ["Curse"]
}
}
}
Message Transport
Messages may be transmitted via different transport mechanisms.
To the transport, a message is an opaque byte sequence.
A transport is a system that supports:
- transmission of request messages
- receipt of corresponding response messages
Servers may send a response message back to the client corresponding to a request message. The mechanism of correspondance is transport-specific. For example, in HTTP it is implicit, since HTTP directly supports requests and responses. But a transport that multiplexes many client threads over a single socket would need to tag messages with unique identifiers.
Transports may be either stateless or stateful. In a stateless transport, messaging assumes no established connection state, while stateful transports establish connections that may be used for multiple messages. This distinction is discussed further in the handshake section below.
HTTP as Transport
When HTTP is used as a transport, each Avro message exchange is an HTTP request/response pair. All messages of an Avro protocol should share a single URL at an HTTP server. Other protocols may also use that URL. Both normal and error Avro response messages should use the 200 (OK) response code. The chunked encoding may be used for requests and responses, but, regardless the Avro request and response are the entire content of an HTTP request and response. The HTTP Content-Type of requests and responses should be specified as “avro/binary”. Requests should be made using the POST method.
HTTP is used by Avro as a stateless transport.
Message Framing
Avro messages are framed as a list of buffers.
Framing is a layer between messages and the transport. It exists to optimize certain operations.
The format of framed message data is:
- a series of buffers, where each buffer consists of:
- a four-byte, big-endian buffer length, followed by
- that many bytes of buffer data.
- A message is always terminated by a zero-length buffer.
Framing is transparent to request and response message formats (described below). Any message may be presented as a single or multiple buffers.
Framing can permit readers to more efficiently get different buffers from different sources and for writers to more efficiently store different buffers to different destinations. In particular, it can reduce the number of times large binary objects are copied. For example, if an RPC parameter consists of a megabyte of file data, that data can be copied directly to a socket from a file descriptor, and, on the other end, it could be written directly to a file descriptor, never entering user space.
A simple, recommended, framing policy is for writers to create a new segment whenever a single binary object is written that is larger than a normal output buffer. Small objects are then appended in buffers, while larger objects are written as their own buffers. When a reader then tries to read a large object the runtime can hand it an entire buffer directly, without having to copy it.
Handshake
The purpose of the handshake is to ensure that the client and the server have each other’s protocol definition, so that the client can correctly deserialize responses, and the server can correctly deserialize requests. Both clients and servers should maintain a cache of recently seen protocols, so that, in most cases, a handshake will be completed without extra round-trip network exchanges or the transmission of full protocol text.
RPC requests and responses may not be processed until a handshake has been completed. With a stateless transport, all requests and responses are prefixed by handshakes. With a stateful transport, handshakes are only attached to requests and responses until a successful handshake response has been returned over a connection. After this, request and response payloads are sent without handshakes for the lifetime of that connection.
The handshake process uses the following record schemas:
{
"type": "record",
"name": "HandshakeRequest", "namespace":"org.apache.avro.ipc",
"fields": [
{"name": "clientHash",
"type": {"type": "fixed", "name": "MD5", "size": 16}},
{"name": "clientProtocol", "type": ["null", "string"]},
{"name": "serverHash", "type": "MD5"},
{"name": "meta", "type": ["null", {"type": "map", "values": "bytes"}]}
]
}
{
"type": "record",
"name": "HandshakeResponse", "namespace": "org.apache.avro.ipc",
"fields": [
{"name": "match",
"type": {"type": "enum", "name": "HandshakeMatch",
"symbols": ["BOTH", "CLIENT", "NONE"]}},
{"name": "serverProtocol",
"type": ["null", "string"]},
{"name": "serverHash",
"type": ["null", {"type": "fixed", "name": "MD5", "size": 16}]},
{"name": "meta",
"type": ["null", {"type": "map", "values": "bytes"}]}
]
}
- A client first prefixes each request with a
HandshakeRequest containing just the hash of its protocol and of the server’s protocol (clientHash!=null, clientProtocol=null, serverHash!=null), where the hashes are 128-bit MD5 hashes of the JSON protocol text. If a client has never connected to a given server, it sends its hash as a guess of the server’s hash, otherwise it sends the hash that it previously obtained from this server.
The server responds with a HandshakeResponse containing one of:
-
match=BOTH, serverProtocol=null, serverHash=null if the client sent the valid hash of the server’s protocol and the server knows what protocol corresponds to the client’s hash. In this case, the request is complete and the response data immediately follows the HandshakeResponse.
-
match=CLIENT, serverProtocol!=null, serverHash!=null if the server has previously seen the client’s protocol, but the client sent an incorrect hash of the server’s protocol. The request is complete and the response data immediately follows the HandshakeResponse. The client must use the returned protocol to process the response and should also cache that protocol and its hash for future interactions with this server.
-
match=NONE if the server has not previously seen the client’s protocol. The serverHash and serverProtocol may also be non-null if the server’s protocol hash was incorrect.
In this case the client must then re-submit its request with its protocol text (clientHash!=null, clientProtocol!=null, serverHash!=null) and the server should respond with a successful match (match=BOTH, serverProtocol=null, serverHash=null) as above.
The meta field is reserved for future handshake enhancements.
A call consists of a request message paired with its resulting response or error message. Requests and responses contain extensible metadata, and both kinds of messages are framed as described above.
The format of a call request is:
- request metadata, a map with values of type bytes
- the message name, an Avro string, followed by
- the message parameters. Parameters are serialized according to the message’s request declaration.
When the empty string is used as a message name a server should ignore the parameters and return an empty response. A client may use this to ping a server or to perform a handshake without sending a protocol message.
When a message is declared one-way and a stateful connection has been established by a successful handshake response, no response data is sent. Otherwise the format of the call response is:
- response metadata, a map with values of type bytes
- a one-byte error flag boolean, followed by either:
- if the error flag is false, the message response, serialized per the message’s response schema.
- if the error flag is true, the error, serialized per the message’s effective error union schema.
Schema Resolution
A reader of Avro data, whether from an RPC or a file, can always parse that data because the original schema must be provided along with the data. However, the reader may be programmed to read data into a different schema. For example, if the data was written with a different version of the software than it is read, then fields may have been added or removed from records. This section specifies how such schema differences should be resolved.
We refer to the schema used to write the data as the writer’s schema, and the schema that the application expects the reader’s schema. Differences between these should be resolved as follows:
- It is an error if the two schemas do not match.
To match, one of the following must hold:
-
both schemas are arrays whose item types match
-
both schemas are maps whose value types match
-
both schemas are enums whose names match
-
both schemas are fixed whose sizes and names match
-
both schemas are records with the same name
-
either schema is a union
-
both schemas have same primitive type
-
the writer’s schema may be promoted to the reader’s as follows:
- int is promotable to long, float, or double
- long is promotable to float or double
- float is promotable to double
- string is promotable to bytes
- bytes is promotable to string
-
if both are records:
- the ordering of fields may be different: fields are matched by name.
- schemas for fields with the same name in both records are resolved recursively.
- if the writer’s record contains a field with a name not present in the reader’s record, the writer’s value for that field is ignored.
- if the reader’s record schema has a field that contains a default value, and writer’s schema does not have a field with the same name, then the reader should use the default value from its field.
- if the reader’s record schema has a field with no default value, and writer’s schema does not have a field with the same name, an error is signalled.
-
if both are enums:
if the writer’s symbol is not present in the reader’s enum and the reader has a default value, then that value is used, otherwise an error is signalled.
This resolution algorithm is applied recursively to the reader’s and writer’s array item schemas.
This resolution algorithm is applied recursively to the reader’s and writer’s value schemas.
The first schema in the reader’s union that matches the selected writer’s union schema is recursively resolved against it. if none match, an error is signalled.
- if reader’s is a union, but writer’s is not
The first schema in the reader’s union that matches the writer’s schema is recursively resolved against it. If none match, an error is signalled.
- if writer’s is a union, but reader’s is not
If the reader’s schema matches the selected writer’s schema, it is recursively resolved against it. If they do not match, an error is signalled.
A schema’s doc fields are ignored for the purposes of schema resolution. Hence, the doc portion of a schema may be dropped at serialization.
One of the defining characteristics of Avro is that a reader must use the schema used by the writer of the data in order to know how to read the data. This assumption results in a data format that’s compact and also amenable to many forms of schema evolution. However, the specification so far has not defined what it means for the reader to have the “same” schema as the writer. Does the schema need to be textually identical? Well, clearly adding or removing some whitespace to a JSON expression does not change its meaning. At the same time, reordering the fields of records clearly does change the meaning. So what does it mean for a reader to have “the same” schema as a writer?
Parsing Canonical Form is a transformation of a writer’s schema that let’s us define what it means for two schemas to be “the same” for the purpose of reading data written against the schema. It is called Parsing Canonical Form because the transformations strip away parts of the schema, like “doc” attributes, that are irrelevant to readers trying to parse incoming data. It is called Canonical Form because the transformations normalize the JSON text (such as the order of attributes) in a way that eliminates unimportant differences between schemas. If the Parsing Canonical Forms of two different schemas are textually equal, then those schemas are “the same” as far as any reader is concerned, i.e., there is no serialized data that would allow a reader to distinguish data generated by a writer using one of the original schemas from data generated by a writing using the other original schema. (We sketch a proof of this property in a companion document.)
The next subsection specifies the transformations that define Parsing Canonical Form. But with a well-defined canonical form, it can be convenient to go one step further, transforming these canonical forms into simple integers (“fingerprints”) that can be used to uniquely identify schemas. The subsection after next recommends some standard practices for generating such fingerprints.
Assuming an input schema (in JSON form) that’s already UTF-8 text for a valid Avro schema (including all quotes as required by JSON), the following transformations will produce its Parsing Canonical Form:
- [PRIMITIVES] Convert primitive schemas to their simple form (e.g., int instead of
{"type":"int"}).
- [FULLNAMES] Replace short names with fullnames, using applicable namespaces to do so. Then eliminate namespace attributes, which are now redundant.
- [STRIP] Keep only attributes that are relevant to parsing data, which are: type, name, fields, symbols, items, values, size. Strip all others (e.g., doc and aliases).
- [ORDER] Order the appearance of fields of JSON objects as follows: name, type, fields, symbols, items, values, size. For example, if an object has type, name, and size fields, then the name field should appear first, followed by the type and then the size fields.
- [STRINGS] For all JSON string literals in the schema text, replace any escaped characters (e.g., \uXXXX escapes) with their UTF-8 equivalents.
- [INTEGERS] Eliminate quotes around and any leading zeros in front of JSON integer literals (which appear in the size attributes of fixed schemas).
- [WHITESPACE] Eliminate all whitespace in JSON outside of string literals.
Schema Fingerprints
“[A] fingerprinting algorithm is a procedure that maps an arbitrarily large data item (such as a computer file) to a much shorter bit string, its fingerprint, that uniquely identifies the original data for all practical purposes” (quoted from Wikipedia). In the Avro context, fingerprints of Parsing Canonical Form can be useful in a number of applications; for example, to cache encoder and decoder objects, to tag data items with a short substitute for the writer’s full schema, and to quickly negotiate common-case schemas between readers and writers.
In designing fingerprinting algorithms, there is a fundamental trade-off between the length of the fingerprint and the probability of collisions. To help application designers find appropriate points within this trade-off space, while encouraging interoperability and ease of implementation, we recommend using one of the following three algorithms when fingerprinting Avro schemas:
- When applications can tolerate longer fingerprints, we recommend using the SHA-256 digest algorithm to generate 256-bit fingerprints of Parsing Canonical Forms. Most languages today have SHA-256 implementations in their libraries.
- At the opposite extreme, the smallest fingerprint we recommend is a 64-bit Rabin fingerprint. Below, we provide pseudo-code for this algorithm that can be easily translated into any programming language. 64-bit fingerprints should guarantee uniqueness for schema caches of up to a million entries (for such a cache, the chance of a collision is 3E-8). We don’t recommend shorter fingerprints, as the chances of collisions is too great (for example, with 32-bit fingerprints, a cache with as few as 100,000 schemas has a 50% chance of having a collision).
- Between these two extremes, we recommend using the MD5 message digest to generate 128-bit fingerprints. These make sense only where very large numbers of schemas are being manipulated (tens of millions); otherwise, 64-bit fingerprints should be sufficient. As with SHA-256, MD5 implementations are found in most libraries today.
These fingerprints are not meant to provide any security guarantees, even the longer SHA-256-based ones. Most Avro applications should be surrounded by security measures that prevent attackers from writing random data and otherwise interfering with the consumers of schemas. We recommend that these surrounding mechanisms be used to prevent collision and pre-image attacks (i.e., “forgery”) on schema fingerprints, rather than relying on the security properties of the fingerprints themselves.
Rabin fingerprints are cyclic redundancy checks computed using irreducible polynomials. In the style of the Appendix of RFC 1952 (pg 10), which defines the CRC-32 algorithm, here’s our definition of the 64-bit AVRO fingerprinting algorithm:
long fingerprint64(byte[] buf) {
if (FP_TABLE == null) initFPTable();
long fp = EMPTY;
for (int i = 0; i < buf.length; i++)
fp = (fp >>> 8) ^ FP_TABLE[(int)(fp ^ buf[i]) & 0xff];
return fp;
}
static long EMPTY = 0xc15d213aa4d7a795L;
static long[] FP_TABLE = null;
void initFPTable() {
FP_TABLE = new long[256];
for (int i = 0; i < 256; i++) {
long fp = i;
for (int j = 0; j < 8; j++)
fp = (fp >>> 1) ^ (EMPTY & -(fp & 1L));
FP_TABLE[i] = fp;
}
}
Readers interested in the mathematics behind this algorithm may want to read this book chapter. (Unlike RFC-1952 and the book chapter, we prepend a single one bit to messages. We do this because CRCs ignore leading zero bits, which can be problematic. Our code prepends a one-bit by initializing fingerprints using EMPTY, rather than initializing using zero as in RFC-1952 and the book chapter.)
Logical Types
A logical type is an Avro primitive or complex type with extra attributes to represent a derived type. The attribute logicalType must always be present for a logical type, and is a string with the name of one of the logical types listed later in this section. Other attributes may be defined for particular logical types.
A logical type is always serialized using its underlying Avro type so that values are encoded in exactly the same way as the equivalent Avro type that does not have a logicalType attribute. Language implementations may choose to represent logical types with an appropriate native type, although this is not required.
Language implementations must ignore unknown logical types when reading, and should use the underlying Avro type. If a logical type is invalid, for example a decimal with scale greater than its precision, then implementations should ignore the logical type and use the underlying Avro type.
Decimal
The decimal logical type represents an arbitrary-precision signed decimal number of the form unscaled × 10-scale.
A decimal logical type annotates Avro bytes or fixed types. The byte array must contain the two’s-complement representation of the unscaled integer value in big-endian byte order. The scale is fixed, and is specified using an attribute.
The following attributes are supported:
- scale, a JSON integer representing the scale (optional). If not specified the scale is 0.
- precision, a JSON integer representing the (maximum) precision of decimals stored in this type (required).
For example, the following schema represents decimal numbers with a maximum precision of 4 and a scale of 2:
{
"type": "bytes",
"logicalType": "decimal",
"precision": 4,
"scale": 2
}
Precision must be a positive integer greater than zero. If the underlying type is a fixed, then the precision is limited by its size. An array of length n can store at most floor(log10(28 × n - 1 - 1)) base-10 digits of precision.
Scale must be zero or a positive integer less than or equal to the precision.
For the purposes of schema resolution, two schemas that are decimal logical types match if their scales and precisions match.
UUID
The uuid logical type represents a random generated universally unique identifier (UUID).
A uuid logical type annotates an Avro string. The string has to conform with RFC-4122
Date
The date logical type represents a date within the calendar, with no reference to a particular time zone or time of day.
A date logical type annotates an Avro int, where the int stores the number of days from the unix epoch, 1 January 1970 (ISO calendar).
Time (millisecond precision)
The time-millis logical type represents a time of day, with no reference to a particular calendar, time zone or date, with a precision of one millisecond.
A time-millis logical type annotates an Avro int, where the int stores the number of milliseconds after midnight, 00:00:00.000.
Time (microsecond precision)
The time-micros logical type represents a time of day, with no reference to a particular calendar, time zone or date, with a precision of one microsecond.
A time-micros logical type annotates an Avro long, where the long stores the number of microseconds after midnight, 00:00:00.000000.
Timestamp (millisecond precision)
The timestamp-millis logical type represents an instant on the global timeline, independent of a particular time zone or calendar, with a precision of one millisecond.
A timestamp-millis logical type annotates an Avro long, where the long stores the number of milliseconds from the unix epoch, 1 January 1970 00:00:00.000 UTC.
Timestamp (microsecond precision)
The timestamp-micros logical type represents an instant on the global timeline, independent of a particular time zone or calendar, with a precision of one microsecond.
A timestamp-micros logical type annotates an Avro long, where the long stores the number of microseconds from the unix epoch, 1 January 1970 00:00:00.000000 UTC.
Duration
The duration logical type represents an amount of time defined by a number of months, days and milliseconds. This is not equivalent to a number of milliseconds, because, depending on the moment in time from which the duration is measured, the number of days in the month and number of milliseconds in a day may differ. Other standard periods such as years, quarters, hours and minutes can be expressed through these basic periods.
A duration logical type annotates Avro fixed type of size 12, which stores three little-endian unsigned integers that represent durations at different granularities of time. The first stores a number in months, the second stores a number in days, and the third stores a number in milliseconds.
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
8.4 - Trevni
Version 0.1
DRAFT
This document is the authoritative specification of a file format. Its intent is to permit compatible, independent implementations that read and/or write files in this format.
Introduction
Data sets are often described as a table composed of rows and columns. Each record in the dataset is considered a row, with each field of the record occupying a different column. Writing records to a file one-by-one as they are created results in a row-major format, like Hadoop’s SequenceFile or Avro data files.
In many cases higher query performance may be achieved if the data is instead organized in a column-major format, where multiple values of a given column are stored adjacently. This document defines such a column-major file format for datasets.
To permit scalable, distributed query evaluation, datasets are partitioned into row groups, containing distinct collections of rows. Each row group is organized in column-major order, while row groups form a row-major partitioning of the entire dataset.
Rationale
Goals
The format is meant satisfy the following goals:
Maximize the size of row groups. Disc drives are used most efficiently when sequentially accessing data. Consider a drive that takes 10ms to seek and transfers at 100MB/second. If a 10-column dataset whose values are all the same size is split into 10MB row groups, then accessing a single column will require a sequence of seek+1MB reads, for a cost of 20ms/MB processed. If the same dataset is split into 100MB row groups then this drops to 11ms/MB processed. This effect is exaggerated for datasets with larger numbers of columns and with columns whose values are smaller than average. So we’d prefer row groups that are 100MB or greater.
Permit random access within a row group. Some queries will first examine one column, and, only when certain relatively rare criteria are met, examine other columns. Rather than iterating through selected columns of the row-group in parallel, one might iterate through one column and randomly access another. This is called support for WHERE clauses, after the SQL operator of that name.
Minimize the number of files per dataset. HDFS is a primary intended deployment platform for these files. The HDFS Namenode requires memory for each file in the filesystem, thus for a format to be HDFS-friendly it should strive to require the minimum number of distinct files.
Support co-location of columns within row-groups. Row groups are the unit of parallel operation on a column dataset. For efficient file i/o, the entirety of a row-group should ideally reside on the host that is evaluating the query in order to avoid network latencies and bottlenecks.
Data integrity. The format should permit applications to detect data corruption. Many file systems may prevent corruption, but files may be moved between filesystems and be subject to corruption at points in that process. It is best if the data in a file can be validated independently.
Extensibility. The format should permit applications to store additional annotations about a datasets in the files, such as type information, origin, etc. Some environments may have metadata stores for such information, but not all do, and files might be moved among systems with different metadata systems. The ability to keep such information within the file simplifies the coordination of such information.
Minimal overhead. The column format should not make datasets appreciably larger. Storage is a primary cost and a choice to use this format should not require additional storage.
Primary format. The column format should be usable as a primary format for datasets, not as an auxiliary, accelerated format. Applications that process a dataset in row-major order should be able to easily consume column files and applications that produce datasets in row-major order should be able to easily generate column files.
Design
To meet these goals we propose the following design.
Each row group is a separate file. All values of a column in a file are written contiguously. This maximizes the row group size, optimizing performance when querying few and small columns.
Each file occupies a single HDFS block. A larger than normal block size may be specified, e.g., ~1GB instead of the typical ~100MB. This guarantees co-location and eliminates network use when query processing can be co-located with the file. This also moderates the memory impact on the HDFS Namenode since no small files are written.
Each column in a file is written as a sequence of ~64kB compressed blocks. The sequence is prefixed by a table describing all of the blocks in the column to permit random access within the column.
Application-specific metadata may be added at the file, column, and block levels.
Checksums are included with each block, providing data integrity.
Discussion
The use of a single block per file achieves the same effect as the custom block placement policy described in the CIF paper, but while still permitting HDFS rebalancing and not increasing the number of files in the namespace.
This section formally describes the proposed column file format.
Data Model
We assume a simple data model, where a record is a set of named fields, and the value of each field is a sequence of untyped bytes. A type system may be layered on top of this, as specified in the Type Mapping section below.
Primitive Values
We define the following primitive value types:
- Signed 64-bit long values are written using a variable-length zig-zag coding, where the high-order bit in each byte determines whether subsequent bytes are present. For example:
decimal value hex bytes
| decimal value |
hex bytes |
|
| 0 |
00 |
|
| -1 |
01 |
|
| 1 |
02 |
|
| … |
|
|
| -64 |
7f |
|
| 64 |
80 01 |
|
| … |
|
|
- bytes are encoded as a long followed by that many bytes of data.
- a string is encoded as a long followed by that many bytes of UTF-8 encoded character data.
For example, the three-character string “foo” would be encoded as the long value 3 (encoded as hex 06) followed by the UTF-8 encoding of ‘f’, ‘o’, and ‘o’ (the hex bytes 66 6f 6f): 06 66 6f 6f
Type Names
The following type names are used to describe column values:
- null, requires zero bytes. Sometimes used in array columns.
- boolean, one bit, packed into bytes, little-endian;
- int, like long, but restricted to 32-bit signed values
- long 64-bit signed values, represented as above
- fixed32 32-bit values stored as four bytes, little-endian.
- fixed64 64-bit values stored as eight bytes, little-endian.
- float 32-bit IEEE floating point value, little-endian
- double 64-bit IEEE floating point value, little-endian
- string as above
- bytes as above, may be used to encapsulate more complex objects
Type names are represented as strings (UTF-8 encoded, length-prefixed).
Metadata consists of:
- A long indicating the number of metadata key/value pairs.
- For each pair, a string key and bytes value.
All metadata properties that start with “trevni.” are reserved.
The following file metadata properties are defined:
- trevni.codec the name of the default compression codec used to compress blocks, as a string. Implementations are required to support the “null” codec. Optional. If absent, it is assumed to be “null”. Codecs are described in more detail below.
- trevni.checksum the name of the checksum algorithm used in this file, as a string. Implementations are required to support the “crc-32” checksum. Optional. If absent, it is assumed to be “null”. Checksums are described in more detail below.
The following column metadata properties are defined:
- trevni.codec the name of the compression codec used to compress the blocks of this column, as a string. Implementations are required to support the “null” codec. Optional. If absent, it is assumed to be “null”. Codecs are described in more detail below.
- trevni.name the name of the column, as a string. Required.
- trevni.type the type of data in the column. One of the type names above. Required.
- trevni.values if present, indicates that the initial value of each block in this column will be stored in the block’s descriptor. Not permitted for array columns or columns that specify a parent.
- trevni.array if present, indicates that each row in this column contains a sequence of values of the named type rather than just a single value. An integer length precedes each sequence of values indicating the count of values in the sequence. If the length is negative then it indicates a sequence of zero or one lengths, where -1 indicates two zeros, -2 two ones, -3 three zeros, -4 three ones, etc.
- trevni.parent if present, the name of an array column whose lengths are also used by this column. Thus values of this column are sequences but no lengths are stored in this column.
For example, consider the following row, as JSON, where all values are primitive types, but one has multiple values.
{"id"=566, "date"=23423234234
"from"="foo@bar.com",
"to"=["bar@baz.com", "bang@foo.com"],
"content"="Hi!"}
The columns for this might be specified as:
name=id type=int
name=date type=long
name=from type=string
name=to type=string array=true
name=content type=string
If a row contains an array of records, e.g. “received” in the following:
{"id"=566, "date"=23423234234
"from"="foo@bar.com",
"to"=["bar@baz.com", "bang@foo.com"],
"content"="Hi!"
"received"=[{"date"=234234234234, "host"="192.168.0.0.1"},
{"date"=234234545645, "host"="192.168.0.0.2"}]
}
Then one can define a parent column followed by a column for each field in the record, adding the following columns:
name=received type=null array=true
name=date type=long parent=received
name=host type=string parent=received
If an array value itself contains an array, e.g. the “sigs” below:
{"id"=566, "date"=23423234234
"from"="foo@bar.com",
"to"=["bar@baz.com", "bang@foo.com"],
"content"="Hi!"
"received"=[{"date"=234234234234, "host"="192.168.0.0.1",
"sigs"=[{"algo"="weak", "value"="0af345de"}]},
{"date"=234234545645, "host"="192.168.0.0.2",
"sigs"=[]}]
}
Then a parent column may be defined that itself has a parent column.
name=sigs type=null array=true parent=received
name=algo type=string parent=sigs
name=value type=string parent=sigs
No block metadata properties are currently defined.
A file consists of:
- A file header, followed by
- one or more columns.
A file header consists of:
- Four bytes, ASCII ‘T’, ‘r’, ‘v’, followed by 0x02.
- a fixed64 indicating the number of rows in the file
- a fixed32 indicating the number of columns in the file
- file metadata.
- for each column, its column metadata
- for each column, its starting position in the file as a fixed64.
A column consists of:
- A fixed32 indicating the number of blocks in this column.
- For each block, a block descriptor
- One or more blocks.
A block descriptor consists of:
- A fixed32 indicating the number of rows in the block
- A fixed32 indicating the size in bytes of the block before the codec is applied (excluding checksum).
- A fixed32 indicating the size in bytes of the block after the codec is applied (excluding checksum).
- If this column’s metadata declares it to include values, the first value in the column, serialized according to this column’s type.
A block consists of:
- The serialized column values. If a column is an array column then value sequences are preceded by their length, as an int. If a codec is specified, the values and lengths are compressed by that codec.
- The checksum, as determined by the file metadata.
Codecs
null
The “null” codec simply passes data through uncompressed.
deflate
The “deflate” codec writes the data block using the deflate algorithm as specified in RFC 1951.
snappy
The “snappy” codec uses Google’s Snappy compression library.
Checksum algorithms
null
The “null” checksum contains zero bytes.
crc-32
Each “crc-32” checksum contains the four bytes of an ISO 3309 CRC-32 checksum of the uncompressed block data as a fixed32.
Type Mappings
We define a standard mapping for how types defined in various serialization systems are represented in a column file. Records from these systems are shredded into columns. When records are nested, a depth-first recursive walk can assign a separate column for each primitive value.
Avro
Protocol Buffers
Thrift
Implementation Notes
Some possible techniques for writing column files include:
- Use a standard ~100MB block, buffer in memory up to the block size, then flush the file directly to HDFS. A single reduce task might create multiple output files. The namenode requires memory proportional to the number of names and blocks*replication. This would increase the number of names but not blocks, so this should still be much better than a file per column.
- Spill each column to a separate local, temporary file then, when the file is closed, append these files, writing a single file to HDFS whose block size is set to be that of the entire file. This would be a bit slower than and may have trouble when the local disk is full, but it would better use HDFS namespace and further reduce seeks when processing columns whose values are small.
- Use a separate mapreduce job to convert row-major files to column-major. The map output would output a by (row#, column#, value) tuple, partitioned by row# but sorted by column# then row#. The reducer could directly write the column file. But the column file format would need to be changed to write counts, descriptors, etc. at the end of files rather than at the front.
(1) is the simplest to implement and most implementations should start with it.
References
CIF Column-Oriented Storage Techniques for MapReduce, Floratou, Patel, Shekita, & Tata, VLDB 2011.
DREMEL Dremel: Interactive Analysis of Web-Scale Datasets, Melnik, Gubarev, Long, Romer, Shivakumar, & Tolton, VLDB 2010.
8.5 - MapReduce guide
Avro provides a convenient way to represent complex data structures within a Hadoop MapReduce job. Avro data can be used as both input to and output from a MapReduce job, as well as the intermediate format. The example in this guide uses Avro data for all three, but it’s possible to mix and match; for instance, MapReduce can be used to aggregate a particular field in an Avro record.
This guide assumes basic familiarity with both Hadoop MapReduce and Avro. See the Hadoop documentation and the Avro getting started guide for introductions to these projects. This guide uses the old MapReduce API (org.apache.hadoop.mapred) and the new MapReduce API (org.apache.hadoop.mapreduce).
Setup
The code from this guide is included in the Avro docs under examples/mr-example. The example is set up as a Maven project that includes the necessary Avro and MapReduce dependencies and the Avro Maven plugin for code generation, so no external jars are needed to run the example. In particular, the POM includes the following dependencies:
<dependency>
<groupId>org.apache.avro</groupId>
<artifactId>avro</artifactId>
<version>1.9.0</version>
</dependency>
<dependency>
<groupId>org.apache.avro</groupId>
<artifactId>avro-mapred</artifactId>
<version>1.9.0</version>
</dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-core</artifactId>
<version>1.1.0</version>
</dependency>
And the following plugin:
<plugin>
<groupId>org.apache.avro</groupId>
<artifactId>avro-maven-plugin</artifactId>
<version>1.9.0</version>
<executions>
<execution>
<phase>generate-sources</phase>
<goals>
<goal>schema</goal>
</goals>
<configuration>
<sourceDirectory>${project.basedir}/../</sourceDirectory>
<outputDirectory>${project.basedir}/target/generated-sources/</outputDirectory>
</configuration>
</execution>
</executions>
</plugin>
If you do not configure the sourceDirectory and outputDirectory properties, the defaults will be used. The sourceDirectory property defaults to src/main/avro. The outputDirectory property defaults to target/generated-sources. You can change the paths to match your project layout.
Alternatively, Avro jars can be downloaded directly from the Apache Avro™ Releases page. The relevant Avro jars for this guide are avro-1.9.0.jar and avro-mapred-1.9.0.jar, as well as avro-tools-1.9.0.jar for code generation and viewing Avro data files as JSON. In addition, you will need to install Hadoop in order to use MapReduce.
Example: ColorCount
Below is a simple example of a MapReduce that uses Avro. There is an example for both the old (org.apache.hadoop.mapred) and new (org.apache.hadoop.mapreduce) APIs under examples/mr-example/src/main/java/example/. MapredColorCount is the example for the older mapred API while MapReduceColorCount is the example for the newer mapreduce API. Both examples are below, but we will detail the mapred API in our subsequent examples.
MapredColorCount:
package example;
import java.io.IOException;
import org.apache.avro.*;
import org.apache.avro.Schema.Type;
import org.apache.avro.mapred.*;
import org.apache.hadoop.conf.*;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.mapred.*;
import org.apache.hadoop.util.*;
import example.avro.User;
public class MapredColorCount extends Configured implements Tool {
public static class ColorCountMapper extends AvroMapper<User, Pair<CharSequence, Integer>> {
@Override
public void map(User user, AvroCollector<Pair<CharSequence, Integer>> collector, Reporter reporter)
throws IOException {
CharSequence color = user.getFavoriteColor();
// We need this check because the User.favorite_color field has type ["string", "null"]
if (color == null) {
color = "none";
}
collector.collect(new Pair<CharSequence, Integer>(color, 1));
}
}
public static class ColorCountReducer extends AvroReducer<CharSequence, Integer,
Pair<CharSequence, Integer>> {
@Override
public void reduce(CharSequence key, Iterable<Integer> values,
AvroCollector<Pair<CharSequence, Integer>> collector,
Reporter reporter)
throws IOException {
int sum = 0;
for (Integer value : values) {
sum += value;
}
collector.collect(new Pair<CharSequence, Integer>(key, sum));
}
}
public int run(String[] args) throws Exception {
if (args.length != 2) {
System.err.println("Usage: MapredColorCount <input path> <output path>");
return -1;
}
JobConf conf = new JobConf(getConf(), MapredColorCount.class);
conf.setJobName("colorcount");
FileInputFormat.setInputPaths(conf, new Path(args[0]));
FileOutputFormat.setOutputPath(conf, new Path(args[1]));
AvroJob.setMapperClass(conf, ColorCountMapper.class);
AvroJob.setReducerClass(conf, ColorCountReducer.class);
// Note that AvroJob.setInputSchema and AvroJob.setOutputSchema set
// relevant config options such as input/output format, map output
// classes, and output key class.
AvroJob.setInputSchema(conf, User.getClassSchema());
AvroJob.setOutputSchema(conf, Pair.getPairSchema(Schema.create(Type.STRING),
Schema.create(Type.INT)));
JobClient.runJob(conf);
return 0;
}
public static void main(String[] args) throws Exception {
int res = ToolRunner.run(new Configuration(), new MapredColorCount(), args);
System.exit(res);
}
}
MapReduceColorCount:
package example;
import java.io.IOException;
import org.apache.avro.Schema;
import org.apache.avro.mapred.AvroKey;
import org.apache.avro.mapred.AvroValue;
import org.apache.avro.mapreduce.AvroJob;
import org.apache.avro.mapreduce.AvroKeyInputFormat;
import org.apache.avro.mapreduce.AvroKeyValueOutputFormat;
import org.apache.hadoop.conf.Configured;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.IntWritable;
import org.apache.hadoop.io.NullWritable;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.mapreduce.Job;
import org.apache.hadoop.mapreduce.Mapper;
import org.apache.hadoop.mapreduce.Reducer;
import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
import org.apache.hadoop.util.Tool;
import org.apache.hadoop.util.ToolRunner;
import example.avro.User;
public class MapReduceColorCount extends Configured implements Tool {
public static class ColorCountMapper extends
Mapper<AvroKey<User>, NullWritable, Text, IntWritable> {
@Override
public void map(AvroKey<User> key, NullWritable value, Context context)
throws IOException, InterruptedException {
CharSequence color = key.datum().getFavoriteColor();
if (color == null) {
color = "none";
}
context.write(new Text(color.toString()), new IntWritable(1));
}
}
public static class ColorCountReducer extends
Reducer<Text, IntWritable, AvroKey<CharSequence>, AvroValue<Integer>> {
@Override
public void reduce(Text key, Iterable<IntWritable> values,
Context context) throws IOException, InterruptedException {
int sum = 0;
for (IntWritable value : values) {
sum += value.get();
}
context.write(new AvroKey<CharSequence>(key.toString()), new AvroValue<Integer>(sum));
}
}
public int run(String[] args) throws Exception {
if (args.length != 2) {
System.err.println("Usage: MapReduceColorCount <input path> <output path>");
return -1;
}
Job job = new Job(getConf());
job.setJarByClass(MapReduceColorCount.class);
job.setJobName("Color Count");
FileInputFormat.setInputPaths(job, new Path(args[0]));
FileOutputFormat.setOutputPath(job, new Path(args[1]));
job.setInputFormatClass(AvroKeyInputFormat.class);
job.setMapperClass(ColorCountMapper.class);
AvroJob.setInputKeySchema(job, User.getClassSchema());
job.setMapOutputKeyClass(Text.class);
job.setMapOutputValueClass(IntWritable.class);
job.setOutputFormatClass(AvroKeyValueOutputFormat.class);
job.setReducerClass(ColorCountReducer.class);
AvroJob.setOutputKeySchema(job, Schema.create(Schema.Type.STRING));
AvroJob.setOutputValueSchema(job, Schema.create(Schema.Type.INT));
return (job.waitForCompletion(true) ? 0 : 1);
}
public static void main(String[] args) throws Exception {
int res = ToolRunner.run(new MapReduceColorCount(), args);
System.exit(res);
}
}
ColorCount reads in data files containing User records, defined in examples/user.avsc, and counts the number of instances of each favorite color. (This example draws inspiration from the canonical WordCount MapReduce application.) This example uses the old MapReduce API. See MapReduceAvroWordCount, found under doc/examples/mr-example/src/main/java/example/ to see the new MapReduce API example. The User schema is defined as follows:
{"namespace": "example.avro",
"type": "record",
"name": "User",
"fields": [
{"name": "name", "type": "string"},
{"name": "favorite_number", "type": ["int", "null"]},
{"name": "favorite_color", "type": ["string", "null"]}
]
}
This schema is compiled into the User class used by ColorCount via the Avro Maven plugin (see examples/mr-example/pom.xml for how this is set up).
ColorCountMapper essentially takes a User as input and extracts the User’s favorite color, emitting the key-value pair <favoriteColor, 1>. ColorCountReducer then adds up how many occurrences of a particular favorite color were emitted, and outputs the result as a Pair record. These Pairs are serialized to an Avro data file.
Running ColorCount
The ColorCount application is provided as a Maven project in the Avro docs under examples/mr-example. To build the project, including the code generation of the User schema, run:
Next, run GenerateData from examples/mr-examples to create an Avro data file, input/users.avro, containing 20 Users with favorite colors chosen randomly from a list:
mvn exec:java -q -Dexec.mainClass=example.GenerateData
Besides creating the data file, GenerateData prints the JSON representations of the Users generated to stdout, for example:
{"name": "user", "favorite_number": null, "favorite_color": "red"}
{"name": "user", "favorite_number": null, "favorite_color": "green"}
{"name": "user", "favorite_number": null, "favorite_color": "purple"}
{"name": "user", "favorite_number": null, "favorite_color": null}
...
Now we’re ready to run ColorCount. We specify our freshly-generated input folder as the input path and output as our output folder (note that MapReduce will not start a job if the output folder already exists):
mvn exec:java -q -Dexec.mainClass=example.MapredColorCount -Dexec.args="input output"
Once ColorCount completes, checking the contents of the new output directory should yield the following:
$ ls output/
part-00000.avro _SUCCESS
You can check the contents of the generated Avro file using the avro-tools jar:
$ java -jar /path/to/avro-tools-1.9.0.jar tojson output/part-00000.avro
{"value": 3, "key": "blue"}
{"value": 7, "key": "green"}
{"value": 1, "key": "none"}
{"value": 2, "key": "orange"}
{"value": 3, "key": "purple"}
{"value": 2, "key": "red"}
{"value": 2, "key": "yellow"}
Now let’s go over the ColorCount example in detail.
Mapper - org.apache.hadoop.mapred API
The easiest way to use Avro data files as input to a MapReduce job is to subclass AvroMapper. An AvroMapper defines a map function that takes an Avro datum as input and outputs a key/value pair represented as a Pair record. In the ColorCount example, ColorCountMapper is an AvroMapper that takes a User as input and outputs a Pair<CharSequence, Integer>>, where the CharSequence key is the user’s favorite color and the Integer value is 1.
public static class ColorCountMapper extends AvroMapper<User, Pair<CharSequence, Integer>> {
@Override
public void map(User user, AvroCollector<Pair<CharSequence, Integer>> collector, Reporter reporter)
throws IOException {
CharSequence color = user.getFavoriteColor();
// We need this check because the User.favorite_color field has type ["string", "null"]
if (color == null) {
color = "none";
}
collector.collect(new Pair<CharSequence, Integer>(color, 1));
}
}
In order to use our AvroMapper, we must call AvroJob.setMapperClass and AvroJob.setInputSchema.
AvroJob.setMapperClass(conf, ColorCountMapper.class);
AvroJob.setInputSchema(conf, User.getClassSchema());
Note that AvroMapper does not implement the Mapper interface. Under the hood, the specified Avro data files are deserialized into AvroWrappers containing the actual data, which are processed by a Mapper that calls the configured AvroMapper’s map function. AvroJob.setInputSchema sets up the relevant configuration parameters needed to make this happen, thus you should not need to call JobConf.setMapperClass, JobConf.setInputFormat, JobConf.setMapOutputKeyClass, JobConf.setMapOutputValueClass, or JobConf.setOutputKeyComparatorClass.
Mapper - org.apache.hadoop.mapreduce API
This document will not go into all the differences between the mapred and mapreduce APIs, however will describe the main differences. As you can see, ColorCountMapper is now a subclass of the Hadoop Mapper class and is passed an AvroKey as it’s key. Additionally, the AvroJob method calls were slightly changed.
public static class ColorCountMapper extends
Mapper<AvroKey<User>, NullWritable, Text, IntWritable> {
@Override
public void map(AvroKey<User> key, NullWritable value, Context context)
throws IOException, InterruptedException {
CharSequence color = key.datum().getFavoriteColor();
if (color == null) {
color = "none";
}
context.write(new Text(color.toString()), new IntWritable(1));
}
}
Reducer - org.apache.hadoop.mapred API
Analogously to AvroMapper, an AvroReducer defines a reducer function that takes the key/value types output by an AvroMapper (or any mapper that outputs Pairs) and outputs a key/value pair represented a Pair record. In the ColorCount example, ColorCountReducer is an AvroReducer that takes the CharSequence key representing a favorite color and the Iterable<Integer> representing the counts for that color (they should all be 1 in this example) and adds up the counts.
public static class ColorCountReducer extends AvroReducer<CharSequence, Integer,
Pair<CharSequence, Integer>> {
@Override
public void reduce(CharSequence key, Iterable<Integer> values,
AvroCollector<Pair<CharSequence, Integer>> collector,
Reporter reporter)
throws IOException {
int sum = 0;
for (Integer value : values) {
sum += value;
}
collector.collect(new Pair<CharSequence, Integer>(key, sum));
}
}
In order to use our AvroReducer, we must call AvroJob.setReducerClass and AvroJob.setOutputSchema.
AvroJob.setReducerClass(conf, ColorCountReducer.class);
AvroJob.setOutputSchema(conf, Pair.getPairSchema(Schema.create(Type.STRING),
Schema.create(Type.INT)));
Note that AvroReducer does not implement the Reducer interface. The intermediate Pairs output by the mapper are split into AvroKeys and AvroValues, which are processed by a Reducer that calls the configured AvroReducer’s reduce function. AvroJob.setOutputSchema sets up the relevant configuration parameters needed to make this happen, thus you should not need to call JobConf.setReducerClass, JobConf.setOutputFormat, JobConf.setOutputKeyClass, JobConf.setMapOutputKeyClass, JobConf.setMapOutputValueClass, or JobConf.setOutputKeyComparatorClass.
Reduce - org.apache.hadoop.mapreduce API
As before we not detail every difference between the APIs. As with the Mapper change ColorCountReducer is now a subclass of Reducer and AvroKey and AvroValue are emitted. Additionally, the AvroJob method calls were slightly changed.
public static class ColorCountReducer extends
Reducer<Text, IntWritable, AvroKey<CharSequence>, AvroValue<Integer>> {
@Override
public void reduce(Text key, Iterable<IntWritable> values,
Context context) throws IOException, InterruptedException {
int sum = 0;
for (IntWritable value : values) {
sum += value.get();
}
context.write(new AvroKey<CharSequence>(key.toString()), new AvroValue<Integer>(sum));
}
}
Learning more
The mapred API allows users to mix Avro AvroMappers and AvroReducers with non-Avro Mappers and Reducers and the mapreduce API allows users input Avro and output non-Avro or vice versa.
The mapred package has API org.apache.avro.mapred documentation as does the org.apache.avro.mapreduce package. MapReduce API (org.apache.hadoop.mapreduce). Similarily to the mapreduce package, it’s possible with the mapred API to implement your own Mappers and Reducers directly using the public classes provided in these libraries. See the AvroWordCount application, found under examples/mr-example/src/main/java/example/AvroWordCount.java in the Avro documentation, for an example of implementing a Reducer that outputs Avro data using the old MapReduce API. See the MapReduceAvroWordCount application, found under examples/mr-example/src/main/java/example/MapReduceAvroWordCount.java in the Avro documentation, for an example of implementing a Reducer that outputs Avro data using the new MapReduce API.
8.6 - IDL Language
Introduction
This document defines Avro IDL, a higher-level language for authoring Avro schemata. Before reading this document, you should have familiarity with the concepts of schemata and protocols, as well as the various primitive and complex types available in Avro.
Overview
Purpose
The aim of the Avro IDL language is to enable developers to author schemata in a way that feels more similar to common programming languages like Java, C++, or Python. Additionally, the Avro IDL language may feel more familiar for those users who have previously used the interface description languages (IDLs) in other frameworks like Thrift, Protocol Buffers, or CORBA.
Usage
Each Avro IDL file defines a single Avro Protocol, and thus generates as its output a JSON-format Avro Protocol file with extension .avpr.
To convert a .avdl file into a .avpr file, it may be processed by the idl tool. For example:
$ java -jar avroj-tools.jar idl src/test/idl/input/namespaces.avdl /tmp/namespaces.avpr
$ head /tmp/namespaces.avpr
{
"protocol" : "TestNamespace",
"namespace" : "avro.test.protocol",
The idl tool can also process input to and from stdin and stdout. See idl --help for full usage information.
A Maven plugin is also provided to compile .avdl files. To use it, add something like the following to your pom.xml:
<build>
<plugins>
<plugin>
<groupId>org.apache.avro</groupId>
<artifactId>avro-maven-plugin</artifactId>
<executions>
<execution>
<goals>
<goal>idl-protocol</goal>
</goals>
</execution>
</executions>
</plugin>
</plugins>
</build>
Defining a Protocol in Avro IDL
An Avro IDL file consists of exactly one protocol definition. The minimal protocol is defined by the following code:
This is equivalent to (and generates) the following JSON protocol definition:
{
"protocol" : "MyProtocol",
"types" : [ ],
"messages" : {
}
}
The namespace of the protocol may be changed using the @namespace annotation:
@namespace("mynamespace")
protocol MyProtocol {
}
This notation is used throughout Avro IDL as a way of specifying properties for the annotated element, as will be described later in this document.
Protocols in Avro IDL can contain the following items:
- Imports of external protocol and schema files.
- Definitions of named schemata, including records, errors, enums, and fixeds.
- Definitions of RPC messages
Imports
Files may be imported in one of three formats:
-
An IDL file may be imported with a statement like:
import idl "foo.avdl";
-
A JSON protocol file may be imported with a statement like:
import protocol "foo.avpr";
-
A JSON schema file may be imported with a statement like:
import schema "foo.avsc";
Messages and types in the imported file are added to this file’s protocol.
Imported file names are resolved relative to the current IDL file.
Defining an Enumeration
Enums are defined in Avro IDL using a syntax similar to C or Java. An Avro Enum supports optional default values. In the case that a reader schema is unable to recognize a symbol written by the writer, the reader will fall back to using the defined default value. This default is only used when an incompatible symbol is read. It is not used if the enum field is missing.
Example Writer Enum Definition
enum Shapes {
SQUARE, TRIANGLE, CIRCLE, OVAL
}
Example Reader Enum Definition
enum Shapes {
SQUARE, TRIANGLE, CIRCLE
} = CIRCLE;
In the above example, the reader will use the default value of CIRCLE whenever reading data written with the OVAL symbol of the writer. Also note that, unlike the JSON format, anonymous enums cannot be defined.
Defining a Fixed Length Field
Fixed fields are defined using the following syntax:
fixed MD5(16);
This example defines a fixed-length type called MD5 which contains 16 bytes.
Defining Records and Errors
Records are defined in Avro IDL using a syntax similar to a struct definition in C:
record Employee {
string name;
boolean active = true;
long salary;
}
The above example defines a record with the name “Employee” with three fields.
To define an error, simply use the keyword error instead of record. For example:
error Kaboom {
string explanation;
int result_code = -1;
}
Each field in a record or error consists of a type and a name, optional property annotations and an optional default value.
A type reference in Avro IDL must be one of:
- A primitive type
- A logical type
- A named schema defined prior to this usage in the same Protocol
- A complex type (array, map, or union)
Primitive Types
The primitive types supported by Avro IDL are the same as those supported by Avro’s JSON format. This list includes int, long, string, boolean, float, double, null, and bytes.
Logical Types
Some of the logical types supported by Avro’s JSON format are also supported by Avro IDL. The currently supported types are:
For example:
record Job {
string jobid;
date submitDate;
time_ms submitTime;
timestamp_ms finishTime;
decimal(9,2) finishRatio;
}
References to Named Schemata
If a named schema has already been defined in the same Avro IDL file, it may be referenced by name as if it were a primitive type:
record Card {
Suit suit; // refers to the enum Card defined above
int number;
}
Default Values
Default values for fields may be optionally specified by using an equals sign after the field name followed by a JSON expression indicating the default value. This JSON is interpreted as described in the spec.
Complex Types
Arrays
Array types are written in a manner that will seem familiar to C++ or Java programmers. An array of any type t is denoted array<t>. For example, an array of strings is denoted array<string>, and a multidimensional array of Foo records would be array<array<Foo>>.
Maps
Map types are written similarly to array types. An array that contains values of type t is written map<t>. As in the JSON schema format, all maps contain string-type keys.
Unions
Union types are denoted as union { typeA, typeB, typeC, … }. For example, this record contains a string field that is optional (unioned with null):
record RecordWithUnion {
union { null, string } optionalString;
}
Note that the same restrictions apply to Avro IDL unions as apply to unions defined in the JSON format; namely, a record may not contain multiple elements of the same type.
Defining RPC Messages
The syntax to define an RPC message within a Avro IDL protocol is similar to the syntax for a method declaration within a C header file or a Java interface. To define an RPC message add which takes two arguments named foo and bar, returning an int, simply include the following definition within the protocol:
int add(int foo, int bar = 0);
Message arguments, like record fields, may specify default values.
To define a message with no response, you may use the alias void, equivalent to the Avro null type:
void logMessage(string message);
If you have previously defined an error type within the same protocol, you may declare that a message can throw this error using the syntax:
void goKaboom() throws Kaboom;
To define a one-way message, use the keyword oneway after the parameter list, for example:
void fireAndForget(string message) oneway;
Other Language Features
All Java-style comments are supported within a Avro IDL file. Any text following // on a line is ignored, as is any text between /* and */, possibly spanning multiple lines.
Comments that begin with /** are used as the documentation string for the type or field definition that follows the comment.
Escaping Identifiers
Occasionally, one will need to use a reserved language keyword as an identifier. In order to do so, backticks (`) may be used to escape the identifier. For example, to define a message with the literal name error, you may write:
This syntax is allowed anywhere an identifier is expected.
Annotations for Ordering and Namespaces
Java-style annotations may be used to add additional properties to types and fields throughout Avro IDL.
For example, to specify the sort order of a field within a record, one may use the @order annotation before the field name as follows:
record MyRecord {
string @order("ascending") myAscendingSortField;
string @order("descending") myDescendingField;
string @order("ignore") myIgnoredField;
}
A field’s type may also be preceded by annotations, e.g.:
record MyRecord {
@java-class("java.util.ArrayList") array<string> myStrings;
}
This can be used to support java classes that can be serialized/deserialized via their toString/String constructor, e.g.:
record MyRecord {
@java-class("java.math.BigDecimal") string value;
@java-key-class("java.io.File") map<string> fileStates;
array<@java-class("java.math.BigDecimal") string> weights;
}
Similarly, a @namespace annotation may be used to modify the namespace when defining a named schema. For example:
@namespace("org.apache.avro.firstNamespace")
protocol MyProto {
@namespace("org.apache.avro.someOtherNamespace")
record Foo {}
record Bar {}
}
will define a protocol in the firstNamespace namespace. The record Foo will be defined in someOtherNamespace and Bar will be defined in firstNamespace as it inherits its default from its container.
Type and field aliases are specified with the @aliases annotation as follows:
@aliases(["org.old.OldRecord", "org.ancient.AncientRecord"])
record MyRecord {
string @aliases(["oldField", "ancientField"]) myNewField;
}
Some annotations like those listed above are handled specially. All other annotations are added as properties to the protocol, message, schema or field.
Complete Example
The following is a complete example of a Avro IDL file that shows most of the above features:
/**
* An example protocol in Avro IDL
*/
@namespace("org.apache.avro.test")
protocol Simple {
@aliases(["org.foo.KindOf"])
enum Kind {
FOO,
BAR, // the bar enum value
BAZ
}
fixed MD5(16);
record TestRecord {
@order("ignore")
string name;
@order("descending")
Kind kind;
MD5 hash;
union { MD5, null} @aliases(["hash"]) nullableHash;
array<long> arrayOfLongs;
}
error TestError {
string message;
}
string hello(string greeting);
TestRecord echo(TestRecord `record`);
int add(int arg1, int arg2);
bytes echoBytes(bytes data);
void `error`() throws TestError;
void ping() oneway;
}
Additional examples may be found in the Avro source tree under the src/test/idl/input directory.
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
8.7 - SASL profile
Introduction
SASL (RFC 2222) provides a framework for authentication and security of network protocols. Each protocol that uses SASL is meant to define a SASL profile. This document provides a SASL profile for connection-based Avro RPC.
Overview
SASL negotiation proceeds as a series of message interactions over a connection between a client and server using a selected SASL mechanism. The client starts this negotiation by sending its chosen mechanism name with an initial (possibly empty) message. Negotiation proceeds with the exchange of messages until either side indicates success or failure. The content of the messages is mechanism-specific. If the negotiation succeeds, then the session can proceed over the connection, otherwise it must be abandoned.
Some mechanisms continue to process session data after negotiation (e.g., encrypting it), while some specify that further session data is transmitted unmodified.
Negotiation
Commands
Avro SASL negotiation uses four one-byte commands.
- 0: START Used in a client’s initial message.
- 1: CONTINUE Used while negotiation is ongoing.
- 2: FAIL Terminates negotiation unsuccessfully.
- 3: COMPLETE Terminates negotiation successfully.
The format of a START message is:
| 0 | 4-byte mechanism name length | mechanism name | 4-byte payload length | payload data |
The format of a CONTINUE message is:
| 1 | 4-byte payload length | payload data |
The format of a FAIL message is:
| 2 | 4-byte message length | UTF-8 message |
The format of a COMPLETE message is:
| 3 | 4-byte payload length | payload data |
Process
Negotiation is initiated by a client sending a START command containing the client’s chosen mechanism name and any mechanism-specific payload data.
The server and client then interchange some number (possibly zero) of CONTINUE messages. Each message contains payload data that is processed by the security mechanism to generate the next message.
Once either the client or server send a FAIL message then negotiation has failed. UTF-8-encoded text is included in the failure message. Once either a FAIL message has been sent or received, or any other error occurs in the negotiation, further communication on this connection must cease.
Once either the client or server send a COMPLETE message then negotiation has completed successfully. Session data may now be transmitted over the connection until it is closed by either side.
Session Data
If no SASL QOP (quality of protection) is negotiated, then all subsequent writes to/reads over this connection are written/read unmodified. In particular, messages use Avro framing, and are of the form:
| 4-byte frame length | frame data | ... | 4 zero bytes |
If a SASL QOP is negotiated, then it must be used by the connection for all subsequent messages. This is done by wrapping each non-empty frame written using the security mechanism and unwrapping each non-empty frame read. The length written in each non-empty frame is the length of the wrapped data. Complete frames must be passed to the security mechanism for unwrapping. Unwrapped data is then passed to the application as the content of the frame.
If at any point processing fails due to wrapping, unwrapping or framing errors, then all further communication on this connection must cease.
Anonymous Mechanism
The SASL anonymous mechanism (RFC 2245) is quite simple to implement. In particular, an initial anonymous request may be prefixed by the following static sequence:
| 0 | 0009 | ANONYMOUS | 0000 |
If a server uses the anonymous mechanism, it should check that the mechanism name in the start message prefixing the first request received is ‘ANONYMOUS’, then simply prefix its initial response with a COMPLETE message of:
| 3 | 0000 |
If an anonymous server recieves some other mechanism name, then it may respond with a FAIL message as simple as:
| 2 | 0000 |
Note that the anonymous mechanism need add no additional round-trip messages between client and server. The START message can be piggybacked on the initial request and the COMPLETE or FAIL message can be piggybacked on the initial response.
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
8.8 - Wiki
The Wiki page can be found here.
8.9 - FAQ
What is Avro?
Avro is a data serialization system.
Avro provides:
- Rich data structures.
- A compact, fast, binary data format.
- A container file, to store persistent data.
- Remote procedure call (RPC).
- Simple integration with dynamic languages. Code generation is not required to read or write data files nor to use or implement RPC protocols. Code generation as an optional optimization, only worth implementing for statically typed languages.
How can I get started quickly with Avro?
Check out the Quick Start Guide.
How do I statically compile a schema or protocol into generated code?
In Java
- Add the avro jar, the jackson-mapper-asl.jar and jackson-core-asl.jar to your CLASSPATH.
- Run java org.apache.avro.specific.SpecificCompiler .
This appears to be out of date, the SpecificCompiler requires two arguments, presumably an input and and output file, but it isn’t clear that this does.
Or use the Schema or Protocol Ant tasks. Avro’s build.xml provides examples of how these are used.
Lastly, you can also use the “avro-tools” jar which ships with an Avro release. Just use the “compile (schema|protocol)” command.
How are Strings represented in Java?
They use org.apache.avro.util.Utf8, not java.lang.String.
More generally, how do Avro types map to Java types?
The mappings are documented in the package javadoc for generic, specific and reflect API.
From Doug Cutting:
HDFS splits files into blocks, and mapreduce runs a map task for each block. When the task starts, it needs to be able to seek into the file to the start of the block process through the block’s end. If the file were, e.g., a gzip file, this would not be possible, since gzip files must be decompressed from the start. One cannot seek into the middle of a gzip file and start decompressing. So Hadoop’s SequenceFile places a marker periodically (~64k) in the file at record and compression boundaries, where processing can be sensibly started. Then, when a map task starts processing an HDFS block, it finds the first marker after the block’s start and continues through the first marker in the next block of the file. This requires a bit of non-local access (~0.1%). Avro’s data file uses the same method as SequenceFile.
Why isn’t every value in Avro nullable?
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 of every value stored and
corresponding computational costs to create this bit on write and
interpret it on read. These costs are wasted when values may not in
fact be null, as is the case in many datasets. In Avro such costs are
only paid when values may actually be null.
Also, allowing values to be null is a well-known source of errors. In
Avro, a value declared as non-null will always be non-null and programs
need not test for null values when processing it nor will they ever fail
for lack of such tests.
Tony Hoare calls his invention of null references his “Billion Dollar
Mistake”.
http://qconlondon.com/london-2009/presentation/Null+References:+The+Billion+Dollar+Mistake
Also note that in some programming languages not all values are permitted to be null. For example, in Java, values of type boolean, byte, short, char, int, float, long, and double may not be null.
How can I serialize directly to/from a byte array?
As pointed out in the specification, Avro data should always be stored with its schema. The Avro provided classes DataFileWriter, DataFileReader, and DataFileStream all ensure this by serializing the Schema in a container header. In some special cases, such as when implementing a new storage system or writing unit tests, you may need to write and read directly with the bare Avro serialized values. The following examples use code generated for Java from an Avro Schema, specifically the User example from the Quickstart guide.
Serializing to a byte array
This example takes a User object and returns a newly allocated byte array with the Avro serialization of that user.
ByteArrayOutputStream out = new ByteArrayOutputStream();
BinaryEncoder encoder = EncoderFactory.get().binaryEncoder(out, null);
DatumWriter<User> writer = new SpecificDatumWriter<User>(User.getClassSchema());
writer.write(user, encoder);
encoder.flush();
out.close();
byte[] serializedBytes = out.toByteArray();
Deserializing from a byte array
This example takes a byte array containing the Avro serialization of a user and returns a User object.SpecificDatumReader reader = new SpecificDatumReader(User.getClassSchema());
Decoder decoder = DecoderFactory.get().binaryDecoder(bytes, null);
User user = reader.read(null, decoder);
Note: If you are serializing or deserializing in a loop or as a method, you should be reusing objects, readers and/or writers. Check the JavaDoc to see how to reuse the objects.
9 - Apache Avro™ 1.8.2 Documentation
Introduction
Apache Avro™ is a data serialization system.
Avro provides:
- Rich data structures.
- A compact, fast, binary data format.
- A container file, to store persistent data.
- Remote procedure call (RPC).
- Simple integration with dynamic languages. Code generation is not required to read or write data files nor to use or implement RPC protocols. Code generation as an optional optimization, only worth implementing for statically typed languages.
Schemas
Avro relies on schemas. When Avro data is read, the schema used when writing it is always present. This permits each datum to be written with no per-value overheads, making serialization both fast and small. This also facilitates use with dynamic, scripting languages, since data, together with its schema, is fully self-describing.
When Avro data is stored in a file, its schema is stored with it, so that files may be processed later by any program. If the program reading the data expects a different schema this can be easily resolved, since both schemas are present.
When Avro is used in RPC, the client and server exchange schemas in the connection handshake. (This can be optimized so that, for most calls, no schemas are actually transmitted.) Since both client and server both have the other’s full schema, correspondence between same named fields, missing fields, extra fields, etc. can all be easily resolved.
Avro schemas are defined with JSON . This facilitates implementation in languages that already have JSON libraries.
Comparison with other systems
Avro provides functionality similar to systems such as Thrift, Protocol Buffers, etc. Avro differs from these systems in the following fundamental aspects.
- Dynamic typing: Avro does not require that code be generated. Data is always accompanied by a schema that permits full processing of that data without code generation, static datatypes, etc. This facilitates construction of generic data-processing systems and languages.
- Untagged data: Since the schema is present when data is read, considerably less type information need be encoded with data, resulting in smaller serialization size.
- No manually-assigned field IDs: When a schema changes, both the old and new schema are always present when processing data, so differences may be resolved symbolically, using field names.
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
9.1 - Getting Started (Java)
This is a short guide for getting started with Apache Avro™ using Java. This guide only covers using Avro for data serialization; see Patrick Hunt’s Avro RPC Quick Start for a good introduction to using Avro for RPC.
Download
Avro implementations for C, C++, C#, Java, PHP, Python, and Ruby can be downloaded from the Apache Avro™ Releases page. This guide uses Avro 1.8.2, the latest version at the time of writing. For the examples in this guide, download avro-1.8.2.jar and avro-tools-1.8.2.jar. The Avro Java implementation also depends on the Jackson JSON library. From the Jackson download page, download the core-asl and mapper-asl jars. Add avro-1.8.2.jar and the Jackson jars to your project’s classpath (avro-tools will be used for code generation).
Alternatively, if you are using Maven, add the following dependency to your POM:
<dependency>
<groupId>org.apache.avro</groupId>
<artifactId>avro</artifactId>
<version>1.8.2</version>
</dependency>
As well as the Avro Maven plugin (for performing code generation):
<plugin>
<groupId>org.apache.avro</groupId>
<artifactId>avro-maven-plugin</artifactId>
<version>1.8.2</version>
<executions>
<execution>
<phase>generate-sources</phase>
<goals>
<goal>schema</goal>
</goals>
<configuration>
<sourceDirectory>${project.basedir}/src/main/avro/</sourceDirectory>
<outputDirectory>${project.basedir}/src/main/java/</outputDirectory>
</configuration>
</execution>
</executions>
</plugin>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-compiler-plugin</artifactId>
<configuration>
<source>1.6</source>
<target>1.6</target>
</configuration>
</plugin>
You may also build the required Avro jars from source. Building Avro is beyond the scope of this guide; see the Build Documentation page in the wiki for more information.
Defining a schema
Avro schemas are defined using JSON. Schemas are composed of primitive types (null, boolean, int, long, float, double, bytes, and string) and complex types (record, enum, array, map, union, and fixed). You can learn more about Avro schemas and types from the specification, but for now let’s start with a simple schema example, user.avsc:
{"namespace": "example.avro",
"type": "record",
"name": "User",
"fields": [
{"name": "name", "type": "string"},
{"name": "favorite_number", "type": ["int", "null"]},
{"name": "favorite_color", "type": ["string", "null"]}
]
}
This schema defines a record representing a hypothetical user. (Note that a schema file can only contain a single schema definition.) At minimum, a record definition must include its type (“type”: “record”), a name (“name”: “User”), and fields, in this case name, favorite_number, and favorite_color. We also define a namespace (“namespace”: “example.avro”), which together with the name attribute defines the “full name” of the schema (example.avro.User in this case).
Fields are defined via an array of objects, each of which defines a name and type (other attributes are optional, see the record specification for more details). The type attribute of a field is another schema object, which can be either a primitive or complex type. For example, the name field of our User schema is the primitive type string, whereas the favorite_number and favorite_color fields are both unions, represented by JSON arrays. unions are a complex type that can be any of the types listed in the array; e.g., favorite_number can either be an int or null, essentially making it an optional field.
Serializing and deserializing with code generation
Compiling the schema
Code generation allows us to automatically create classes based on our previously-defined schema. Once we have defined the relevant classes, there is no need to use the schema directly in our programs. We use the avro-tools jar to generate code as follows:
java -jar /path/to/avro-tools-1.8.2.jar compile schema <schema file> <destination>
This will generate the appropriate source files in a package based on the schema’s namespace in the provided destination folder. For instance, to generate a User class in package example.avro from the schema defined above, run
java -jar /path/to/avro-tools-1.8.2.jar compile schema user.avsc .
Note that if you using the Avro Maven plugin, there is no need to manually invoke the schema compiler; the plugin automatically performs code generation on any .avsc files present in the configured source directory.
Creating Users
Now that we’ve completed the code generation, let’s create some Users, serialize them to a data file on disk, and then read back the file and deserialize the User objects.
First let’s create some Users and set their fields.
User user1 = new User();
user1.setName("Alyssa");
user1.setFavoriteNumber(256);
// Leave favorite color null
// Alternate constructor
User user2 = new User("Ben", 7, "red");
// Construct via builder
User user3 = User.newBuilder()
.setName("Charlie")
.setFavoriteColor("blue")
.setFavoriteNumber(null)
.build();
As shown in this example, Avro objects can be created either by invoking a constructor directly or by using a builder. Unlike constructors, builders will automatically set any default values specified in the schema. Additionally, builders validate the data as it set, whereas objects constructed directly will not cause an error until the object is serialized. However, using constructors directly generally offers better performance, as builders create a copy of the datastructure before it is written.
Note that we do not set user1’s favorite color. Since that record is of type [“string”, “null”], we can either set it to a string or leave it null; it is essentially optional. Similarly, we set user3’s favorite number to null (using a builder requires setting all fields, even if they are null).
Serializing
Now let’s serialize our Users to disk.
/// Serialize user1, user2 and user3 to disk
DatumWriter<User> userDatumWriter = new SpecificDatumWriter<User>(User.class);
DataFileWriter<User> dataFileWriter = new DataFileWriter<User>(userDatumWriter);
dataFileWriter.create(user1.getSchema(), new File("users.avro"));
dataFileWriter.append(user1);
dataFileWriter.append(user2);
dataFileWriter.append(user3);
dataFileWriter.close();
We create a DatumWriter, which converts Java objects into an in-memory serialized format. The SpecificDatumWriter class is used with generated classes and extracts the schema from the specified generated type.
Next we create a DataFileWriter, which writes the serialized records, as well as the schema, to the file specified in the dataFileWriter.create call. We write our users to the file via calls to the dataFileWriter.append method. When we are done writing, we close the data file.
Deserializing
Finally, let’s deserialize the data file we just created.
// Deserialize Users from disk
DatumReader<User> userDatumReader = new SpecificDatumReader<User>(User.class);
DataFileReader<User> dataFileReader = new DataFileReader<User>(file, userDatumReader);
User user = null;
while (dataFileReader.hasNext()) {
// Reuse user object by passing it to next(). This saves us from
// allocating and garbage collecting many objects for files with
// many items.
user = dataFileReader.next(user);
System.out.println(user);
}
This snippet will output:
{"name": "Alyssa", "favorite_number": 256, "favorite_color": null}
{"name": "Ben", "favorite_number": 7, "favorite_color": "red"}
{"name": "Charlie", "favorite_number": null, "favorite_color": "blue"}
Deserializing is very similar to serializing. We create a SpecificDatumReader, analogous to the SpecificDatumWriter we used in serialization, which converts in-memory serialized items into instances of our generated class, in this case User. We pass the DatumReader and the previously created File to a DataFileReader, analogous to the DataFileWriter, which reads the data file on disk.
Next we use the DataFileReader to iterate through the serialized Users and print the deserialized object to stdout. Note how we perform the iteration: we create a single User object which we store the current deserialized user in, and pass this record object to every call of dataFileReader.next. This is a performance optimization that allows the DataFileReader to reuse the same User object rather than allocating a new User for every iteration, which can be very expensive in terms of object allocation and garbage collection if we deserialize a large data file. While this technique is the standard way to iterate through a data file, it’s also possible to use for (User user : dataFileReader) if performance is not a concern.
Compiling and running the example code
This example code is included as a Maven project in the examples/java-example directory in the Avro docs. From this directory, execute the following commands to build and run the example:
$ mvn compile # includes code generation via Avro Maven plugin
$ mvn -q exec:java -Dexec.mainClass=example.SpecificMain
Serializing and deserializing without code generation
Data in Avro is always stored with its corresponding schema, meaning we can always read a serialized item regardless of whether we know the schema ahead of time. This allows us to perform serialization and deserialization without code generation.
Let’s go over the same example as in the previous section, but without using code generation: we’ll create some users, serialize them to a data file on disk, and then read back the file and deserialize the users objects.
Creating users
First, we use a Parser to read our schema definition and create a Schema object.
Schema schema = new Schema.Parser().parse(new File("user.avsc"));
Using this schema, let’s create some users.
GenericRecord user1 = new GenericData.Record(schema);
user1.put("name", "Alyssa");
user1.put("favorite_number", 256);
// Leave favorite color null
GenericRecord user2 = new GenericData.Record(schema);
user2.put("name", "Ben");
user2.put("favorite_number", 7);
user2.put("favorite_color", "red");
Since we’re not using code generation, we use GenericRecords to represent users. GenericRecord uses the schema to verify that we only specify valid fields. If we try to set a non-existent field (e.g., user1.put(“favorite_animal”, “cat”)), we’ll get an AvroRuntimeException when we run the program.
Note that we do not set user1’s favorite color. Since that record is of type [“string”, “null”], we can either set it to a string or leave it null; it is essentially optional.
Serializing
Now that we’ve created our user objects, serializing and deserializing them is almost identical to the example above which uses code generation. The main difference is that we use generic instead of specific readers and writers.
First we’ll serialize our users to a data file on disk.
// Serialize user1 and user2 to disk
File file = new File("users.avro");
DatumWriter<GenericRecord> datumWriter = new GenericDatumWriter<GenericRecord>(schema);
DataFileWriter<GenericRecord> dataFileWriter = new DataFileWriter<GenericRecord>(datumWriter);
dataFileWriter.create(schema, file);
dataFileWriter.append(user1);
dataFileWriter.append(user2);
dataFileWriter.close();
We create a DatumWriter, which converts Java objects into an in-memory serialized format. Since we are not using code generation, we create a GenericDatumWriter. It requires the schema both to determine how to write the GenericRecords and to verify that all non-nullable fields are present.
As in the code generation example, we also create a DataFileWriter, which writes the serialized records, as well as the schema, to the file specified in the dataFileWriter.create call. We write our users to the file via calls to the dataFileWriter.append method. When we are done writing, we close the data file.
Deserializing
Finally, we’ll deserialize the data file we just created.
// Deserialize users from disk
DatumReader<GenericRecord> datumReader = new GenericDatumReader<GenericRecord>(schema);
DataFileReader<GenericRecord> dataFileReader = new DataFileReader<GenericRecord>(file, datumReader);
GenericRecord user = null;
while (dataFileReader.hasNext()) {
// Reuse user object by passing it to next(). This saves us from
// allocating and garbage collecting many objects for files with
// many items.
user = dataFileReader.next(user);
System.out.println(user);
This outputs:
{"name": "Alyssa", "favorite_number": 256, "favorite_color": null}
{"name": "Ben", "favorite_number": 7, "favorite_color": "red"}
Deserializing is very similar to serializing. We create a GenericDatumReader, analogous to the GenericDatumWriter we used in serialization, which converts in-memory serialized items into GenericRecords. We pass the DatumReader and the previously created File to a DataFileReader, analogous to the DataFileWriter, which reads the data file on disk.
Next, we use the DataFileReader to iterate through the serialized users and print the deserialized object to stdout. Note how we perform the iteration: we create a single GenericRecord object which we store the current deserialized user in, and pass this record object to every call of dataFileReader.next. This is a performance optimization that allows the DataFileReader to reuse the same record object rather than allocating a new GenericRecord for every iteration, which can be very expensive in terms of object allocation and garbage collection if we deserialize a large data file. While this technique is the standard way to iterate through a data file, it’s also possible to use for (GenericRecord user : dataFileReader) if performance is not a concern.
Compiling and running the example code
This example code is included as a Maven project in the examples/java-example directory in the Avro docs. From this directory, execute the following commands to build and run the example:
$ mvn compile
$ mvn -q exec:java -Dexec.mainClass=example.GenericMain
9.2 - Getting Started (Python)
This is a short guide for getting started with Apache Avro™ using Python. This guide only covers using Avro for data serialization; see Patrick Hunt’s Avro RPC Quick Start for a good introduction to using Avro for RPC.
Download
Avro implementations for C, C++, C#, Java, PHP, Python, and Ruby can be downloaded from the Apache Avro™ Releases page. This guide uses Avro 1.8.2, the latest version at the time of writing. Download and unzip avro-1.8.2.tar.gz, and install via python setup.py (this will probably require root privileges). Ensure that you can import avro from a Python prompt.
$ tar xvf avro-1.8.2.tar.gz
$ cd avro-1.8.2
$ sudo python setup.py install
$ python
>>> import avro # should not raise ImportError
Alternatively, you may build the Avro Python library from source. From your the root Avro directory, run the commands
$ cd lang/py/
$ ant
$ sudo python setup.py install
$ python
>>> import avro # should not raise ImportError
Defining a schema
Avro schemas are defined using JSON. Schemas are composed of primitive types (null, boolean, int, long, float, double, bytes, and string) and complex types (record, enum, array, map, union, and fixed). You can learn more about Avro schemas and types from the specification, but for now let’s start with a simple schema example, user.avsc:
{"namespace": "example.avro",
"type": "record",
"name": "User",
"fields": [
{"name": "name", "type": "string"},
{"name": "favorite_number", "type": ["int", "null"]},
{"name": "favorite_color", "type": ["string", "null"]}
]
}
This schema defines a record representing a hypothetical user. (Note that a schema file can only contain a single schema definition.) At minimum, a record definition must include its type (“type”: “record”), a name (“name”: “User”), and fields, in this case name, favorite_number, and favorite_color. We also define a namespace (“namespace”: “example.avro”), which together with the name attribute defines the “full name” of the schema (example.avro.User in this case).
Fields are defined via an array of objects, each of which defines a name and type (other attributes are optional, see the record specification for more details). The type attribute of a field is another schema object, which can be either a primitive or complex type. For example, the name field of our User schema is the primitive type string, whereas the favorite_number and favorite_color fields are both unions, represented by JSON arrays. unions are a complex type that can be any of the types listed in the array; e.g., favorite_number can either be an int or null, essentially making it an optional field.
Serializing and deserializing without code generation
Data in Avro is always stored with its corresponding schema, meaning we can always read a serialized item, regardless of whether we know the schema ahead of time. This allows us to perform serialization and deserialization without code generation. Note that the Avro Python library does not support code generation.
Try running the following code snippet, which serializes two users to a data file on disk, and then reads back and deserializes the data file:
import avro.schema
from avro.datafile import DataFileReader, DataFileWriter
from avro.io import DatumReader, DatumWriter
schema = avro.schema.parse(open("user.avsc", "rb").read())
writer = DataFileWriter(open("users.avro", "wb"), DatumWriter(), schema)
writer.append({"name": "Alyssa", "favorite_number": 256})
writer.append({"name": "Ben", "favorite_number": 7, "favorite_color": "red"})
writer.close()
reader = DataFileReader(open("users.avro", "rb"), DatumReader())
for user in reader:
print user
reader.close()
This outputs:
{u'favorite_color': None, u'favorite_number': 256, u'name': u'Alyssa'}
{u'favorite_color': u'red', u'favorite_number': 7, u'name': u'Ben'}
Do make sure that you open your files in binary mode (i.e. using the modes wb or rb respectively). Otherwise you might generate corrupt files due to automatic replacement of newline characters with the platform-specific representations.
Let’s take a closer look at what’s going on here.
schema = avro.schema.parse(open("user.avsc", "rb").read())
avro.schema.parse takes a string containing a JSON schema definition as input and outputs a avro.schema.Schema object (specifically a subclass of Schema, in this case RecordSchema). We’re passing in the contents of our user.avsc schema file here.
writer = DataFileWriter(open("users.avro", "wb"), DatumWriter(), schema)
We create a DataFileWriter, which we’ll use to write serialized items to a data file on disk. The DataFileWriter constructor takes three arguments:
- The file we’ll serialize to
- A DatumWriter, which is responsible for actually serializing the items to Avro’s binary format (DatumWriters can be used separately from DataFileWriters, e.g., to perform IPC Avro TODO: is this true??).
- The schema we’re using. The DataFileWriter needs the schema both to write the schema to the data file, and to verify that the items we write are valid items and write the appropriate fields.
writer.append({"name": "Alyssa", "favorite_number": 256})
writer.append({"name": "Ben", "favorite_number": 7, "favorite_color": "red"})
We use DataFileWriter.append to add items to our data file. Avro records are represented as Python dicts. Since the field favorite_color has type [“int”, “null”], we are not required to specify this field, as shown in the first append. Were we to omit the required name field, an exception would be raised. Any extra entries not corresponding to a field are present in the dict are ignored.
reader = DataFileReader(open("users.avro", "rb"), DatumReader())
We open the file again, this time for reading back from disk. We use a DataFileReader and DatumReader analagous to the DataFileWriter and DatumWriter above.
for user in reader:
print user
The DataFileReader is an iterator that returns dicts corresponding to the serialized items.
9.3 - Specification
Introduction
This document defines Apache Avro. It is intended to be the authoritative specification. Implementations of Avro must adhere to this document.
Schema Declaration
A Schema is represented in JSON by one of:
- A JSON string, naming a defined type.
- A JSON object, of the form:
{"type": "typeName" ...attributes...}
where typeName is either a primitive or derived type name, as defined below. Attributes not defined in this document are permitted as metadata, but must not affect the format of serialized data.
- A JSON array, representing a union of embedded types.
Primitive Types
The set of primitive type names is:
- null: no value
- boolean: a binary value
- int: 32-bit signed integer
- long: 64-bit signed integer
- float: single precision (32-bit) IEEE 754 floating-point number
- double: double precision (64-bit) IEEE 754 floating-point number
- bytes: sequence of 8-bit unsigned bytes
- string: unicode character sequence
Primitive types have no specified attributes.
Primitive type names are also defined type names. Thus, for example, the schema “string” is equivalent to:
Complex Types
Avro supports six kinds of complex types: records, enums, arrays, maps, unions and fixed.
Records
Records use the type name “record” and support three attributes:
- name: a JSON string providing the name of the record (required).
- namespace, a JSON string that qualifies the name;
- doc: a JSON string providing documentation to the user of this schema (optional).
- aliases: a JSON array of strings, providing alternate names for this record (optional).
- fields: a JSON array, listing fields (required). Each field is a JSON object with the following attributes:
- name: a JSON string providing the name of the field (required), and
- doc: a JSON string describing this field for users (optional).
- type: A JSON object defining a schema, or a JSON string naming a record definition (required).
- _default: A default value for this field, used when reading instances that lack this field (optional). 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. Default values for bytes and fixed fields are JSON strings, where Unicode code points 0-255 are mapped to unsigned 8-bit byte values 0-255.
field default values
| avro type |
json type |
example |
| null |
null |
null |
| boolean |
boolean |
true |
| int,long |
integer |
1 |
| float,double |
number |
1.1 |
| bytes |
string |
"\u00FF" |
| string |
string |
"foo" |
| record |
object |
{"a": 1} |
| enum |
string |
"FOO" |
| array |
array |
[1] |
| map |
object |
{"a": 1} |
| fixed |
string |
"\u00ff" |
- order: specifies how this field impacts sort ordering of this record (optional). Valid values are “ascending” (the default), “descending”, or “ignore”. For more details on how this is used, see the sort order section below.
- aliases: a JSON array of strings, providing alternate names for this field (optional).
For example, a linked-list of 64-bit values may be defined with:
{
"type": "record",
"name": "LongList",
"aliases": ["LinkedLongs"], // old name for this
"fields" : [
{"name": "value", "type": "long"}, // each element has a long
{"name": "next", "type": ["null", "LongList"]} // optional next element
]
}
Enums
Enums use the type name “enum” and support the following attributes:
- name: a JSON string providing the name of the enum (required).
- namespace, a JSON string that qualifies the name ;
- aliases: a JSON array of strings, providing alternate names for this enum (optional).
- doc: a JSON string providing documentation to the user of this schema (optional).
- symbols: a JSON array, listing symbols, as JSON strings (required). All symbols in an enum must be unique; duplicates are prohibited. Every symbol must match the regular expression [A-Za-z_][A-Za-z0-9_]* (the same requirement as for names).
- default: A default value for this enumeration, used during resolution when the reader encounters a symbol from the writer that isn’t defined in the reader’s schema (optional). The value provided here must be a JSON string that’s a member of the symbols array. See documentation on schema resolution for how this gets used.
For example, playing card suits might be defined with:
{
"type": "enum",
"name": "Suit",
"symbols" : ["SPADES", "HEARTS", "DIAMONDS", "CLUBS"]
}
Arrays
Arrays use the type name “array” and support a single attribute:
- items: the schema of the array’s items.
For example, an array of strings is declared with:
{
"type": "array",
"items" : "string"
}
Maps
Maps use the type name “map” and support one attribute:
- values: the schema of the map’s values.
Map keys are assumed to be strings.
For example, a map from string to long is declared with:
{
"type": "map",
"values" : "long"
}
Unions
Unions, as mentioned above, are represented using JSON arrays. For example, ["null", "string"] declares a schema which may be either a null or string.
(Note that when a default value is specified for a record field whose type is a union, the type of the default value must match the first element of the union. Thus, for unions containing “null”, the “null” is usually listed first, since the default value of such unions is typically null.)
Unions may not contain more than one schema with the same type, except for the named types record, fixed and enum. For example, unions containing two array types or two map types are not permitted, but two types with different names are permitted. (Names permit efficient resolution when reading and writing unions.)
Unions may not immediately contain other unions.
Fixed
Fixed uses the type name “fixed” and supports two attributes:
- name: a string naming this fixed (required).
- namespace, a string that qualifies the name;
- aliases: a JSON array of strings, providing alternate names for this enum (optional).
- size: an integer, specifying the number of bytes per value (required).
For example, 16-byte quantity may be declared with:
{"type": "fixed", "size": 16, "name": "md5"}
Names
Record, enums and fixed are named types. Each has a fullname that is composed of two parts; a name and a namespace. Equality of names is defined on the fullname.
The name portion of a fullname, record field names, and enum symbols must:
- start with [A-Za-z_]
- subsequently contain only [A-Za-z0-9_]
A namespace is a dot-separated sequence of such names. The empty string may also be used as a namespace to indicate the null namespace. Equality of names (including field names and enum symbols) as well as fullnames is case-sensitive.
In record, enum and fixed definitions, the fullname is determined in one of the following ways:
A name and namespace are both specified. For example, one might use “name”: “X”, “namespace”: “org.foo” to indicate the fullname org.foo.X.
-
A fullname is specified. If the name specified contains a dot, then it is assumed to be a fullname, and any namespace also specified is ignored. For example, use “name”: “org.foo.X” to indicate the fullname org.foo.X.
-
A name only is specified, i.e., a name that contains no dots. In this case the namespace is taken from the most tightly enclosing schema or protocol. For example, if “name”: “X” is specified, and this occurs within a field of the record definition of org.foo.Y, then the fullname is org.foo.X. If there is no enclosing namespace then the null namespace is used.
References to previously defined names are as in the latter two cases above: if they contain a dot they are a fullname, if they do not contain a dot, the namespace is the namespace of the enclosing definition.
Primitive type names have no namespace and their names may not be defined in any namespace.
A schema or protocol may not contain multiple definitions of a fullname. Further, a name must be defined before it is used (“before” in the depth-first, left-to-right traversal of the JSON parse tree, where the types attribute of a protocol is always deemed to come “before” the messages attribute.)
Aliases
Named types and fields may have aliases. An implementation may optionally use aliases to map a writer’s schema to the reader’s. This facilitates both schema evolution as well as processing disparate datasets.
Aliases function by re-writing the writer’s schema using aliases from the reader’s schema. For example, if the writer’s schema was named “Foo” and the reader’s schema is named “Bar” and has an alias of “Foo”, then the implementation would act as though “Foo” were named “Bar” when reading. Similarly, if data was written as a record with a field named “x” and is read as a record with a field named “y” with alias “x”, then the implementation would act as though “x” were named “y” when reading.
A type alias may be specified either as a fully namespace-qualified, or relative to the namespace of the name it is an alias for. For example, if a type named “a.b” has aliases of “c” and “x.y”, then the fully qualified names of its aliases are “a.c” and “x.y”.
Data Serialization
Avro data is always serialized with its schema. Files that store Avro data should always also include the schema for that data in the same file. Avro-based remote procedure call (RPC) systems must also guarantee that remote recipients of data have a copy of the schema used to write that data.
Because the schema used to write data is always available when the data is read, Avro data itself is not tagged with type information. The schema is required to parse data.
In general, both serialization and deserialization proceed as a depth-first, left-to-right traversal of the schema, serializing primitive types as they are encountered.
Encodings
Avro specifies two serialization encodings: binary and JSON. Most applications will use the binary encoding, as it is smaller and faster. But, for debugging and web-based applications, the JSON encoding may sometimes be appropriate.
Binary Encoding
Primitive Types
Primitive types are encoded in binary as follows:
- null is written as zero bytes.
- a boolean is written as a single byte whose value is either 0 (false) or 1 (true).
- int and long values are written using variable-length zig-zag coding. Some examples:
| value |
hex |
| 0 |
00 |
| -1 |
01 |
| 1 |
02 |
| -2 |
03 |
| 2 |
04 |
| … |
|
| -64 |
7f |
| 64 |
80 01 |
| … |
|
- a float is written as 4 bytes. The float is converted into a 32-bit integer using a method equivalent to Java’s floatToIntBits and then encoded in little-endian format.
- a double is written as 8 bytes. The double is converted into a 64-bit integer using a method equivalent to Java’s doubleToLongBits and then encoded in little-endian format.
- bytes are encoded as a long followed by that many bytes of data.
- a string is encoded as a long followed by that many bytes of UTF-8 encoded character data.
For example, the three-character string “foo” would be encoded as the long value 3 (encoded as hex 06) followed by the UTF-8 encoding of ‘f’, ‘o’, and ‘o’ (the hex bytes 66 6f 6f):
06 66 6f 6f
Complex Types
Complex types are encoded in binary as follows:
Records
A record is encoded by encoding the values of its fields in the order that they are declared. In other words, a record is encoded as just the concatenation of the encodings of its fields. Field values are encoded per their schema.
For example, the record schema
{
"type": "record",
"name": "test",
"fields" : [
{"name": "a", "type": "long"},
{"name": "b", "type": "string"}
]
}
An instance of this record whose a field has value 27 (encoded as hex 36) and whose b field has value “foo” (encoded as hex bytes 06 66 6f 6f), would be encoded simply as the concatenation of these, namely the hex byte sequence:
36 06 66 6f 6f
Enums
An enum is encoded by a int, representing the zero-based position of the symbol in the schema.
For example, consider the enum:
{"type": "enum", "name": "Foo", "symbols": ["A", "B", "C", "D"] }
This would be encoded by an int between zero and three, with zero indicating “A”, and 3 indicating “D”.
Arrays
Arrays are encoded as a series of blocks. Each block consists of a long count value, followed by that many array items. A block with count zero indicates the end of the array. Each item is encoded per the array’s item schema.
If a block’s count is negative, its absolute value is used, and the count is followed immediately by a long block size indicating the number of bytes in the block. This block size permits fast skipping through data, e.g., when projecting a record to a subset of its fields.
For example, the array schema
{"type": "array", "items": "long"}
an array containing the items 3 and 27 could be encoded as the long value 2 (encoded as hex 04) followed by long values 3 and 27 (encoded as hex 06 36) terminated by zero:
04 06 36 00
The blocked representation permits one to read and write arrays larger than can be buffered in memory, since one can start writing items without knowing the full length of the array.
Maps
Maps are encoded as a series of blocks. Each block consists of a long count value, followed by that many key/value pairs. A block with count zero indicates the end of the map. Each item is encoded per the map’s value schema.
If a block’s count is negative, its absolute value is used, and the count is followed immediately by a long block size indicating the number of bytes in the block. This block size permits fast skipping through data, e.g., when projecting a record to a subset of its fields.
The blocked representation permits one to read and write maps larger than can be buffered in memory, since one can start writing items without knowing the full length of the map.
Unions
A union is encoded by first writing a long value indicating the zero-based position within the union of the schema of its value. The value is then encoded per the indicated schema within the union.
For example, the union schema ["null","string"] would encode:
-
null as zero (the index of “null” in the union):
00
-
the string “a” as one (the index of “string” in the union, 1, encoded as hex 02), followed by the serialized string:
02 02 61
Fixed
Fixed instances are encoded using the number of bytes declared in the schema.
JSON Encoding
Except for unions, the JSON encoding is the same as is used to encode field default values.
The value of a union is encoded in JSON as follows:
- if its type is null, then it is encoded as a JSON null;
- otherwise it is encoded as a JSON object with one name/value pair whose name is the type’s name and whose value is the recursively encoded value. For Avro’s named types (record, fixed or enum) the user-specified name is used, for other types the type name is used.
For example, the union schema ["null","string","Foo"], where Foo is a record name, would encode:
- null as null;
- the string “a” as
{"string": "a"} and
- a Foo instance as
{"Foo": {...}}, where {...} indicates the JSON encoding of a Foo instance.
Note that the original schema is still required to correctly process JSON-encoded data. For example, the JSON encoding does not distinguish between int and long, float and double, records and maps, enums and strings, etc.
Single-object encoding
In some situations a single Avro serialized object is to be stored for a longer period of time. One very common example is storing Avro records for several weeks in an Apache Kafka topic.
In the period after a schema change this persistence system will contain records that have been written with different schemas. So the need arises to know which schema was used to write a record to support schema evolution correctly. In most cases the schema itself is too large to include in the message, so this binary wrapper format supports the use case more effectively.
Single object encoding specification
Single Avro objects are encoded as follows:
- A two-byte marker,
C3 01, to show that the message is Avro and uses this single-record format (version 1).
- The 8-byte little-endian CRC-64-AVRO fingerprint of the object’s schema.
- The Avro object encoded using Avro’s binary encoding.
Implementations use the 2-byte marker to determine whether a payload is Avro. This check helps avoid expensive lookups that resolve the schema from a fingerprint, when the message is not an encoded Avro payload.
Sort Order
Avro defines a standard sort order for data. This permits data written by one system to be efficiently sorted by another system. This can be an important optimization, as sort order comparisons are sometimes the most frequent per-object operation. Note also that Avro binary-encoded data can be efficiently ordered without deserializing it to objects.
Data items may only be compared if they have identical schemas. Pairwise comparisons are implemented recursively with a depth-first, left-to-right traversal of the schema. The first mismatch encountered determines the order of the items.
Two items with the same schema are compared according to the following rules.
- null data is always equal.
- boolean data is ordered with false before true.
- int, long, float and double data is ordered by ascending numeric value.
- bytes and fixed data are compared lexicographically by unsigned 8-bit values.
- string data is compared lexicographically by Unicode code point. Note that since UTF-8 is used as the binary encoding for strings, sorting of bytes and string binary data is identical.
- array data is compared lexicographically by element.
- enum data is ordered by the symbol’s position in the enum schema. For example, an enum whose symbols are
["z", "a"] would sort “z” values before “a” values.
- union data is first ordered by the branch within the union, and, within that, by the type of the branch. For example, an
["int", "string"] union would order all int values before all string values, with the ints and strings themselves ordered as defined above.
- record data is ordered lexicographically by field. If a field specifies that its order is:
- “ascending”, then the order of its values is unaltered.
- “descending”, then the order of its values is reversed.
- “ignore”, then its values are ignored when sorting.
- map data may not be compared. It is an error to attempt to compare data containing maps unless those maps are in an
"order":"ignore" record field.
Object Container Files
Avro includes a simple object container file format. A file has a schema, and all objects stored in the file must be written according to that schema, using binary encoding. Objects are stored in blocks that may be compressed. Syncronization markers are used between blocks to permit efficient splitting of files for MapReduce processing.
Files may include arbitrary user-specified metadata.
A file consists of:
- A file header, followed by
- one or more file data blocks.
A file header consists of:
- Four bytes, ASCII ‘O’, ‘b’, ‘j’, followed by 1.
- file metadata, including the schema.
- The 16-byte, randomly-generated sync marker for this file.
File metadata is written as if defined by the following map schema:
{"type": "map", "values": "bytes"}
All metadata properties that start with “avro.” are reserved. The following file metadata properties are currently used:
- avro.schema contains the schema of objects stored in the file, as JSON data (required).
- avro.codec the name of the compression codec used to compress blocks, as a string. Implementations are required to support the following codecs: “null” and “deflate”. If codec is absent, it is assumed to be “null”. The codecs are described with more detail below.
A file header is thus described by the following schema:
{"type": "record", "name": "org.apache.avro.file.Header",
"fields" : [
{"name": "magic", "type": {"type": "fixed", "name": "Magic", "size": 4}},
{"name": "meta", "type": {"type": "map", "values": "bytes"}},
{"name": "sync", "type": {"type": "fixed", "name": "Sync", "size": 16}}
]
}
A file data block consists of:
- A long indicating the count of objects in this block.
- A long indicating the size in bytes of the serialized objects in the current block, after any codec is applied
- The serialized objects. If a codec is specified, this is compressed by that codec.
- The file’s 16-byte sync marker.
Thus, each block’s binary data can be efficiently extracted or skipped without deserializing the contents. The combination of block size, object counts, and sync markers enable detection of corrupt blocks and help ensure data integrity.
Required Codecs
null
The “null” codec simply passes through data uncompressed.
deflate
The “deflate” codec writes the data block using the deflate algorithm as specified in RFC 1951, and typically implemented using the zlib library. Note that this format (unlike the “zlib format” in RFC 1950) does not have a checksum.
Optional Codecs
snappy
The “snappy” codec uses Google’s Snappy compression library. Each compressed block is followed by the 4-byte, big-endian CRC32 checksum of the uncompressed data in the block.
Protocol Declaration
Avro protocols describe RPC interfaces. Like schemas, they are defined with JSON text.
A protocol is a JSON object with the following attributes:
- protocol, a string, the name of the protocol (required);
- namespace, an optional string that qualifies the name;
- doc, an optional string describing this protocol;
- types, an optional list of definitions of named types (records, enums, fixed and errors). An error definition is just like a record definition except it uses “error” instead of “record”. Note that forward references to named types are not permitted.
- messages, an optional JSON object whose keys are message names and whose values are objects whose attributes are described below. No two messages may have the same name.
The name and namespace qualification rules defined for schema objects apply to protocols as well.
Messages
A message has attributes:
- a doc, an optional description of the message,
- a request, a list of named, typed parameter schemas (this has the same form as the fields of a record declaration);
- a response schema;
- an optional union of declared error schemas. The effective union has “string” prepended to the declared union, to permit transmission of undeclared “system” errors. For example, if the declared error union is
["AccessError"], then the effective union is ["string", "AccessError"]. When no errors are declared, the effective error union is ["string"]. Errors are serialized using the effective union; however, a protocol’s JSON declaration contains only the declared union.
- an optional one-way boolean parameter.
A request parameter list is processed equivalently to an anonymous record. Since record field lists may vary between reader and writer, request parameters may also differ between the caller and responder, and such differences are resolved in the same manner as record field differences.
The one-way parameter may only be true when the response type is "null" and no errors are listed.
Sample Protocol
For example, one may define a simple HelloWorld protocol with:
{
"namespace": "com.acme",
"protocol": "HelloWorld",
"doc": "Protocol Greetings",
"types": [
{"name": "Greeting", "type": "record", "fields": [
{"name": "message", "type": "string"}]},
{"name": "Curse", "type": "error", "fields": [
{"name": "message", "type": "string"}]}
],
"messages": {
"hello": {
"doc": "Say hello.",
"request": [{"name": "greeting", "type": "Greeting" }],
"response": "Greeting",
"errors": ["Curse"]
}
}
}
Message Transport
Messages may be transmitted via different transport mechanisms.
To the transport, a message is an opaque byte sequence.
A transport is a system that supports:
- transmission of request messages
- receipt of corresponding response messages
Servers may send a response message back to the client corresponding to a request message. The mechanism of correspondance is transport-specific. For example, in HTTP it is implicit, since HTTP directly supports requests and responses. But a transport that multiplexes many client threads over a single socket would need to tag messages with unique identifiers.
Transports may be either stateless or stateful. In a stateless transport, messaging assumes no established connection state, while stateful transports establish connections that may be used for multiple messages. This distinction is discussed further in the handshake section below.
HTTP as Transport
When HTTP is used as a transport, each Avro message exchange is an HTTP request/response pair. All messages of an Avro protocol should share a single URL at an HTTP server. Other protocols may also use that URL. Both normal and error Avro response messages should use the 200 (OK) response code. The chunked encoding may be used for requests and responses, but, regardless the Avro request and response are the entire content of an HTTP request and response. The HTTP Content-Type of requests and responses should be specified as “avro/binary”. Requests should be made using the POST method.
HTTP is used by Avro as a stateless transport.
Message Framing
Avro messages are framed as a list of buffers.
Framing is a layer between messages and the transport. It exists to optimize certain operations.
The format of framed message data is:
- a series of buffers, where each buffer consists of:
- a four-byte, big-endian buffer length, followed by
- that many bytes of buffer data.
- A message is always terminated by a zero-length buffer.
Framing is transparent to request and response message formats (described below). Any message may be presented as a single or multiple buffers.
Framing can permit readers to more efficiently get different buffers from different sources and for writers to more efficiently store different buffers to different destinations. In particular, it can reduce the number of times large binary objects are copied. For example, if an RPC parameter consists of a megabyte of file data, that data can be copied directly to a socket from a file descriptor, and, on the other end, it could be written directly to a file descriptor, never entering user space.
A simple, recommended, framing policy is for writers to create a new segment whenever a single binary object is written that is larger than a normal output buffer. Small objects are then appended in buffers, while larger objects are written as their own buffers. When a reader then tries to read a large object the runtime can hand it an entire buffer directly, without having to copy it.
Handshake
The purpose of the handshake is to ensure that the client and the server have each other’s protocol definition, so that the client can correctly deserialize responses, and the server can correctly deserialize requests. Both clients and servers should maintain a cache of recently seen protocols, so that, in most cases, a handshake will be completed without extra round-trip network exchanges or the transmission of full protocol text.
RPC requests and responses may not be processed until a handshake has been completed. With a stateless transport, all requests and responses are prefixed by handshakes. With a stateful transport, handshakes are only attached to requests and responses until a successful handshake response has been returned over a connection. After this, request and response payloads are sent without handshakes for the lifetime of that connection.
The handshake process uses the following record schemas:
{
"type": "record",
"name": "HandshakeRequest", "namespace":"org.apache.avro.ipc",
"fields": [
{"name": "clientHash",
"type": {"type": "fixed", "name": "MD5", "size": 16}},
{"name": "clientProtocol", "type": ["null", "string"]},
{"name": "serverHash", "type": "MD5"},
{"name": "meta", "type": ["null", {"type": "map", "values": "bytes"}]}
]
}
{
"type": "record",
"name": "HandshakeResponse", "namespace": "org.apache.avro.ipc",
"fields": [
{"name": "match",
"type": {"type": "enum", "name": "HandshakeMatch",
"symbols": ["BOTH", "CLIENT", "NONE"]}},
{"name": "serverProtocol",
"type": ["null", "string"]},
{"name": "serverHash",
"type": ["null", {"type": "fixed", "name": "MD5", "size": 16}]},
{"name": "meta",
"type": ["null", {"type": "map", "values": "bytes"}]}
]
}
- A client first prefixes each request with a
HandshakeRequest containing just the hash of its protocol and of the server’s protocol (clientHash!=null, clientProtocol=null, serverHash!=null), where the hashes are 128-bit MD5 hashes of the JSON protocol text. If a client has never connected to a given server, it sends its hash as a guess of the server’s hash, otherwise it sends the hash that it previously obtained from this server.
The server responds with a HandshakeResponse containing one of:
-
match=BOTH, serverProtocol=null, serverHash=null if the client sent the valid hash of the server’s protocol and the server knows what protocol corresponds to the client’s hash. In this case, the request is complete and the response data immediately follows the HandshakeResponse.
-
match=CLIENT, serverProtocol!=null, serverHash!=null if the server has previously seen the client’s protocol, but the client sent an incorrect hash of the server’s protocol. The request is complete and the response data immediately follows the HandshakeResponse. The client must use the returned protocol to process the response and should also cache that protocol and its hash for future interactions with this server.
-
match=NONE if the server has not previously seen the client’s protocol. The serverHash and serverProtocol may also be non-null if the server’s protocol hash was incorrect.
In this case the client must then re-submit its request with its protocol text (clientHash!=null, clientProtocol!=null, serverHash!=null) and the server should respond with a successful match (match=BOTH, serverProtocol=null, serverHash=null) as above.
The meta field is reserved for future handshake enhancements.
A call consists of a request message paired with its resulting response or error message. Requests and responses contain extensible metadata, and both kinds of messages are framed as described above.
The format of a call request is:
- request metadata, a map with values of type bytes
- the message name, an Avro string, followed by
- the message parameters. Parameters are serialized according to the message’s request declaration.
When the empty string is used as a message name a server should ignore the parameters and return an empty response. A client may use this to ping a server or to perform a handshake without sending a protocol message.
When a message is declared one-way and a stateful connection has been established by a successful handshake response, no response data is sent. Otherwise the format of the call response is:
- response metadata, a map with values of type bytes
- a one-byte error flag boolean, followed by either:
- if the error flag is false, the message response, serialized per the message’s response schema.
- if the error flag is true, the error, serialized per the message’s effective error union schema.
Schema Resolution
A reader of Avro data, whether from an RPC or a file, can always parse that data because the original schema must be provided along with the data. However, the reader may be programmed to read data into a different schema. For example, if the data was written with a different version of the software than it is read, then fields may have been added or removed from records. This section specifies how such schema differences should be resolved.
We refer to the schema used to write the data as the writer’s schema, and the schema that the application expects the reader’s schema. Differences between these should be resolved as follows:
- It is an error if the two schemas do not match.
To match, one of the following must hold:
-
both schemas are arrays whose item types match
-
both schemas are maps whose value types match
-
both schemas are enums whose names match
-
both schemas are fixed whose sizes and names match
-
both schemas are records with the same name
-
either schema is a union
-
both schemas have same primitive type
-
the writer’s schema may be promoted to the reader’s as follows:
- int is promotable to long, float, or double
- long is promotable to float or double
- float is promotable to double
- string is promotable to bytes
- bytes is promotable to string
-
if both are records:
- the ordering of fields may be different: fields are matched by name.
- schemas for fields with the same name in both records are resolved recursively.
- if the writer’s record contains a field with a name not present in the reader’s record, the writer’s value for that field is ignored.
- if the reader’s record schema has a field that contains a default value, and writer’s schema does not have a field with the same name, then the reader should use the default value from its field.
- if the reader’s record schema has a field with no default value, and writer’s schema does not have a field with the same name, an error is signalled.
-
if both are enums:
if the writer’s symbol is not present in the reader’s enum and the reader has a default value, then that value is used, otherwise an error is signalled.
This resolution algorithm is applied recursively to the reader’s and writer’s array item schemas.
This resolution algorithm is applied recursively to the reader’s and writer’s value schemas.
The first schema in the reader’s union that matches the selected writer’s union schema is recursively resolved against it. if none match, an error is signalled.
- if reader’s is a union, but writer’s is not
The first schema in the reader’s union that matches the writer’s schema is recursively resolved against it. If none match, an error is signalled.
- if writer’s is a union, but reader’s is not
If the reader’s schema matches the selected writer’s schema, it is recursively resolved against it. If they do not match, an error is signalled.
A schema’s doc fields are ignored for the purposes of schema resolution. Hence, the doc portion of a schema may be dropped at serialization.
One of the defining characteristics of Avro is that a reader must use the schema used by the writer of the data in order to know how to read the data. This assumption results in a data format that’s compact and also amenable to many forms of schema evolution. However, the specification so far has not defined what it means for the reader to have the “same” schema as the writer. Does the schema need to be textually identical? Well, clearly adding or removing some whitespace to a JSON expression does not change its meaning. At the same time, reordering the fields of records clearly does change the meaning. So what does it mean for a reader to have “the same” schema as a writer?
Parsing Canonical Form is a transformation of a writer’s schema that let’s us define what it means for two schemas to be “the same” for the purpose of reading data written against the schema. It is called Parsing Canonical Form because the transformations strip away parts of the schema, like “doc” attributes, that are irrelevant to readers trying to parse incoming data. It is called Canonical Form because the transformations normalize the JSON text (such as the order of attributes) in a way that eliminates unimportant differences between schemas. If the Parsing Canonical Forms of two different schemas are textually equal, then those schemas are “the same” as far as any reader is concerned, i.e., there is no serialized data that would allow a reader to distinguish data generated by a writer using one of the original schemas from data generated by a writing using the other original schema. (We sketch a proof of this property in a companion document.)
The next subsection specifies the transformations that define Parsing Canonical Form. But with a well-defined canonical form, it can be convenient to go one step further, transforming these canonical forms into simple integers (“fingerprints”) that can be used to uniquely identify schemas. The subsection after next recommends some standard practices for generating such fingerprints.
Assuming an input schema (in JSON form) that’s already UTF-8 text for a valid Avro schema (including all quotes as required by JSON), the following transformations will produce its Parsing Canonical Form:
- [PRIMITIVES] Convert primitive schemas to their simple form (e.g., int instead of
{"type":"int"}).
- [FULLNAMES] Replace short names with fullnames, using applicable namespaces to do so. Then eliminate namespace attributes, which are now redundant.
- [STRIP] Keep only attributes that are relevant to parsing data, which are: type, name, fields, symbols, items, values, size. Strip all others (e.g., doc and aliases).
- [ORDER] Order the appearance of fields of JSON objects as follows: name, type, fields, symbols, items, values, size. For example, if an object has type, name, and size fields, then the name field should appear first, followed by the type and then the size fields.
- [STRINGS] For all JSON string literals in the schema text, replace any escaped characters (e.g., \uXXXX escapes) with their UTF-8 equivalents.
- [INTEGERS] Eliminate quotes around and any leading zeros in front of JSON integer literals (which appear in the size attributes of fixed schemas).
- [WHITESPACE] Eliminate all whitespace in JSON outside of string literals.
Schema Fingerprints
“[A] fingerprinting algorithm is a procedure that maps an arbitrarily large data item (such as a computer file) to a much shorter bit string, its fingerprint, that uniquely identifies the original data for all practical purposes” (quoted from Wikipedia). In the Avro context, fingerprints of Parsing Canonical Form can be useful in a number of applications; for example, to cache encoder and decoder objects, to tag data items with a short substitute for the writer’s full schema, and to quickly negotiate common-case schemas between readers and writers.
In designing fingerprinting algorithms, there is a fundamental trade-off between the length of the fingerprint and the probability of collisions. To help application designers find appropriate points within this trade-off space, while encouraging interoperability and ease of implementation, we recommend using one of the following three algorithms when fingerprinting Avro schemas:
- When applications can tolerate longer fingerprints, we recommend using the SHA-256 digest algorithm to generate 256-bit fingerprints of Parsing Canonical Forms. Most languages today have SHA-256 implementations in their libraries.
- At the opposite extreme, the smallest fingerprint we recommend is a 64-bit Rabin fingerprint. Below, we provide pseudo-code for this algorithm that can be easily translated into any programming language. 64-bit fingerprints should guarantee uniqueness for schema caches of up to a million entries (for such a cache, the chance of a collision is 3E-8). We don’t recommend shorter fingerprints, as the chances of collisions is too great (for example, with 32-bit fingerprints, a cache with as few as 100,000 schemas has a 50% chance of having a collision).
- Between these two extremes, we recommend using the MD5 message digest to generate 128-bit fingerprints. These make sense only where very large numbers of schemas are being manipulated (tens of millions); otherwise, 64-bit fingerprints should be sufficient. As with SHA-256, MD5 implementations are found in most libraries today.
These fingerprints are not meant to provide any security guarantees, even the longer SHA-256-based ones. Most Avro applications should be surrounded by security measures that prevent attackers from writing random data and otherwise interfering with the consumers of schemas. We recommend that these surrounding mechanisms be used to prevent collision and pre-image attacks (i.e., “forgery”) on schema fingerprints, rather than relying on the security properties of the fingerprints themselves.
Rabin fingerprints are cyclic redundancy checks computed using irreducible polynomials. In the style of the Appendix of RFC 1952 (pg 10), which defines the CRC-32 algorithm, here’s our definition of the 64-bit AVRO fingerprinting algorithm:
long fingerprint64(byte[] buf) {
if (FP_TABLE == null) initFPTable();
long fp = EMPTY;
for (int i = 0; i < buf.length; i++)
fp = (fp >>> 8) ^ FP_TABLE[(int)(fp ^ buf[i]) & 0xff];
return fp;
}
static long EMPTY = 0xc15d213aa4d7a795L;
static long[] FP_TABLE = null;
void initFPTable() {
FP_TABLE = new long[256];
for (int i = 0; i < 256; i++) {
long fp = i;
for (int j = 0; j < 8; j++)
fp = (fp >>> 1) ^ (EMPTY & -(fp & 1L));
FP_TABLE[i] = fp;
}
}
Readers interested in the mathematics behind this algorithm may want to read this book chapter. (Unlike RFC-1952 and the book chapter, we prepend a single one bit to messages. We do this because CRCs ignore leading zero bits, which can be problematic. Our code prepends a one-bit by initializing fingerprints using EMPTY, rather than initializing using zero as in RFC-1952 and the book chapter.)
Logical Types
A logical type is an Avro primitive or complex type with extra attributes to represent a derived type. The attribute logicalType must always be present for a logical type, and is a string with the name of one of the logical types listed later in this section. Other attributes may be defined for particular logical types.
A logical type is always serialized using its underlying Avro type so that values are encoded in exactly the same way as the equivalent Avro type that does not have a logicalType attribute. Language implementations may choose to represent logical types with an appropriate native type, although this is not required.
Language implementations must ignore unknown logical types when reading, and should use the underlying Avro type. If a logical type is invalid, for example a decimal with scale greater than its precision, then implementations should ignore the logical type and use the underlying Avro type.
Decimal
The decimal logical type represents an arbitrary-precision signed decimal number of the form unscaled × 10-scale.
A decimal logical type annotates Avro bytes or fixed types. The byte array must contain the two’s-complement representation of the unscaled integer value in big-endian byte order. The scale is fixed, and is specified using an attribute.
The following attributes are supported:
- scale, a JSON integer representing the scale (optional). If not specified the scale is 0.
- precision, a JSON integer representing the (maximum) precision of decimals stored in this type (required).
For example, the following schema represents decimal numbers with a maximum precision of 4 and a scale of 2:
{
"type": "bytes",
"logicalType": "decimal",
"precision": 4,
"scale": 2
}
Precision must be a positive integer greater than zero. If the underlying type is a fixed, then the precision is limited by its size. An array of length n can store at most floor(log10(28 × n - 1 - 1)) base-10 digits of precision.
Scale must be zero or a positive integer less than or equal to the precision.
For the purposes of schema resolution, two schemas that are decimal logical types match if their scales and precisions match.
Date
The date logical type represents a date within the calendar, with no reference to a particular time zone or time of day.
A date logical type annotates an Avro int, where the int stores the number of days from the unix epoch, 1 January 1970 (ISO calendar).
Time (millisecond precision)
The time-millis logical type represents a time of day, with no reference to a particular calendar, time zone or date, with a precision of one millisecond.
A time-millis logical type annotates an Avro int, where the int stores the number of milliseconds after midnight, 00:00:00.000.
Time (microsecond precision)
The time-micros logical type represents a time of day, with no reference to a particular calendar, time zone or date, with a precision of one microsecond.
A time-micros logical type annotates an Avro long, where the long stores the number of microseconds after midnight, 00:00:00.000000.
Timestamp (millisecond precision)
The timestamp-millis logical type represents an instant on the global timeline, independent of a particular time zone or calendar, with a precision of one millisecond.
A timestamp-millis logical type annotates an Avro long, where the long stores the number of milliseconds from the unix epoch, 1 January 1970 00:00:00.000 UTC.
Timestamp (microsecond precision)
The timestamp-micros logical type represents an instant on the global timeline, independent of a particular time zone or calendar, with a precision of one microsecond.
A timestamp-micros logical type annotates an Avro long, where the long stores the number of microseconds from the unix epoch, 1 January 1970 00:00:00.000000 UTC.
Duration
The duration logical type represents an amount of time defined by a number of months, days and milliseconds. This is not equivalent to a number of milliseconds, because, depending on the moment in time from which the duration is measured, the number of days in the month and number of milliseconds in a day may differ. Other standard periods such as years, quarters, hours and minutes can be expressed through these basic periods.
A duration logical type annotates Avro fixed type of size 12, which stores three little-endian unsigned integers that represent durations at different granularities of time. The first stores a number in months, the second stores a number in days, and the third stores a number in milliseconds.
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
9.4 - Trevni
Version 0.1
DRAFT
This document is the authoritative specification of a file format. Its intent is to permit compatible, independent implementations that read and/or write files in this format.
Introduction
Data sets are often described as a table composed of rows and columns. Each record in the dataset is considered a row, with each field of the record occupying a different column. Writing records to a file one-by-one as they are created results in a row-major format, like Hadoop’s SequenceFile or Avro data files.
In many cases higher query performance may be achieved if the data is instead organized in a column-major format, where multiple values of a given column are stored adjacently. This document defines such a column-major file format for datasets.
To permit scalable, distributed query evaluation, datasets are partitioned into row groups, containing distinct collections of rows. Each row group is organized in column-major order, while row groups form a row-major partitioning of the entire dataset.
Rationale
Goals
The format is meant satisfy the following goals:
Maximize the size of row groups. Disc drives are used most efficiently when sequentially accessing data. Consider a drive that takes 10ms to seek and transfers at 100MB/second. If a 10-column dataset whose values are all the same size is split into 10MB row groups, then accessing a single column will require a sequence of seek+1MB reads, for a cost of 20ms/MB processed. If the same dataset is split into 100MB row groups then this drops to 11ms/MB processed. This effect is exaggerated for datasets with larger numbers of columns and with columns whose values are smaller than average. So we’d prefer row groups that are 100MB or greater.
Permit random access within a row group. Some queries will first examine one column, and, only when certain relatively rare criteria are met, examine other columns. Rather than iterating through selected columns of the row-group in parallel, one might iterate through one column and randomly access another. This is called support for WHERE clauses, after the SQL operator of that name.
Minimize the number of files per dataset. HDFS is a primary intended deployment platform for these files. The HDFS Namenode requires memory for each file in the filesystem, thus for a format to be HDFS-friendly it should strive to require the minimum number of distinct files.
Support co-location of columns within row-groups. Row groups are the unit of parallel operation on a column dataset. For efficient file i/o, the entirety of a row-group should ideally reside on the host that is evaluating the query in order to avoid network latencies and bottlenecks.
Data integrity. The format should permit applications to detect data corruption. Many file systems may prevent corruption, but files may be moved between filesystems and be subject to corruption at points in that process. It is best if the data in a file can be validated independently.
Extensibility. The format should permit applications to store additional annotations about a datasets in the files, such as type information, origin, etc. Some environments may have metadata stores for such information, but not all do, and files might be moved among systems with different metadata systems. The ability to keep such information within the file simplifies the coordination of such information.
Minimal overhead. The column format should not make datasets appreciably larger. Storage is a primary cost and a choice to use this format should not require additional storage.
Primary format. The column format should be usable as a primary format for datasets, not as an auxiliary, accelerated format. Applications that process a dataset in row-major order should be able to easily consume column files and applications that produce datasets in row-major order should be able to easily generate column files.
Design
To meet these goals we propose the following design.
Each row group is a separate file. All values of a column in a file are written contiguously. This maximizes the row group size, optimizing performance when querying few and small columns.
Each file occupies a single HDFS block. A larger than normal block size may be specified, e.g., ~1GB instead of the typical ~100MB. This guarantees co-location and eliminates network use when query processing can be co-located with the file. This also moderates the memory impact on the HDFS Namenode since no small files are written.
Each column in a file is written as a sequence of ~64kB compressed blocks. The sequence is prefixed by a table describing all of the blocks in the column to permit random access within the column.
Application-specific metadata may be added at the file, column, and block levels.
Checksums are included with each block, providing data integrity.
Discussion
The use of a single block per file achieves the same effect as the custom block placement policy described in the CIF paper, but while still permitting HDFS rebalancing and not increasing the number of files in the namespace.
This section formally describes the proposed column file format.
Data Model
We assume a simple data model, where a record is a set of named fields, and the value of each field is a sequence of untyped bytes. A type system may be layered on top of this, as specified in the Type Mapping section below.
Primitive Values
We define the following primitive value types:
- Signed 64-bit long values are written using a variable-length zig-zag coding, where the high-order bit in each byte determines whether subsequent bytes are present. For example:
decimal value hex bytes
| decimal value |
hex bytes |
|
| 0 |
00 |
|
| -1 |
01 |
|
| 1 |
02 |
|
| … |
|
|
| -64 |
7f |
|
| 64 |
80 01 |
|
| … |
|
|
- bytes are encoded as a long followed by that many bytes of data.
- a string is encoded as a long followed by that many bytes of UTF-8 encoded character data.
For example, the three-character string “foo” would be encoded as the long value 3 (encoded as hex 06) followed by the UTF-8 encoding of ‘f’, ‘o’, and ‘o’ (the hex bytes 66 6f 6f): 06 66 6f 6f
Type Names
The following type names are used to describe column values:
- null, requires zero bytes. Sometimes used in array columns.
- boolean, one bit, packed into bytes, little-endian;
- int, like long, but restricted to 32-bit signed values
- long 64-bit signed values, represented as above
- fixed32 32-bit values stored as four bytes, little-endian.
- fixed64 64-bit values stored as eight bytes, little-endian.
- float 32-bit IEEE floating point value, little-endian
- double 64-bit IEEE floating point value, little-endian
- string as above
- bytes as above, may be used to encapsulate more complex objects
Type names are represented as strings (UTF-8 encoded, length-prefixed).
Metadata consists of:
- A long indicating the number of metadata key/value pairs.
- For each pair, a string key and bytes value.
All metadata properties that start with “trevni.” are reserved.
The following file metadata properties are defined:
- trevni.codec the name of the default compression codec used to compress blocks, as a string. Implementations are required to support the “null” codec. Optional. If absent, it is assumed to be “null”. Codecs are described in more detail below.
- trevni.checksum the name of the checksum algorithm used in this file, as a string. Implementations are required to support the “crc-32” checksum. Optional. If absent, it is assumed to be “null”. Checksums are described in more detail below.
The following column metadata properties are defined:
- trevni.codec the name of the compression codec used to compress the blocks of this column, as a string. Implementations are required to support the “null” codec. Optional. If absent, it is assumed to be “null”. Codecs are described in more detail below.
- trevni.name the name of the column, as a string. Required.
- trevni.type the type of data in the column. One of the type names above. Required.
- trevni.values if present, indicates that the initial value of each block in this column will be stored in the block’s descriptor. Not permitted for array columns or columns that specify a parent.
- trevni.array if present, indicates that each row in this column contains a sequence of values of the named type rather than just a single value. An integer length precedes each sequence of values indicating the count of values in the sequence. If the length is negative then it indicates a sequence of zero or one lengths, where -1 indicates two zeros, -2 two ones, -3 three zeros, -4 three ones, etc.
- trevni.parent if present, the name of an array column whose lengths are also used by this column. Thus values of this column are sequences but no lengths are stored in this column.
For example, consider the following row, as JSON, where all values are primitive types, but one has multiple values.
{"id"=566, "date"=23423234234
"from"="foo@bar.com",
"to"=["bar@baz.com", "bang@foo.com"],
"content"="Hi!"}
The columns for this might be specified as:
name=id type=int
name=date type=long
name=from type=string
name=to type=string array=true
name=content type=string
If a row contains an array of records, e.g. “received” in the following:
{"id"=566, "date"=23423234234
"from"="foo@bar.com",
"to"=["bar@baz.com", "bang@foo.com"],
"content"="Hi!"
"received"=[{"date"=234234234234, "host"="192.168.0.0.1"},
{"date"=234234545645, "host"="192.168.0.0.2"}]
}
Then one can define a parent column followed by a column for each field in the record, adding the following columns:
name=received type=null array=true
name=date type=long parent=received
name=host type=string parent=received
If an array value itself contains an array, e.g. the “sigs” below:
{"id"=566, "date"=23423234234
"from"="foo@bar.com",
"to"=["bar@baz.com", "bang@foo.com"],
"content"="Hi!"
"received"=[{"date"=234234234234, "host"="192.168.0.0.1",
"sigs"=[{"algo"="weak", "value"="0af345de"}]},
{"date"=234234545645, "host"="192.168.0.0.2",
"sigs"=[]}]
}
Then a parent column may be defined that itself has a parent column.
name=sigs type=null array=true parent=received
name=algo type=string parent=sigs
name=value type=string parent=sigs
No block metadata properties are currently defined.
A file consists of:
- A file header, followed by
- one or more columns.
A file header consists of:
- Four bytes, ASCII ‘T’, ‘r’, ‘v’, followed by 0x02.
- a fixed64 indicating the number of rows in the file
- a fixed32 indicating the number of columns in the file
- file metadata.
- for each column, its column metadata
- for each column, its starting position in the file as a fixed64.
A column consists of:
- A fixed32 indicating the number of blocks in this column.
- For each block, a block descriptor
- One or more blocks.
A block descriptor consists of:
- A fixed32 indicating the number of rows in the block
- A fixed32 indicating the size in bytes of the block before the codec is applied (excluding checksum).
- A fixed32 indicating the size in bytes of the block after the codec is applied (excluding checksum).
- If this column’s metadata declares it to include values, the first value in the column, serialized according to this column’s type.
A block consists of:
- The serialized column values. If a column is an array column then value sequences are preceded by their length, as an int. If a codec is specified, the values and lengths are compressed by that codec.
- The checksum, as determined by the file metadata.
Codecs
null
The “null” codec simply passes data through uncompressed.
deflate
The “deflate” codec writes the data block using the deflate algorithm as specified in RFC 1951.
snappy
The “snappy” codec uses Google’s Snappy compression library.
Checksum algorithms
null
The “null” checksum contains zero bytes.
crc-32
Each “crc-32” checksum contains the four bytes of an ISO 3309 CRC-32 checksum of the uncompressed block data as a fixed32.
Type Mappings
We define a standard mapping for how types defined in various serialization systems are represented in a column file. Records from these systems are shredded into columns. When records are nested, a depth-first recursive walk can assign a separate column for each primitive value.
Avro
Protocol Buffers
Thrift
Implementation Notes
Some possible techniques for writing column files include:
- Use a standard ~100MB block, buffer in memory up to the block size, then flush the file directly to HDFS. A single reduce task might create multiple output files. The namenode requires memory proportional to the number of names and blocks*replication. This would increase the number of names but not blocks, so this should still be much better than a file per column.
- Spill each column to a separate local, temporary file then, when the file is closed, append these files, writing a single file to HDFS whose block size is set to be that of the entire file. This would be a bit slower than and may have trouble when the local disk is full, but it would better use HDFS namespace and further reduce seeks when processing columns whose values are small.
- Use a separate mapreduce job to convert row-major files to column-major. The map output would output a by (row#, column#, value) tuple, partitioned by row# but sorted by column# then row#. The reducer could directly write the column file. But the column file format would need to be changed to write counts, descriptors, etc. at the end of files rather than at the front.
(1) is the simplest to implement and most implementations should start with it.
References
CIF Column-Oriented Storage Techniques for MapReduce, Floratou, Patel, Shekita, & Tata, VLDB 2011.
DREMEL Dremel: Interactive Analysis of Web-Scale Datasets, Melnik, Gubarev, Long, Romer, Shivakumar, & Tolton, VLDB 2010.
9.5 - MapReduce guide
Avro provides a convenient way to represent complex data structures within a Hadoop MapReduce job. Avro data can be used as both input to and output from a MapReduce job, as well as the intermediate format. The example in this guide uses Avro data for all three, but it’s possible to mix and match; for instance, MapReduce can be used to aggregate a particular field in an Avro record.
This guide assumes basic familiarity with both Hadoop MapReduce and Avro. See the Hadoop documentation and the Avro getting started guide for introductions to these projects. This guide uses the old MapReduce API (org.apache.hadoop.mapred) and the new MapReduce API (org.apache.hadoop.mapreduce).
Setup
The code from this guide is included in the Avro docs under examples/mr-example. The example is set up as a Maven project that includes the necessary Avro and MapReduce dependencies and the Avro Maven plugin for code generation, so no external jars are needed to run the example. In particular, the POM includes the following dependencies:
<dependency>
<groupId>org.apache.avro</groupId>
<artifactId>avro</artifactId>
<version>1.8.2</version>
</dependency>
<dependency>
<groupId>org.apache.avro</groupId>
<artifactId>avro-mapred</artifactId>
<version>1.8.2</version>
</dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-core</artifactId>
<version>1.1.0</version>
</dependency>
And the following plugin:
<plugin>
<groupId>org.apache.avro</groupId>
<artifactId>avro-maven-plugin</artifactId>
<version>1.8.2</version>
<executions>
<execution>
<phase>generate-sources</phase>
<goals>
<goal>schema</goal>
</goals>
<configuration>
<sourceDirectory>${project.basedir}/../</sourceDirectory>
<outputDirectory>${project.basedir}/target/generated-sources/</outputDirectory>
</configuration>
</execution>
</executions>
</plugin>
If you do not configure the sourceDirectory and outputDirectory properties, the defaults will be used. The sourceDirectory property defaults to src/main/avro. The outputDirectory property defaults to target/generated-sources. You can change the paths to match your project layout.
Alternatively, Avro jars can be downloaded directly from the Apache Avro™ Releases page. The relevant Avro jars for this guide are avro-1.8.2.jar and avro-mapred-1.8.2.jar, as well as avro-tools-1.8.2.jar for code generation and viewing Avro data files as JSON. In addition, you will need to install Hadoop in order to use MapReduce.
Example: ColorCount
Below is a simple example of a MapReduce that uses Avro. There is an example for both the old (org.apache.hadoop.mapred) and new (org.apache.hadoop.mapreduce) APIs under examples/mr-example/src/main/java/example/. MapredColorCount is the example for the older mapred API while MapReduceColorCount is the example for the newer mapreduce API. Both examples are below, but we will detail the mapred API in our subsequent examples.
MapredColorCount:
package example;
import java.io.IOException;
import org.apache.avro.*;
import org.apache.avro.Schema.Type;
import org.apache.avro.mapred.*;
import org.apache.hadoop.conf.*;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.mapred.*;
import org.apache.hadoop.util.*;
import example.avro.User;
public class MapredColorCount extends Configured implements Tool {
public static class ColorCountMapper extends AvroMapper<User, Pair<CharSequence, Integer>> {
@Override
public void map(User user, AvroCollector<Pair<CharSequence, Integer>> collector, Reporter reporter)
throws IOException {
CharSequence color = user.getFavoriteColor();
// We need this check because the User.favorite_color field has type ["string", "null"]
if (color == null) {
color = "none";
}
collector.collect(new Pair<CharSequence, Integer>(color, 1));
}
}
public static class ColorCountReducer extends AvroReducer<CharSequence, Integer,
Pair<CharSequence, Integer>> {
@Override
public void reduce(CharSequence key, Iterable<Integer> values,
AvroCollector<Pair<CharSequence, Integer>> collector,
Reporter reporter)
throws IOException {
int sum = 0;
for (Integer value : values) {
sum += value;
}
collector.collect(new Pair<CharSequence, Integer>(key, sum));
}
}
public int run(String[] args) throws Exception {
if (args.length != 2) {
System.err.println("Usage: MapredColorCount <input path> <output path>");
return -1;
}
JobConf conf = new JobConf(getConf(), MapredColorCount.class);
conf.setJobName("colorcount");
FileInputFormat.setInputPaths(conf, new Path(args[0]));
FileOutputFormat.setOutputPath(conf, new Path(args[1]));
AvroJob.setMapperClass(conf, ColorCountMapper.class);
AvroJob.setReducerClass(conf, ColorCountReducer.class);
// Note that AvroJob.setInputSchema and AvroJob.setOutputSchema set
// relevant config options such as input/output format, map output
// classes, and output key class.
AvroJob.setInputSchema(conf, User.getClassSchema());
AvroJob.setOutputSchema(conf, Pair.getPairSchema(Schema.create(Type.STRING),
Schema.create(Type.INT)));
JobClient.runJob(conf);
return 0;
}
public static void main(String[] args) throws Exception {
int res = ToolRunner.run(new Configuration(), new MapredColorCount(), args);
System.exit(res);
}
}
MapReduceColorCount:
package example;
import java.io.IOException;
import org.apache.avro.Schema;
import org.apache.avro.mapred.AvroKey;
import org.apache.avro.mapred.AvroValue;
import org.apache.avro.mapreduce.AvroJob;
import org.apache.avro.mapreduce.AvroKeyInputFormat;
import org.apache.avro.mapreduce.AvroKeyValueOutputFormat;
import org.apache.hadoop.conf.Configured;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.IntWritable;
import org.apache.hadoop.io.NullWritable;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.mapreduce.Job;
import org.apache.hadoop.mapreduce.Mapper;
import org.apache.hadoop.mapreduce.Reducer;
import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
import org.apache.hadoop.util.Tool;
import org.apache.hadoop.util.ToolRunner;
import example.avro.User;
public class MapReduceColorCount extends Configured implements Tool {
public static class ColorCountMapper extends
Mapper<AvroKey<User>, NullWritable, Text, IntWritable> {
@Override
public void map(AvroKey<User> key, NullWritable value, Context context)
throws IOException, InterruptedException {
CharSequence color = key.datum().getFavoriteColor();
if (color == null) {
color = "none";
}
context.write(new Text(color.toString()), new IntWritable(1));
}
}
public static class ColorCountReducer extends
Reducer<Text, IntWritable, AvroKey<CharSequence>, AvroValue<Integer>> {
@Override
public void reduce(Text key, Iterable<IntWritable> values,
Context context) throws IOException, InterruptedException {
int sum = 0;
for (IntWritable value : values) {
sum += value.get();
}
context.write(new AvroKey<CharSequence>(key.toString()), new AvroValue<Integer>(sum));
}
}
public int run(String[] args) throws Exception {
if (args.length != 2) {
System.err.println("Usage: MapReduceColorCount <input path> <output path>");
return -1;
}
Job job = new Job(getConf());
job.setJarByClass(MapReduceColorCount.class);
job.setJobName("Color Count");
FileInputFormat.setInputPaths(job, new Path(args[0]));
FileOutputFormat.setOutputPath(job, new Path(args[1]));
job.setInputFormatClass(AvroKeyInputFormat.class);
job.setMapperClass(ColorCountMapper.class);
AvroJob.setInputKeySchema(job, User.getClassSchema());
job.setMapOutputKeyClass(Text.class);
job.setMapOutputValueClass(IntWritable.class);
job.setOutputFormatClass(AvroKeyValueOutputFormat.class);
job.setReducerClass(ColorCountReducer.class);
AvroJob.setOutputKeySchema(job, Schema.create(Schema.Type.STRING));
AvroJob.setOutputValueSchema(job, Schema.create(Schema.Type.INT));
return (job.waitForCompletion(true) ? 0 : 1);
}
public static void main(String[] args) throws Exception {
int res = ToolRunner.run(new MapReduceColorCount(), args);
System.exit(res);
}
}
ColorCount reads in data files containing User records, defined in examples/user.avsc, and counts the number of instances of each favorite color. (This example draws inspiration from the canonical WordCount MapReduce application.) This example uses the old MapReduce API. See MapReduceAvroWordCount, found under doc/examples/mr-example/src/main/java/example/ to see the new MapReduce API example. The User schema is defined as follows:
{"namespace": "example.avro",
"type": "record",
"name": "User",
"fields": [
{"name": "name", "type": "string"},
{"name": "favorite_number", "type": ["int", "null"]},
{"name": "favorite_color", "type": ["string", "null"]}
]
}
This schema is compiled into the User class used by ColorCount via the Avro Maven plugin (see examples/mr-example/pom.xml for how this is set up).
ColorCountMapper essentially takes a User as input and extracts the User’s favorite color, emitting the key-value pair <favoriteColor, 1>. ColorCountReducer then adds up how many occurrences of a particular favorite color were emitted, and outputs the result as a Pair record. These Pairs are serialized to an Avro data file.
Running ColorCount
The ColorCount application is provided as a Maven project in the Avro docs under examples/mr-example. To build the project, including the code generation of the User schema, run:
Next, run GenerateData from examples/mr-examples to create an Avro data file, input/users.avro, containing 20 Users with favorite colors chosen randomly from a list:
mvn exec:java -q -Dexec.mainClass=example.GenerateData
Besides creating the data file, GenerateData prints the JSON representations of the Users generated to stdout, for example:
{"name": "user", "favorite_number": null, "favorite_color": "red"}
{"name": "user", "favorite_number": null, "favorite_color": "green"}
{"name": "user", "favorite_number": null, "favorite_color": "purple"}
{"name": "user", "favorite_number": null, "favorite_color": null}
...
Now we’re ready to run ColorCount. We specify our freshly-generated input folder as the input path and output as our output folder (note that MapReduce will not start a job if the output folder already exists):
mvn exec:java -q -Dexec.mainClass=example.MapredColorCount -Dexec.args="input output"
Once ColorCount completes, checking the contents of the new output directory should yield the following:
$ ls output/
part-00000.avro _SUCCESS
You can check the contents of the generated Avro file using the avro-tools jar:
$ java -jar /path/to/avro-tools-1.8.2.jar tojson output/part-00000.avro
{"value": 3, "key": "blue"}
{"value": 7, "key": "green"}
{"value": 1, "key": "none"}
{"value": 2, "key": "orange"}
{"value": 3, "key": "purple"}
{"value": 2, "key": "red"}
{"value": 2, "key": "yellow"}
Now let’s go over the ColorCount example in detail.
Mapper - org.apache.hadoop.mapred API
The easiest way to use Avro data files as input to a MapReduce job is to subclass AvroMapper. An AvroMapper defines a map function that takes an Avro datum as input and outputs a key/value pair represented as a Pair record. In the ColorCount example, ColorCountMapper is an AvroMapper that takes a User as input and outputs a Pair<CharSequence, Integer>>, where the CharSequence key is the user’s favorite color and the Integer value is 1.
public static class ColorCountMapper extends AvroMapper<User, Pair<CharSequence, Integer>> {
@Override
public void map(User user, AvroCollector<Pair<CharSequence, Integer>> collector, Reporter reporter)
throws IOException {
CharSequence color = user.getFavoriteColor();
// We need this check because the User.favorite_color field has type ["string", "null"]
if (color == null) {
color = "none";
}
collector.collect(new Pair<CharSequence, Integer>(color, 1));
}
}
In order to use our AvroMapper, we must call AvroJob.setMapperClass and AvroJob.setInputSchema.
AvroJob.setMapperClass(conf, ColorCountMapper.class);
AvroJob.setInputSchema(conf, User.getClassSchema());
Note that AvroMapper does not implement the Mapper interface. Under the hood, the specified Avro data files are deserialized into AvroWrappers containing the actual data, which are processed by a Mapper that calls the configured AvroMapper’s map function. AvroJob.setInputSchema sets up the relevant configuration parameters needed to make this happen, thus you should not need to call JobConf.setMapperClass, JobConf.setInputFormat, JobConf.setMapOutputKeyClass, JobConf.setMapOutputValueClass, or JobConf.setOutputKeyComparatorClass.
Mapper - org.apache.hadoop.mapreduce API
This document will not go into all the differences between the mapred and mapreduce APIs, however will describe the main differences. As you can see, ColorCountMapper is now a subclass of the Hadoop Mapper class and is passed an AvroKey as it’s key. Additionally, the AvroJob method calls were slightly changed.
public static class ColorCountMapper extends
Mapper<AvroKey<User>, NullWritable, Text, IntWritable> {
@Override
public void map(AvroKey<User> key, NullWritable value, Context context)
throws IOException, InterruptedException {
CharSequence color = key.datum().getFavoriteColor();
if (color == null) {
color = "none";
}
context.write(new Text(color.toString()), new IntWritable(1));
}
}
Reducer - org.apache.hadoop.mapred API
Analogously to AvroMapper, an AvroReducer defines a reducer function that takes the key/value types output by an AvroMapper (or any mapper that outputs Pairs) and outputs a key/value pair represented a Pair record. In the ColorCount example, ColorCountReducer is an AvroReducer that takes the CharSequence key representing a favorite color and the Iterable<Integer> representing the counts for that color (they should all be 1 in this example) and adds up the counts.
public static class ColorCountReducer extends AvroReducer<CharSequence, Integer,
Pair<CharSequence, Integer>> {
@Override
public void reduce(CharSequence key, Iterable<Integer> values,
AvroCollector<Pair<CharSequence, Integer>> collector,
Reporter reporter)
throws IOException {
int sum = 0;
for (Integer value : values) {
sum += value;
}
collector.collect(new Pair<CharSequence, Integer>(key, sum));
}
}
In order to use our AvroReducer, we must call AvroJob.setReducerClass and AvroJob.setOutputSchema.
AvroJob.setReducerClass(conf, ColorCountReducer.class);
AvroJob.setOutputSchema(conf, Pair.getPairSchema(Schema.create(Type.STRING),
Schema.create(Type.INT)));
Note that AvroReducer does not implement the Reducer interface. The intermediate Pairs output by the mapper are split into AvroKeys and AvroValues, which are processed by a Reducer that calls the configured AvroReducer’s reduce function. AvroJob.setOutputSchema sets up the relevant configuration parameters needed to make this happen, thus you should not need to call JobConf.setReducerClass, JobConf.setOutputFormat, JobConf.setOutputKeyClass, JobConf.setMapOutputKeyClass, JobConf.setMapOutputValueClass, or JobConf.setOutputKeyComparatorClass.
Reduce - org.apache.hadoop.mapreduce API
As before we not detail every difference between the APIs. As with the Mapper change ColorCountReducer is now a subclass of Reducer and AvroKey and AvroValue are emitted. Additionally, the AvroJob method calls were slightly changed.
public static class ColorCountReducer extends
Reducer<Text, IntWritable, AvroKey<CharSequence>, AvroValue<Integer>> {
@Override
public void reduce(Text key, Iterable<IntWritable> values,
Context context) throws IOException, InterruptedException {
int sum = 0;
for (IntWritable value : values) {
sum += value.get();
}
context.write(new AvroKey<CharSequence>(key.toString()), new AvroValue<Integer>(sum));
}
}
Learning more
The mapred API allows users to mix Avro AvroMappers and AvroReducers with non-Avro Mappers and Reducers and the mapreduce API allows users input Avro and output non-Avro or vice versa.
The mapred package has API org.apache.avro.mapred documentation as does the org.apache.avro.mapreduce package. MapReduce API (org.apache.hadoop.mapreduce). Similarily to the mapreduce package, it’s possible with the mapred API to implement your own Mappers and Reducers directly using the public classes provided in these libraries. See the AvroWordCount application, found under examples/mr-example/src/main/java/example/AvroWordCount.java in the Avro documentation, for an example of implementing a Reducer that outputs Avro data using the old MapReduce API. See the MapReduceAvroWordCount application, found under examples/mr-example/src/main/java/example/MapReduceAvroWordCount.java in the Avro documentation, for an example of implementing a Reducer that outputs Avro data using the new MapReduce API.
9.6 - IDL Language
Introduction
This document defines Avro IDL, a higher-level language for authoring Avro schemata. Before reading this document, you should have familiarity with the concepts of schemata and protocols, as well as the various primitive and complex types available in Avro.
Overview
Purpose
The aim of the Avro IDL language is to enable developers to author schemata in a way that feels more similar to common programming languages like Java, C++, or Python. Additionally, the Avro IDL language may feel more familiar for those users who have previously used the interface description languages (IDLs) in other frameworks like Thrift, Protocol Buffers, or CORBA.
Usage
Each Avro IDL file defines a single Avro Protocol, and thus generates as its output a JSON-format Avro Protocol file with extension .avpr.
To convert a .avdl file into a .avpr file, it may be processed by the idl tool. For example:
$ java -jar avroj-tools.jar idl src/test/idl/input/namespaces.avdl /tmp/namespaces.avpr
$ head /tmp/namespaces.avpr
{
"protocol" : "TestNamespace",
"namespace" : "avro.test.protocol",
The idl tool can also process input to and from stdin and stdout. See idl --help for full usage information.
A Maven plugin is also provided to compile .avdl files. To use it, add something like the following to your pom.xml:
<build>
<plugins>
<plugin>
<groupId>org.apache.avro</groupId>
<artifactId>avro-maven-plugin</artifactId>
<executions>
<execution>
<goals>
<goal>idl-protocol</goal>
</goals>
</execution>
</executions>
</plugin>
</plugins>
</build>
Defining a Protocol in Avro IDL
An Avro IDL file consists of exactly one protocol definition. The minimal protocol is defined by the following code:
This is equivalent to (and generates) the following JSON protocol definition:
{
"protocol" : "MyProtocol",
"types" : [ ],
"messages" : {
}
}
The namespace of the protocol may be changed using the @namespace annotation:
@namespace("mynamespace")
protocol MyProtocol {
}
This notation is used throughout Avro IDL as a way of specifying properties for the annotated element, as will be described later in this document.
Protocols in Avro IDL can contain the following items:
- Imports of external protocol and schema files.
- Definitions of named schemata, including records, errors, enums, and fixeds.
- Definitions of RPC messages
Imports
Files may be imported in one of three formats:
-
An IDL file may be imported with a statement like:
import idl "foo.avdl";
-
A JSON protocol file may be imported with a statement like:
import protocol "foo.avpr";
-
A JSON schema file may be imported with a statement like:
import schema "foo.avsc";
Messages and types in the imported file are added to this file’s protocol.
Imported file names are resolved relative to the current IDL file.
Defining an Enumeration
Enums are defined in Avro IDL using a syntax similar to C or Java.
enum Suit {
SPADES, DIAMONDS, CLUBS, HEARTS
}
Note that, unlike the JSON format, anonymous enums cannot be defined.
Defining a Fixed Length Field
Fixed fields are defined using the following syntax:
fixed MD5(16);
This example defines a fixed-length type called MD5 which contains 16 bytes.
Defining Records and Errors
Records are defined in Avro IDL using a syntax similar to a struct definition in C:
record Employee {
string name;
boolean active = true;
long salary;
}
The above example defines a record with the name “Employee” with three fields.
To define an error, simply use the keyword error instead of record. For example:
error Kaboom {
string explanation;
int result_code = -1;
}
Each field in a record or error consists of a type and a name, optional property annotations and an optional default value.
A type reference in Avro IDL must be one of:
- A primitive type
- A logical type
- A named schema defined prior to this usage in the same Protocol
- A complex type (array, map, or union)
Primitive Types
The primitive types supported by Avro IDL are the same as those supported by Avro’s JSON format. This list includes int, long, string, boolean, float, double, null, and bytes.
Logical Types
Some of the logical types supported by Avro’s JSON format are also supported by Avro IDL. The currently supported types are:
For example:
record Job {
string jobid;
date submitDate;
time_ms submitTime;
timestamp_ms finishTime;
decimal(9,2) finishRatio;
}
References to Named Schemata
If a named schema has already been defined in the same Avro IDL file, it may be referenced by name as if it were a primitive type:
record Card {
Suit suit; // refers to the enum Card defined above
int number;
}
Default Values
Default values for fields may be optionally specified by using an equals sign after the field name followed by a JSON expression indicating the default value. This JSON is interpreted as described in the spec.
Complex Types
Arrays
Array types are written in a manner that will seem familiar to C++ or Java programmers. An array of any type t is denoted array<t>. For example, an array of strings is denoted array<string>, and a multidimensional array of Foo records would be array<array<Foo>>.
Maps
Map types are written similarly to array types. An array that contains values of type t is written map<t>. As in the JSON schema format, all maps contain string-type keys.
Unions
Union types are denoted as union { typeA, typeB, typeC, … }. For example, this record contains a string field that is optional (unioned with null):
record RecordWithUnion {
union { null, string } optionalString;
}
Note that the same restrictions apply to Avro IDL unions as apply to unions defined in the JSON format; namely, a record may not contain multiple elements of the same type.
Defining RPC Messages
The syntax to define an RPC message within a Avro IDL protocol is similar to the syntax for a method declaration within a C header file or a Java interface. To define an RPC message add which takes two arguments named foo and bar, returning an int, simply include the following definition within the protocol:
int add(int foo, int bar = 0);
Message arguments, like record fields, may specify default values.
To define a message with no response, you may use the alias void, equivalent to the Avro null type:
void logMessage(string message);
If you have previously defined an error type within the same protocol, you may declare that a message can throw this error using the syntax:
void goKaboom() throws Kaboom;
To define a one-way message, use the keyword oneway after the parameter list, for example:
void fireAndForget(string message) oneway;
Other Language Features
All Java-style comments are supported within a Avro IDL file. Any text following // on a line is ignored, as is any text between /* and */, possibly spanning multiple lines.
Comments that begin with /** are used as the documentation string for the type or field definition that follows the comment.
Escaping Identifiers
Occasionally, one will need to use a reserved language keyword as an identifier. In order to do so, backticks (`) may be used to escape the identifier. For example, to define a message with the literal name error, you may write:
This syntax is allowed anywhere an identifier is expected.
Annotations for Ordering and Namespaces
Java-style annotations may be used to add additional properties to types and fields throughout Avro IDL.
For example, to specify the sort order of a field within a record, one may use the @order annotation before the field name as follows:
record MyRecord {
string @order("ascending") myAscendingSortField;
string @order("descending") myDescendingField;
string @order("ignore") myIgnoredField;
}
A field’s type may also be preceded by annotations, e.g.:
record MyRecord {
@java-class("java.util.ArrayList") array<string> myStrings;
}
This can be used to support java classes that can be serialized/deserialized via their toString/String constructor, e.g.:
record MyRecord {
@java-class("java.math.BigDecimal") string value;
@java-key-class("java.io.File") map<string> fileStates;
array<@java-class("java.math.BigDecimal") string> weights;
}
Similarly, a @namespace annotation may be used to modify the namespace when defining a named schema. For example:
@namespace("org.apache.avro.firstNamespace")
protocol MyProto {
@namespace("org.apache.avro.someOtherNamespace")
record Foo {}
record Bar {}
}
will define a protocol in the firstNamespace namespace. The record Foo will be defined in someOtherNamespace and Bar will be defined in firstNamespace as it inherits its default from its container.
Type and field aliases are specified with the @aliases annotation as follows:
@aliases(["org.old.OldRecord", "org.ancient.AncientRecord"])
record MyRecord {
string @aliases(["oldField", "ancientField"]) myNewField;
}
Some annotations like those listed above are handled specially. All other annotations are added as properties to the protocol, message, schema or field.
Complete Example
The following is a complete example of a Avro IDL file that shows most of the above features:
/**
* An example protocol in Avro IDL
*/
@namespace("org.apache.avro.test")
protocol Simple {
@aliases(["org.foo.KindOf"])
enum Kind {
FOO,
BAR, // the bar enum value
BAZ
}
fixed MD5(16);
record TestRecord {
@order("ignore")
string name;
@order("descending")
Kind kind;
MD5 hash;
union { MD5, null} @aliases(["hash"]) nullableHash;
array<long> arrayOfLongs;
}
error TestError {
string message;
}
string hello(string greeting);
TestRecord echo(TestRecord `record`);
int add(int arg1, int arg2);
bytes echoBytes(bytes data);
void `error`() throws TestError;
void ping() oneway;
}
Additional examples may be found in the Avro source tree under the src/test/idl/input directory.
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
9.7 - SASL profile
Introduction
SASL (RFC 2222) provides a framework for authentication and security of network protocols. Each protocol that uses SASL is meant to define a SASL profile. This document provides a SASL profile for connection-based Avro RPC.
Overview
SASL negotiation proceeds as a series of message interactions over a connection between a client and server using a selected SASL mechanism. The client starts this negotiation by sending its chosen mechanism name with an initial (possibly empty) message. Negotiation proceeds with the exchange of messages until either side indicates success or failure. The content of the messages is mechanism-specific. If the negotiation succeeds, then the session can proceed over the connection, otherwise it must be abandoned.
Some mechanisms continue to process session data after negotiation (e.g., encrypting it), while some specify that further session data is transmitted unmodified.
Negotiation
Commands
Avro SASL negotiation uses four one-byte commands.
- 0: START Used in a client’s initial message.
- 1: CONTINUE Used while negotiation is ongoing.
- 2: FAIL Terminates negotiation unsuccessfully.
- 3: COMPLETE Terminates negotiation successfully.
The format of a START message is:
| 0 | 4-byte mechanism name length | mechanism name | 4-byte payload length | payload data |
The format of a CONTINUE message is:
| 1 | 4-byte payload length | payload data |
The format of a FAIL message is:
| 2 | 4-byte message length | UTF-8 message |
The format of a COMPLETE message is:
| 3 | 4-byte payload length | payload data |
Process
Negotiation is initiated by a client sending a START command containing the client’s chosen mechanism name and any mechanism-specific payload data.
The server and client then interchange some number (possibly zero) of CONTINUE messages. Each message contains payload data that is processed by the security mechanism to generate the next message.
Once either the client or server send a FAIL message then negotiation has failed. UTF-8-encoded text is included in the failure message. Once either a FAIL message has been sent or received, or any other error occurs in the negotiation, further communication on this connection must cease.
Once either the client or server send a COMPLETE message then negotiation has completed successfully. Session data may now be transmitted over the connection until it is closed by either side.
Session Data
If no SASL QOP (quality of protection) is negotiated, then all subsequent writes to/reads over this connection are written/read unmodified. In particular, messages use Avro framing, and are of the form:
| 4-byte frame length | frame data | ... | 4 zero bytes |
If a SASL QOP is negotiated, then it must be used by the connection for all subsequent messages. This is done by wrapping each non-empty frame written using the security mechanism and unwrapping each non-empty frame read. The length written in each non-empty frame is the length of the wrapped data. Complete frames must be passed to the security mechanism for unwrapping. Unwrapped data is then passed to the application as the content of the frame.
If at any point processing fails due to wrapping, unwrapping or framing errors, then all further communication on this connection must cease.
Anonymous Mechanism
The SASL anonymous mechanism (RFC 2245) is quite simple to implement. In particular, an initial anonymous request may be prefixed by the following static sequence:
| 0 | 0009 | ANONYMOUS | 0000 |
If a server uses the anonymous mechanism, it should check that the mechanism name in the start message prefixing the first request received is ‘ANONYMOUS’, then simply prefix its initial response with a COMPLETE message of:
| 3 | 0000 |
If an anonymous server recieves some other mechanism name, then it may respond with a FAIL message as simple as:
| 2 | 0000 |
Note that the anonymous mechanism need add no additional round-trip messages between client and server. The START message can be piggybacked on the initial request and the COMPLETE or FAIL message can be piggybacked on the initial response.
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
9.8 - Wiki
The Wiki page can be found here.
9.9 - FAQ
What is Avro?
Avro is a data serialization system.
Avro provides:
- Rich data structures.
- A compact, fast, binary data format.
- A container file, to store persistent data.
- Remote procedure call (RPC).
- Simple integration with dynamic languages. Code generation is not required to read or write data files nor to use or implement RPC protocols. Code generation as an optional optimization, only worth implementing for statically typed languages.
How can I get started quickly with Avro?
Check out the Quick Start Guide.
How do I statically compile a schema or protocol into generated code?
In Java
- Add the avro jar, the jackson-mapper-asl.jar and jackson-core-asl.jar to your CLASSPATH.
- Run java org.apache.avro.specific.SpecificCompiler .
This appears to be out of date, the SpecificCompiler requires two arguments, presumably an input and and output file, but it isn’t clear that this does.
Or use the Schema or Protocol Ant tasks. Avro’s build.xml provides examples of how these are used.
Lastly, you can also use the “avro-tools” jar which ships with an Avro release. Just use the “compile (schema|protocol)” command.
How are Strings represented in Java?
They use org.apache.avro.util.Utf8, not java.lang.String.
More generally, how do Avro types map to Java types?
The mappings are documented in the package javadoc for generic, specific and reflect API.
From Doug Cutting:
HDFS splits files into blocks, and mapreduce runs a map task for each block. When the task starts, it needs to be able to seek into the file to the start of the block process through the block’s end. If the file were, e.g., a gzip file, this would not be possible, since gzip files must be decompressed from the start. One cannot seek into the middle of a gzip file and start decompressing. So Hadoop’s SequenceFile places a marker periodically (~64k) in the file at record and compression boundaries, where processing can be sensibly started. Then, when a map task starts processing an HDFS block, it finds the first marker after the block’s start and continues through the first marker in the next block of the file. This requires a bit of non-local access (~0.1%). Avro’s data file uses the same method as SequenceFile.
Why isn’t every value in Avro nullable?
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 of every value stored and
corresponding computational costs to create this bit on write and
interpret it on read. These costs are wasted when values may not in
fact be null, as is the case in many datasets. In Avro such costs are
only paid when values may actually be null.
Also, allowing values to be null is a well-known source of errors. In
Avro, a value declared as non-null will always be non-null and programs
need not test for null values when processing it nor will they ever fail
for lack of such tests.
Tony Hoare calls his invention of null references his “Billion Dollar
Mistake”.
http://qconlondon.com/london-2009/presentation/Null+References:+The+Billion+Dollar+Mistake
Also note that in some programming languages not all values are permitted to be null. For example, in Java, values of type boolean, byte, short, char, int, float, long, and double may not be null.
How can I serialize directly to/from a byte array?
As pointed out in the specification, Avro data should always be stored with its schema. The Avro provided classes DataFileWriter, DataFileReader, and DataFileStream all ensure this by serializing the Schema in a container header. In some special cases, such as when implementing a new storage system or writing unit tests, you may need to write and read directly with the bare Avro serialized values. The following examples use code generated for Java from an Avro Schema, specifically the User example from the Quickstart guide.
Serializing to a byte array
This example takes a User object and returns a newly allocated byte array with the Avro serialization of that user.
ByteArrayOutputStream out = new ByteArrayOutputStream();
BinaryEncoder encoder = EncoderFactory.get().binaryEncoder(out, null);
DatumWriter<User> writer = new SpecificDatumWriter<User>(User.getClassSchema());
writer.write(user, encoder);
encoder.flush();
out.close();
byte[] serializedBytes = out.toByteArray();
Deserializing from a byte array
This example takes a byte array containing the Avro serialization of a user and returns a User object.SpecificDatumReader reader = new SpecificDatumReader(User.getClassSchema());
Decoder decoder = DecoderFactory.get().binaryDecoder(bytes, null);
User user = reader.read(null, decoder);
Note: If you are serializing or deserializing in a loop or as a method, you should be reusing objects, readers and/or writers. Check the JavaDoc to see how to reuse the objects.
10 - Apache Avro™ 1.8.1 Documentation
Introduction
Apache Avro™ is a data serialization system.
Avro provides:
- Rich data structures.
- A compact, fast, binary data format.
- A container file, to store persistent data.
- Remote procedure call (RPC).
- Simple integration with dynamic languages. Code generation is not required to read or write data files nor to use or implement RPC protocols. Code generation as an optional optimization, only worth implementing for statically typed languages.
Schemas
Avro relies on schemas. When Avro data is read, the schema used when writing it is always present. This permits each datum to be written with no per-value overheads, making serialization both fast and small. This also facilitates use with dynamic, scripting languages, since data, together with its schema, is fully self-describing.
When Avro data is stored in a file, its schema is stored with it, so that files may be processed later by any program. If the program reading the data expects a different schema this can be easily resolved, since both schemas are present.
When Avro is used in RPC, the client and server exchange schemas in the connection handshake. (This can be optimized so that, for most calls, no schemas are actually transmitted.) Since both client and server both have the other’s full schema, correspondence between same named fields, missing fields, extra fields, etc. can all be easily resolved.
Avro schemas are defined with JSON . This facilitates implementation in languages that already have JSON libraries.
Comparison with other systems
Avro provides functionality similar to systems such as Thrift, Protocol Buffers, etc. Avro differs from these systems in the following fundamental aspects.
- Dynamic typing: Avro does not require that code be generated. Data is always accompanied by a schema that permits full processing of that data without code generation, static datatypes, etc. This facilitates construction of generic data-processing systems and languages.
- Untagged data: Since the schema is present when data is read, considerably less type information need be encoded with data, resulting in smaller serialization size.
- No manually-assigned field IDs: When a schema changes, both the old and new schema are always present when processing data, so differences may be resolved symbolically, using field names.
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
10.1 - Getting Started (Java)
This is a short guide for getting started with Apache Avro™ using Java. This guide only covers using Avro for data serialization; see Patrick Hunt’s Avro RPC Quick Start for a good introduction to using Avro for RPC.
Download
Avro implementations for C, C++, C#, Java, PHP, Python, and Ruby can be downloaded from the Apache Avro™ Releases page. This guide uses Avro 1.8.1, the latest version at the time of writing. For the examples in this guide, download avro-1.8.1.jar and avro-tools-1.8.1.jar. The Avro Java implementation also depends on the Jackson JSON library. From the Jackson download page, download the core-asl and mapper-asl jars. Add avro-1.8.1.jar and the Jackson jars to your project’s classpath (avro-tools will be used for code generation).
Alternatively, if you are using Maven, add the following dependency to your POM:
<dependency>
<groupId>org.apache.avro</groupId>
<artifactId>avro</artifactId>
<version>1.8.1</version>
</dependency>
As well as the Avro Maven plugin (for performing code generation):
<plugin>
<groupId>org.apache.avro</groupId>
<artifactId>avro-maven-plugin</artifactId>
<version>1.8.1</version>
<executions>
<execution>
<phase>generate-sources</phase>
<goals>
<goal>schema</goal>
</goals>
<configuration>
<sourceDirectory>${project.basedir}/src/main/avro/</sourceDirectory>
<outputDirectory>${project.basedir}/src/main/java/</outputDirectory>
</configuration>
</execution>
</executions>
</plugin>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-compiler-plugin</artifactId>
<configuration>
<source>1.6</source>
<target>1.6</target>
</configuration>
</plugin>
You may also build the required Avro jars from source. Building Avro is beyond the scope of this guide; see the Build Documentation page in the wiki for more information.
Defining a schema
Avro schemas are defined using JSON. Schemas are composed of primitive types (null, boolean, int, long, float, double, bytes, and string) and complex types (record, enum, array, map, union, and fixed). You can learn more about Avro schemas and types from the specification, but for now let’s start with a simple schema example, user.avsc:
{"namespace": "example.avro",
"type": "record",
"name": "User",
"fields": [
{"name": "name", "type": "string"},
{"name": "favorite_number", "type": ["int", "null"]},
{"name": "favorite_color", "type": ["string", "null"]}
]
}
This schema defines a record representing a hypothetical user. (Note that a schema file can only contain a single schema definition.) At minimum, a record definition must include its type (“type”: “record”), a name (“name”: “User”), and fields, in this case name, favorite_number, and favorite_color. We also define a namespace (“namespace”: “example.avro”), which together with the name attribute defines the “full name” of the schema (example.avro.User in this case).
Fields are defined via an array of objects, each of which defines a name and type (other attributes are optional, see the record specification for more details). The type attribute of a field is another schema object, which can be either a primitive or complex type. For example, the name field of our User schema is the primitive type string, whereas the favorite_number and favorite_color fields are both unions, represented by JSON arrays. unions are a complex type that can be any of the types listed in the array; e.g., favorite_number can either be an int or null, essentially making it an optional field.
Serializing and deserializing with code generation
Compiling the schema
Code generation allows us to automatically create classes based on our previously-defined schema. Once we have defined the relevant classes, there is no need to use the schema directly in our programs. We use the avro-tools jar to generate code as follows:
java -jar /path/to/avro-tools-1.8.1.jar compile schema <schema file> <destination>
This will generate the appropriate source files in a package based on the schema’s namespace in the provided destination folder. For instance, to generate a User class in package example.avro from the schema defined above, run
java -jar /path/to/avro-tools-1.8.1.jar compile schema user.avsc .
Note that if you using the Avro Maven plugin, there is no need to manually invoke the schema compiler; the plugin automatically performs code generation on any .avsc files present in the configured source directory.
Creating Users
Now that we’ve completed the code generation, let’s create some Users, serialize them to a data file on disk, and then read back the file and deserialize the User objects.
First let’s create some Users and set their fields.
User user1 = new User();
user1.setName("Alyssa");
user1.setFavoriteNumber(256);
// Leave favorite color null
// Alternate constructor
User user2 = new User("Ben", 7, "red");
// Construct via builder
User user3 = User.newBuilder()
.setName("Charlie")
.setFavoriteColor("blue")
.setFavoriteNumber(null)
.build();
As shown in this example, Avro objects can be created either by invoking a constructor directly or by using a builder. Unlike constructors, builders will automatically set any default values specified in the schema. Additionally, builders validate the data as it set, whereas objects constructed directly will not cause an error until the object is serialized. However, using constructors directly generally offers better performance, as builders create a copy of the datastructure before it is written.
Note that we do not set user1’s favorite color. Since that record is of type [“string”, “null”], we can either set it to a string or leave it null; it is essentially optional. Similarly, we set user3’s favorite number to null (using a builder requires setting all fields, even if they are null).
Serializing
Now let’s serialize our Users to disk.
/// Serialize user1, user2 and user3 to disk
DatumWriter<User> userDatumWriter = new SpecificDatumWriter<User>(User.class);
DataFileWriter<User> dataFileWriter = new DataFileWriter<User>(userDatumWriter);
dataFileWriter.create(user1.getSchema(), new File("users.avro"));
dataFileWriter.append(user1);
dataFileWriter.append(user2);
dataFileWriter.append(user3);
dataFileWriter.close();
We create a DatumWriter, which converts Java objects into an in-memory serialized format. The SpecificDatumWriter class is used with generated classes and extracts the schema from the specified generated type.
Next we create a DataFileWriter, which writes the serialized records, as well as the schema, to the file specified in the dataFileWriter.create call. We write our users to the file via calls to the dataFileWriter.append method. When we are done writing, we close the data file.
Deserializing
Finally, let’s deserialize the data file we just created.
// Deserialize Users from disk
DatumReader<User> userDatumReader = new SpecificDatumReader<User>(User.class);
DataFileReader<User> dataFileReader = new DataFileReader<User>(file, userDatumReader);
User user = null;
while (dataFileReader.hasNext()) {
// Reuse user object by passing it to next(). This saves us from
// allocating and garbage collecting many objects for files with
// many items.
user = dataFileReader.next(user);
System.out.println(user);
}
This snippet will output:
{"name": "Alyssa", "favorite_number": 256, "favorite_color": null}
{"name": "Ben", "favorite_number": 7, "favorite_color": "red"}
{"name": "Charlie", "favorite_number": null, "favorite_color": "blue"}
Deserializing is very similar to serializing. We create a SpecificDatumReader, analogous to the SpecificDatumWriter we used in serialization, which converts in-memory serialized items into instances of our generated class, in this case User. We pass the DatumReader and the previously created File to a DataFileReader, analogous to the DataFileWriter, which reads the data file on disk.
Next we use the DataFileReader to iterate through the serialized Users and print the deserialized object to stdout. Note how we perform the iteration: we create a single User object which we store the current deserialized user in, and pass this record object to every call of dataFileReader.next. This is a performance optimization that allows the DataFileReader to reuse the same User object rather than allocating a new User for every iteration, which can be very expensive in terms of object allocation and garbage collection if we deserialize a large data file. While this technique is the standard way to iterate through a data file, it’s also possible to use for (User user : dataFileReader) if performance is not a concern.
Compiling and running the example code
This example code is included as a Maven project in the examples/java-example directory in the Avro docs. From this directory, execute the following commands to build and run the example:
$ mvn compile # includes code generation via Avro Maven plugin
$ mvn -q exec:java -Dexec.mainClass=example.SpecificMain
Serializing and deserializing without code generation
Data in Avro is always stored with its corresponding schema, meaning we can always read a serialized item regardless of whether we know the schema ahead of time. This allows us to perform serialization and deserialization without code generation.
Let’s go over the same example as in the previous section, but without using code generation: we’ll create some users, serialize them to a data file on disk, and then read back the file and deserialize the users objects.
Creating users
First, we use a Parser to read our schema definition and create a Schema object.
Schema schema = new Schema.Parser().parse(new File("user.avsc"));
Using this schema, let’s create some users.
GenericRecord user1 = new GenericData.Record(schema);
user1.put("name", "Alyssa");
user1.put("favorite_number", 256);
// Leave favorite color null
GenericRecord user2 = new GenericData.Record(schema);
user2.put("name", "Ben");
user2.put("favorite_number", 7);
user2.put("favorite_color", "red");
Since we’re not using code generation, we use GenericRecords to represent users. GenericRecord uses the schema to verify that we only specify valid fields. If we try to set a non-existent field (e.g., user1.put(“favorite_animal”, “cat”)), we’ll get an AvroRuntimeException when we run the program.
Note that we do not set user1’s favorite color. Since that record is of type [“string”, “null”], we can either set it to a string or leave it null; it is essentially optional.
Serializing
Now that we’ve created our user objects, serializing and deserializing them is almost identical to the example above which uses code generation. The main difference is that we use generic instead of specific readers and writers.
First we’ll serialize our users to a data file on disk.
// Serialize user1 and user2 to disk
File file = new File("users.avro");
DatumWriter<GenericRecord> datumWriter = new GenericDatumWriter<GenericRecord>(schema);
DataFileWriter<GenericRecord> dataFileWriter = new DataFileWriter<GenericRecord>(datumWriter);
dataFileWriter.create(schema, file);
dataFileWriter.append(user1);
dataFileWriter.append(user2);
dataFileWriter.close();
We create a DatumWriter, which converts Java objects into an in-memory serialized format. Since we are not using code generation, we create a GenericDatumWriter. It requires the schema both to determine how to write the GenericRecords and to verify that all non-nullable fields are present.
As in the code generation example, we also create a DataFileWriter, which writes the serialized records, as well as the schema, to the file specified in the dataFileWriter.create call. We write our users to the file via calls to the dataFileWriter.append method. When we are done writing, we close the data file.
Deserializing
Finally, we’ll deserialize the data file we just created.
// Deserialize users from disk
DatumReader<GenericRecord> datumReader = new GenericDatumReader<GenericRecord>(schema);
DataFileReader<GenericRecord> dataFileReader = new DataFileReader<GenericRecord>(file, datumReader);
GenericRecord user = null;
while (dataFileReader.hasNext()) {
// Reuse user object by passing it to next(). This saves us from
// allocating and garbage collecting many objects for files with
// many items.
user = dataFileReader.next(user);
System.out.println(user);
This outputs:
{"name": "Alyssa", "favorite_number": 256, "favorite_color": null}
{"name": "Ben", "favorite_number": 7, "favorite_color": "red"}
Deserializing is very similar to serializing. We create a GenericDatumReader, analogous to the GenericDatumWriter we used in serialization, which converts in-memory serialized items into GenericRecords. We pass the DatumReader and the previously created File to a DataFileReader, analogous to the DataFileWriter, which reads the data file on disk.
Next, we use the DataFileReader to iterate through the serialized users and print the deserialized object to stdout. Note how we perform the iteration: we create a single GenericRecord object which we store the current deserialized user in, and pass this record object to every call of dataFileReader.next. This is a performance optimization that allows the DataFileReader to reuse the same record object rather than allocating a new GenericRecord for every iteration, which can be very expensive in terms of object allocation and garbage collection if we deserialize a large data file. While this technique is the standard way to iterate through a data file, it’s also possible to use for (GenericRecord user : dataFileReader) if performance is not a concern.
Compiling and running the example code
This example code is included as a Maven project in the examples/java-example directory in the Avro docs. From this directory, execute the following commands to build and run the example:
$ mvn compile
$ mvn -q exec:java -Dexec.mainClass=example.GenericMain
10.2 - Getting Started (Python)
This is a short guide for getting started with Apache Avro™ using Python. This guide only covers using Avro for data serialization; see Patrick Hunt’s Avro RPC Quick Start for a good introduction to using Avro for RPC.
Download
Avro implementations for C, C++, C#, Java, PHP, Python, and Ruby can be downloaded from the Apache Avro™ Releases page. This guide uses Avro 1.8.1, the latest version at the time of writing. Download and unzip avro-1.8.1.tar.gz, and install via python setup.py (this will probably require root privileges). Ensure that you can import avro from a Python prompt.
$ tar xvf avro-1.8.1.tar.gz
$ cd avro-1.8.1
$ sudo python setup.py install
$ python
>>> import avro # should not raise ImportError
Alternatively, you may build the Avro Python library from source. From your the root Avro directory, run the commands
$ cd lang/py/
$ ant
$ sudo python setup.py install
$ python
>>> import avro # should not raise ImportError
Defining a schema
Avro schemas are defined using JSON. Schemas are composed of primitive types (null, boolean, int, long, float, double, bytes, and string) and complex types (record, enum, array, map, union, and fixed). You can learn more about Avro schemas and types from the specification, but for now let’s start with a simple schema example, user.avsc:
{"namespace": "example.avro",
"type": "record",
"name": "User",
"fields": [
{"name": "name", "type": "string"},
{"name": "favorite_number", "type": ["int", "null"]},
{"name": "favorite_color", "type": ["string", "null"]}
]
}
This schema defines a record representing a hypothetical user. (Note that a schema file can only contain a single schema definition.) At minimum, a record definition must include its type (“type”: “record”), a name (“name”: “User”), and fields, in this case name, favorite_number, and favorite_color. We also define a namespace (“namespace”: “example.avro”), which together with the name attribute defines the “full name” of the schema (example.avro.User in this case).
Fields are defined via an array of objects, each of which defines a name and type (other attributes are optional, see the record specification for more details). The type attribute of a field is another schema object, which can be either a primitive or complex type. For example, the name field of our User schema is the primitive type string, whereas the favorite_number and favorite_color fields are both unions, represented by JSON arrays. unions are a complex type that can be any of the types listed in the array; e.g., favorite_number can either be an int or null, essentially making it an optional field.
Serializing and deserializing without code generation
Data in Avro is always stored with its corresponding schema, meaning we can always read a serialized item, regardless of whether we know the schema ahead of time. This allows us to perform serialization and deserialization without code generation. Note that the Avro Python library does not support code generation.
Try running the following code snippet, which serializes two users to a data file on disk, and then reads back and deserializes the data file:
import avro.schema
from avro.datafile import DataFileReader, DataFileWriter
from avro.io import DatumReader, DatumWriter
schema = avro.schema.parse(open("user.avsc", "rb").read())
writer = DataFileWriter(open("users.avro", "wb"), DatumWriter(), schema)
writer.append({"name": "Alyssa", "favorite_number": 256})
writer.append({"name": "Ben", "favorite_number": 7, "favorite_color": "red"})
writer.close()
reader = DataFileReader(open("users.avro", "rb"), DatumReader())
for user in reader:
print user
reader.close()
This outputs:
{u'favorite_color': None, u'favorite_number': 256, u'name': u'Alyssa'}
{u'favorite_color': u'red', u'favorite_number': 7, u'name': u'Ben'}
Do make sure that you open your files in binary mode (i.e. using the modes wb or rb respectively). Otherwise you might generate corrupt files due to automatic replacement of newline characters with the platform-specific representations.
Let’s take a closer look at what’s going on here.
schema = avro.schema.parse(open("user.avsc", "rb").read())
avro.schema.parse takes a string containing a JSON schema definition as input and outputs a avro.schema.Schema object (specifically a subclass of Schema, in this case RecordSchema). We’re passing in the contents of our user.avsc schema file here.
writer = DataFileWriter(open("users.avro", "wb"), DatumWriter(), schema)
We create a DataFileWriter, which we’ll use to write serialized items to a data file on disk. The DataFileWriter constructor takes three arguments:
- The file we’ll serialize to
- A DatumWriter, which is responsible for actually serializing the items to Avro’s binary format (DatumWriters can be used separately from DataFileWriters, e.g., to perform IPC Avro TODO: is this true??).
- The schema we’re using. The DataFileWriter needs the schema both to write the schema to the data file, and to verify that the items we write are valid items and write the appropriate fields.
writer.append({"name": "Alyssa", "favorite_number": 256})
writer.append({"name": "Ben", "favorite_number": 7, "favorite_color": "red"})
We use DataFileWriter.append to add items to our data file. Avro records are represented as Python dicts. Since the field favorite_color has type [“int”, “null”], we are not required to specify this field, as shown in the first append. Were we to omit the required name field, an exception would be raised. Any extra entries not corresponding to a field are present in the dict are ignored.
reader = DataFileReader(open("users.avro", "rb"), DatumReader())
We open the file again, this time for reading back from disk. We use a DataFileReader and DatumReader analagous to the DataFileWriter and DatumWriter above.
for user in reader:
print user
The DataFileReader is an iterator that returns dicts corresponding to the serialized items.
10.3 - Specification
Introduction
This document defines Apache Avro. It is intended to be the authoritative specification. Implementations of Avro must adhere to this document.
Schema Declaration
A Schema is represented in JSON by one of:
- A JSON string, naming a defined type.
- A JSON object, of the form:
{"type": "typeName" ...attributes...}
where typeName is either a primitive or derived type name, as defined below. Attributes not defined in this document are permitted as metadata, but must not affect the format of serialized data.
- A JSON array, representing a union of embedded types.
Primitive Types
The set of primitive type names is:
- null: no value
- boolean: a binary value
- int: 32-bit signed integer
- long: 64-bit signed integer
- float: single precision (32-bit) IEEE 754 floating-point number
- double: double precision (64-bit) IEEE 754 floating-point number
- bytes: sequence of 8-bit unsigned bytes
- string: unicode character sequence
Primitive types have no specified attributes.
Primitive type names are also defined type names. Thus, for example, the schema “string” is equivalent to:
Complex Types
Avro supports six kinds of complex types: records, enums, arrays, maps, unions and fixed.
Records
Records use the type name “record” and support three attributes:
- name: a JSON string providing the name of the record (required).
- namespace, a JSON string that qualifies the name;
- doc: a JSON string providing documentation to the user of this schema (optional).
- aliases: a JSON array of strings, providing alternate names for this record (optional).
- fields: a JSON array, listing fields (required). Each field is a JSON object with the following attributes:
- name: a JSON string providing the name of the field (required), and
- doc: a JSON string describing this field for users (optional).
- type: A JSON object defining a schema, or a JSON string naming a record definition (required).
- _default: A default value for this field, used when reading instances that lack this field (optional). 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. Default values for bytes and fixed fields are JSON strings, where Unicode code points 0-255 are mapped to unsigned 8-bit byte values 0-255.
field default values
| avro type |
json type |
example |
| null |
null |
null |
| boolean |
boolean |
true |
| int,long |
integer |
1 |
| float,double |
number |
1.1 |
| bytes |
string |
"\u00FF" |
| string |
string |
"foo" |
| record |
object |
{"a": 1} |
| enum |
string |
"FOO" |
| array |
array |
[1] |
| map |
object |
{"a": 1} |
| fixed |
string |
"\u00ff" |
- order: specifies how this field impacts sort ordering of this record (optional). Valid values are “ascending” (the default), “descending”, or “ignore”. For more details on how this is used, see the sort order section below.
- aliases: a JSON array of strings, providing alternate names for this field (optional).
For example, a linked-list of 64-bit values may be defined with:
{
"type": "record",
"name": "LongList",
"aliases": ["LinkedLongs"], // old name for this
"fields" : [
{"name": "value", "type": "long"}, // each element has a long
{"name": "next", "type": ["null", "LongList"]} // optional next element
]
}
Enums
Enums use the type name “enum” and support the following attributes:
- name: a JSON string providing the name of the enum (required).
- namespace, a JSON string that qualifies the name ;
- aliases: a JSON array of strings, providing alternate names for this enum (optional).
- doc: a JSON string providing documentation to the user of this schema (optional).
- symbols: a JSON array, listing symbols, as JSON strings (required). All symbols in an enum must be unique; duplicates are prohibited. Every symbol must match the regular expression [A-Za-z_][A-Za-z0-9_]* (the same requirement as for names).
- default: A default value for this enumeration, used during resolution when the reader encounters a symbol from the writer that isn’t defined in the reader’s schema (optional). The value provided here must be a JSON string that’s a member of the symbols array. See documentation on schema resolution for how this gets used.
For example, playing card suits might be defined with:
{
"type": "enum",
"name": "Suit",
"symbols" : ["SPADES", "HEARTS", "DIAMONDS", "CLUBS"]
}
Arrays
Arrays use the type name “array” and support a single attribute:
- items: the schema of the array’s items.
For example, an array of strings is declared with:
{
"type": "array",
"items" : "string"
}
Maps
Maps use the type name “map” and support one attribute:
- values: the schema of the map’s values.
Map keys are assumed to be strings.
For example, a map from string to long is declared with:
{
"type": "map",
"values" : "long"
}
Unions
Unions, as mentioned above, are represented using JSON arrays. For example, ["null", "string"] declares a schema which may be either a null or string.
(Note that when a default value is specified for a record field whose type is a union, the type of the default value must match the first element of the union. Thus, for unions containing “null”, the “null” is usually listed first, since the default value of such unions is typically null.)
Unions may not contain more than one schema with the same type, except for the named types record, fixed and enum. For example, unions containing two array types or two map types are not permitted, but two types with different names are permitted. (Names permit efficient resolution when reading and writing unions.)
Unions may not immediately contain other unions.
Fixed
Fixed uses the type name “fixed” and supports two attributes:
- name: a string naming this fixed (required).
- namespace, a string that qualifies the name;
- aliases: a JSON array of strings, providing alternate names for this enum (optional).
- size: an integer, specifying the number of bytes per value (required).
For example, 16-byte quantity may be declared with:
{"type": "fixed", "size": 16, "name": "md5"}
Names
Record, enums and fixed are named types. Each has a fullname that is composed of two parts; a name and a namespace. Equality of names is defined on the fullname.
The name portion of a fullname, record field names, and enum symbols must:
- start with [A-Za-z_]
- subsequently contain only [A-Za-z0-9_]
A namespace is a dot-separated sequence of such names. The empty string may also be used as a namespace to indicate the null namespace. Equality of names (including field names and enum symbols) as well as fullnames is case-sensitive.
In record, enum and fixed definitions, the fullname is determined in one of the following ways:
A name and namespace are both specified. For example, one might use “name”: “X”, “namespace”: “org.foo” to indicate the fullname org.foo.X.
-
A fullname is specified. If the name specified contains a dot, then it is assumed to be a fullname, and any namespace also specified is ignored. For example, use “name”: “org.foo.X” to indicate the fullname org.foo.X.
-
A name only is specified, i.e., a name that contains no dots. In this case the namespace is taken from the most tightly enclosing schema or protocol. For example, if “name”: “X” is specified, and this occurs within a field of the record definition of org.foo.Y, then the fullname is org.foo.X. If there is no enclosing namespace then the null namespace is used.
References to previously defined names are as in the latter two cases above: if they contain a dot they are a fullname, if they do not contain a dot, the namespace is the namespace of the enclosing definition.
Primitive type names have no namespace and their names may not be defined in any namespace.
A schema or protocol may not contain multiple definitions of a fullname. Further, a name must be defined before it is used (“before” in the depth-first, left-to-right traversal of the JSON parse tree, where the types attribute of a protocol is always deemed to come “before” the messages attribute.)
Aliases
Named types and fields may have aliases. An implementation may optionally use aliases to map a writer’s schema to the reader’s. This facilitates both schema evolution as well as processing disparate datasets.
Aliases function by re-writing the writer’s schema using aliases from the reader’s schema. For example, if the writer’s schema was named “Foo” and the reader’s schema is named “Bar” and has an alias of “Foo”, then the implementation would act as though “Foo” were named “Bar” when reading. Similarly, if data was written as a record with a field named “x” and is read as a record with a field named “y” with alias “x”, then the implementation would act as though “x” were named “y” when reading.
A type alias may be specified either as a fully namespace-qualified, or relative to the namespace of the name it is an alias for. For example, if a type named “a.b” has aliases of “c” and “x.y”, then the fully qualified names of its aliases are “a.c” and “x.y”.
Data Serialization
Avro data is always serialized with its schema. Files that store Avro data should always also include the schema for that data in the same file. Avro-based remote procedure call (RPC) systems must also guarantee that remote recipients of data have a copy of the schema used to write that data.
Because the schema used to write data is always available when the data is read, Avro data itself is not tagged with type information. The schema is required to parse data.
In general, both serialization and deserialization proceed as a depth-first, left-to-right traversal of the schema, serializing primitive types as they are encountered.
Encodings
Avro specifies two serialization encodings: binary and JSON. Most applications will use the binary encoding, as it is smaller and faster. But, for debugging and web-based applications, the JSON encoding may sometimes be appropriate.
Binary Encoding
Primitive Types
Primitive types are encoded in binary as follows:
- null is written as zero bytes.
- a boolean is written as a single byte whose value is either 0 (false) or 1 (true).
- int and long values are written using variable-length zig-zag coding. Some examples:
| value |
hex |
| 0 |
00 |
| -1 |
01 |
| 1 |
02 |
| -2 |
03 |
| 2 |
04 |
| … |
|
| -64 |
7f |
| 64 |
80 01 |
| … |
|
- a float is written as 4 bytes. The float is converted into a 32-bit integer using a method equivalent to Java’s floatToIntBits and then encoded in little-endian format.
- a double is written as 8 bytes. The double is converted into a 64-bit integer using a method equivalent to Java’s doubleToLongBits and then encoded in little-endian format.
- bytes are encoded as a long followed by that many bytes of data.
- a string is encoded as a long followed by that many bytes of UTF-8 encoded character data.
For example, the three-character string “foo” would be encoded as the long value 3 (encoded as hex 06) followed by the UTF-8 encoding of ‘f’, ‘o’, and ‘o’ (the hex bytes 66 6f 6f):
06 66 6f 6f
Complex Types
Complex types are encoded in binary as follows:
Records
A record is encoded by encoding the values of its fields in the order that they are declared. In other words, a record is encoded as just the concatenation of the encodings of its fields. Field values are encoded per their schema.
For example, the record schema
{
"type": "record",
"name": "test",
"fields" : [
{"name": "a", "type": "long"},
{"name": "b", "type": "string"}
]
}
An instance of this record whose a field has value 27 (encoded as hex 36) and whose b field has value “foo” (encoded as hex bytes 06 66 6f 6f), would be encoded simply as the concatenation of these, namely the hex byte sequence:
36 06 66 6f 6f
Enums
An enum is encoded by a int, representing the zero-based position of the symbol in the schema.
For example, consider the enum:
{"type": "enum", "name": "Foo", "symbols": ["A", "B", "C", "D"] }
This would be encoded by an int between zero and three, with zero indicating “A”, and 3 indicating “D”.
Arrays
Arrays are encoded as a series of blocks. Each block consists of a long count value, followed by that many array items. A block with count zero indicates the end of the array. Each item is encoded per the array’s item schema.
If a block’s count is negative, its absolute value is used, and the count is followed immediately by a long block size indicating the number of bytes in the block. This block size permits fast skipping through data, e.g., when projecting a record to a subset of its fields.
For example, the array schema
{"type": "array", "items": "long"}
an array containing the items 3 and 27 could be encoded as the long value 2 (encoded as hex 04) followed by long values 3 and 27 (encoded as hex 06 36) terminated by zero:
04 06 36 00
The blocked representation permits one to read and write arrays larger than can be buffered in memory, since one can start writing items without knowing the full length of the array.
Maps
Maps are encoded as a series of blocks. Each block consists of a long count value, followed by that many key/value pairs. A block with count zero indicates the end of the map. Each item is encoded per the map’s value schema.
If a block’s count is negative, its absolute value is used, and the count is followed immediately by a long block size indicating the number of bytes in the block. This block size permits fast skipping through data, e.g., when projecting a record to a subset of its fields.
The blocked representation permits one to read and write maps larger than can be buffered in memory, since one can start writing items without knowing the full length of the map.
Unions
A union is encoded by first writing a long value indicating the zero-based position within the union of the schema of its value. The value is then encoded per the indicated schema within the union.
For example, the union schema ["null","string"] would encode:
-
null as zero (the index of “null” in the union):
00
-
the string “a” as one (the index of “string” in the union, 1, encoded as hex 02), followed by the serialized string:
02 02 61
Fixed
Fixed instances are encoded using the number of bytes declared in the schema.
JSON Encoding
Except for unions, the JSON encoding is the same as is used to encode field default values.
The value of a union is encoded in JSON as follows:
- if its type is null, then it is encoded as a JSON null;
- otherwise it is encoded as a JSON object with one name/value pair whose name is the type’s name and whose value is the recursively encoded value. For Avro’s named types (record, fixed or enum) the user-specified name is used, for other types the type name is used.
For example, the union schema ["null","string","Foo"], where Foo is a record name, would encode:
- null as null;
- the string “a” as
{"string": "a"} and
- a Foo instance as
{"Foo": {...}}, where {...} indicates the JSON encoding of a Foo instance.
Note that a schema is still required to correctly process JSON-encoded data. For example, the JSON encoding does not distinguish between int and long, float and double, records and maps, enums and strings, etc.
Sort Order
Avro defines a standard sort order for data. This permits data written by one system to be efficiently sorted by another system. This can be an important optimization, as sort order comparisons are sometimes the most frequent per-object operation. Note also that Avro binary-encoded data can be efficiently ordered without deserializing it to objects.
Data items may only be compared if they have identical schemas. Pairwise comparisons are implemented recursively with a depth-first, left-to-right traversal of the schema. The first mismatch encountered determines the order of the items.
Two items with the same schema are compared according to the following rules.
- null data is always equal.
- boolean data is ordered with false before true.
- int, long, float and double data is ordered by ascending numeric value.
- bytes and fixed data are compared lexicographically by unsigned 8-bit values.
- string data is compared lexicographically by Unicode code point. Note that since UTF-8 is used as the binary encoding for strings, sorting of bytes and string binary data is identical.
- array data is compared lexicographically by element.
- enum data is ordered by the symbol’s position in the enum schema. For example, an enum whose symbols are
["z", "a"] would sort “z” values before “a” values.
- union data is first ordered by the branch within the union, and, within that, by the type of the branch. For example, an
["int", "string"] union would order all int values before all string values, with the ints and strings themselves ordered as defined above.
- record data is ordered lexicographically by field. If a field specifies that its order is:
- “ascending”, then the order of its values is unaltered.
- “descending”, then the order of its values is reversed.
- “ignore”, then its values are ignored when sorting.
- map data may not be compared. It is an error to attempt to compare data containing maps unless those maps are in an
"order":"ignore" record field.
Object Container Files
Avro includes a simple object container file format. A file has a schema, and all objects stored in the file must be written according to that schema, using binary encoding. Objects are stored in blocks that may be compressed. Syncronization markers are used between blocks to permit efficient splitting of files for MapReduce processing.
Files may include arbitrary user-specified metadata.
A file consists of:
- A file header, followed by
- one or more file data blocks.
A file header consists of:
- Four bytes, ASCII ‘O’, ‘b’, ‘j’, followed by 1.
- file metadata, including the schema.
- The 16-byte, randomly-generated sync marker for this file.
File metadata is written as if defined by the following map schema:
{"type": "map", "values": "bytes"}
All metadata properties that start with “avro.” are reserved. The following file metadata properties are currently used:
- avro.schema contains the schema of objects stored in the file, as JSON data (required).
- avro.codec the name of the compression codec used to compress blocks, as a string. Implementations are required to support the following codecs: “null” and “deflate”. If codec is absent, it is assumed to be “null”. The codecs are described with more detail below.
A file header is thus described by the following schema:
{"type": "record", "name": "org.apache.avro.file.Header",
"fields" : [
{"name": "magic", "type": {"type": "fixed", "name": "Magic", "size": 4}},
{"name": "meta", "type": {"type": "map", "values": "bytes"}},
{"name": "sync", "type": {"type": "fixed", "name": "Sync", "size": 16}}
]
}
A file data block consists of:
- A long indicating the count of objects in this block.
- A long indicating the size in bytes of the serialized objects in the current block, after any codec is applied
- The serialized objects. If a codec is specified, this is compressed by that codec.
- The file’s 16-byte sync marker.
Thus, each block’s binary data can be efficiently extracted or skipped without deserializing the contents. The combination of block size, object counts, and sync markers enable detection of corrupt blocks and help ensure data integrity.
Required Codecs
null
The “null” codec simply passes through data uncompressed.
deflate
The “deflate” codec writes the data block using the deflate algorithm as specified in RFC 1951, and typically implemented using the zlib library. Note that this format (unlike the “zlib format” in RFC 1950) does not have a checksum.
Optional Codecs
snappy
The “snappy” codec uses Google’s Snappy compression library. Each compressed block is followed by the 4-byte, big-endian CRC32 checksum of the uncompressed data in the block.
Protocol Declaration
Avro protocols describe RPC interfaces. Like schemas, they are defined with JSON text.
A protocol is a JSON object with the following attributes:
- protocol, a string, the name of the protocol (required);
- namespace, an optional string that qualifies the name;
- doc, an optional string describing this protocol;
- types, an optional list of definitions of named types (records, enums, fixed and errors). An error definition is just like a record definition except it uses “error” instead of “record”. Note that forward references to named types are not permitted.
- messages, an optional JSON object whose keys are message names and whose values are objects whose attributes are described below. No two messages may have the same name.
The name and namespace qualification rules defined for schema objects apply to protocols as well.
Messages
A message has attributes:
- a doc, an optional description of the message,
- a request, a list of named, typed parameter schemas (this has the same form as the fields of a record declaration);
- a response schema;
- an optional union of declared error schemas. The effective union has “string” prepended to the declared union, to permit transmission of undeclared “system” errors. For example, if the declared error union is
["AccessError"], then the effective union is ["string", "AccessError"]. When no errors are declared, the effective error union is ["string"]. Errors are serialized using the effective union; however, a protocol’s JSON declaration contains only the declared union.
- an optional one-way boolean parameter.
A request parameter list is processed equivalently to an anonymous record. Since record field lists may vary between reader and writer, request parameters may also differ between the caller and responder, and such differences are resolved in the same manner as record field differences.
The one-way parameter may only be true when the response type is "null" and no errors are listed.
Sample Protocol
For example, one may define a simple HelloWorld protocol with:
{
"namespace": "com.acme",
"protocol": "HelloWorld",
"doc": "Protocol Greetings",
"types": [
{"name": "Greeting", "type": "record", "fields": [
{"name": "message", "type": "string"}]},
{"name": "Curse", "type": "error", "fields": [
{"name": "message", "type": "string"}]}
],
"messages": {
"hello": {
"doc": "Say hello.",
"request": [{"name": "greeting", "type": "Greeting" }],
"response": "Greeting",
"errors": ["Curse"]
}
}
}
Message Transport
Messages may be transmitted via different transport mechanisms.
To the transport, a message is an opaque byte sequence.
A transport is a system that supports:
- transmission of request messages
- receipt of corresponding response messages
Servers may send a response message back to the client corresponding to a request message. The mechanism of correspondance is transport-specific. For example, in HTTP it is implicit, since HTTP directly supports requests and responses. But a transport that multiplexes many client threads over a single socket would need to tag messages with unique identifiers.
Transports may be either stateless or stateful. In a stateless transport, messaging assumes no established connection state, while stateful transports establish connections that may be used for multiple messages. This distinction is discussed further in the handshake section below.
HTTP as Transport
When HTTP is used as a transport, each Avro message exchange is an HTTP request/response pair. All messages of an Avro protocol should share a single URL at an HTTP server. Other protocols may also use that URL. Both normal and error Avro response messages should use the 200 (OK) response code. The chunked encoding may be used for requests and responses, but, regardless the Avro request and response are the entire content of an HTTP request and response. The HTTP Content-Type of requests and responses should be specified as “avro/binary”. Requests should be made using the POST method.
HTTP is used by Avro as a stateless transport.
Message Framing
Avro messages are framed as a list of buffers.
Framing is a layer between messages and the transport. It exists to optimize certain operations.
The format of framed message data is:
- a series of buffers, where each buffer consists of:
- a four-byte, big-endian buffer length, followed by
- that many bytes of buffer data.
- A message is always terminated by a zero-length buffer.
Framing is transparent to request and response message formats (described below). Any message may be presented as a single or multiple buffers.
Framing can permit readers to more efficiently get different buffers from different sources and for writers to more efficiently store different buffers to different destinations. In particular, it can reduce the number of times large binary objects are copied. For example, if an RPC parameter consists of a megabyte of file data, that data can be copied directly to a socket from a file descriptor, and, on the other end, it could be written directly to a file descriptor, never entering user space.
A simple, recommended, framing policy is for writers to create a new segment whenever a single binary object is written that is larger than a normal output buffer. Small objects are then appended in buffers, while larger objects are written as their own buffers. When a reader then tries to read a large object the runtime can hand it an entire buffer directly, without having to copy it.
Handshake
The purpose of the handshake is to ensure that the client and the server have each other’s protocol definition, so that the client can correctly deserialize responses, and the server can correctly deserialize requests. Both clients and servers should maintain a cache of recently seen protocols, so that, in most cases, a handshake will be completed without extra round-trip network exchanges or the transmission of full protocol text.
RPC requests and responses may not be processed until a handshake has been completed. With a stateless transport, all requests and responses are prefixed by handshakes. With a stateful transport, handshakes are only attached to requests and responses until a successful handshake response has been returned over a connection. After this, request and response payloads are sent without handshakes for the lifetime of that connection.
The handshake process uses the following record schemas:
{
"type": "record",
"name": "HandshakeRequest", "namespace":"org.apache.avro.ipc",
"fields": [
{"name": "clientHash",
"type": {"type": "fixed", "name": "MD5", "size": 16}},
{"name": "clientProtocol", "type": ["null", "string"]},
{"name": "serverHash", "type": "MD5"},
{"name": "meta", "type": ["null", {"type": "map", "values": "bytes"}]}
]
}
{
"type": "record",
"name": "HandshakeResponse", "namespace": "org.apache.avro.ipc",
"fields": [
{"name": "match",
"type": {"type": "enum", "name": "HandshakeMatch",
"symbols": ["BOTH", "CLIENT", "NONE"]}},
{"name": "serverProtocol",
"type": ["null", "string"]},
{"name": "serverHash",
"type": ["null", {"type": "fixed", "name": "MD5", "size": 16}]},
{"name": "meta",
"type": ["null", {"type": "map", "values": "bytes"}]}
]
}
- A client first prefixes each request with a
HandshakeRequest containing just the hash of its protocol and of the server’s protocol (clientHash!=null, clientProtocol=null, serverHash!=null), where the hashes are 128-bit MD5 hashes of the JSON protocol text. If a client has never connected to a given server, it sends its hash as a guess of the server’s hash, otherwise it sends the hash that it previously obtained from this server.
The server responds with a HandshakeResponse containing one of:
-
match=BOTH, serverProtocol=null, serverHash=null if the client sent the valid hash of the server’s protocol and the server knows what protocol corresponds to the client’s hash. In this case, the request is complete and the response data immediately follows the HandshakeResponse.
-
match=CLIENT, serverProtocol!=null, serverHash!=null if the server has previously seen the client’s protocol, but the client sent an incorrect hash of the server’s protocol. The request is complete and the response data immediately follows the HandshakeResponse. The client must use the returned protocol to process the response and should also cache that protocol and its hash for future interactions with this server.
-
match=NONE if the server has not previously seen the client’s protocol. The serverHash and serverProtocol may also be non-null if the server’s protocol hash was incorrect.
In this case the client must then re-submit its request with its protocol text (clientHash!=null, clientProtocol!=null, serverHash!=null) and the server should respond with a successful match (match=BOTH, serverProtocol=null, serverHash=null) as above.
The meta field is reserved for future handshake enhancements.
A call consists of a request message paired with its resulting response or error message. Requests and responses contain extensible metadata, and both kinds of messages are framed as described above.
The format of a call request is:
- request metadata, a map with values of type bytes
- the message name, an Avro string, followed by
- the message parameters. Parameters are serialized according to the message’s request declaration.
When the empty string is used as a message name a server should ignore the parameters and return an empty response. A client may use this to ping a server or to perform a handshake without sending a protocol message.
When a message is declared one-way and a stateful connection has been established by a successful handshake response, no response data is sent. Otherwise the format of the call response is:
- response metadata, a map with values of type bytes
- a one-byte error flag boolean, followed by either:
- if the error flag is false, the message response, serialized per the message’s response schema.
- if the error flag is true, the error, serialized per the message’s effective error union schema.
Schema Resolution
A reader of Avro data, whether from an RPC or a file, can always parse that data because the original schema must be provided along with the data. However, the reader may be programmed to read data into a different schema. For example, if the data was written with a different version of the software than it is read, then fields may have been added or removed from records. This section specifies how such schema differences should be resolved.
We refer to the schema used to write the data as the writer’s schema, and the schema that the application expects the reader’s schema. Differences between these should be resolved as follows:
- It is an error if the two schemas do not match.
To match, one of the following must hold:
-
both schemas are arrays whose item types match
-
both schemas are maps whose value types match
-
both schemas are enums whose names match
-
both schemas are fixed whose sizes and names match
-
both schemas are records with the same name
-
either schema is a union
-
both schemas have same primitive type
-
the writer’s schema may be promoted to the reader’s as follows:
- int is promotable to long, float, or double
- long is promotable to float or double
- float is promotable to double
- string is promotable to bytes
- bytes is promotable to string
-
if both are records:
- the ordering of fields may be different: fields are matched by name.
- schemas for fields with the same name in both records are resolved recursively.
- if the writer’s record contains a field with a name not present in the reader’s record, the writer’s value for that field is ignored.
- if the reader’s record schema has a field that contains a default value, and writer’s schema does not have a field with the same name, then the reader should use the default value from its field.
- if the reader’s record schema has a field with no default value, and writer’s schema does not have a field with the same name, an error is signalled.
-
if both are enums:
if the writer’s symbol is not present in the reader’s enum and the reader has a default value, then that value is used, otherwise an error is signalled.
This resolution algorithm is applied recursively to the reader’s and writer’s array item schemas.
This resolution algorithm is applied recursively to the reader’s and writer’s value schemas.
The first schema in the reader’s union that matches the selected writer’s union schema is recursively resolved against it. if none match, an error is signalled.
- if reader’s is a union, but writer’s is not
The first schema in the reader’s union that matches the writer’s schema is recursively resolved against it. If none match, an error is signalled.
- if writer’s is a union, but reader’s is not
If the reader’s schema matches the selected writer’s schema, it is recursively resolved against it. If they do not match, an error is signalled.
A schema’s doc fields are ignored for the purposes of schema resolution. Hence, the doc portion of a schema may be dropped at serialization.
One of the defining characteristics of Avro is that a reader must use the schema used by the writer of the data in order to know how to read the data. This assumption results in a data format that’s compact and also amenable to many forms of schema evolution. However, the specification so far has not defined what it means for the reader to have the “same” schema as the writer. Does the schema need to be textually identical? Well, clearly adding or removing some whitespace to a JSON expression does not change its meaning. At the same time, reordering the fields of records clearly does change the meaning. So what does it mean for a reader to have “the same” schema as a writer?
Parsing Canonical Form is a transformation of a writer’s schema that let’s us define what it means for two schemas to be “the same” for the purpose of reading data written against the schema. It is called Parsing Canonical Form because the transformations strip away parts of the schema, like “doc” attributes, that are irrelevant to readers trying to parse incoming data. It is called Canonical Form because the transformations normalize the JSON text (such as the order of attributes) in a way that eliminates unimportant differences between schemas. If the Parsing Canonical Forms of two different schemas are textually equal, then those schemas are “the same” as far as any reader is concerned, i.e., there is no serialized data that would allow a reader to distinguish data generated by a writer using one of the original schemas from data generated by a writing using the other original schema. (We sketch a proof of this property in a companion document.)
The next subsection specifies the transformations that define Parsing Canonical Form. But with a well-defined canonical form, it can be convenient to go one step further, transforming these canonical forms into simple integers (“fingerprints”) that can be used to uniquely identify schemas. The subsection after next recommends some standard practices for generating such fingerprints.
Assuming an input schema (in JSON form) that’s already UTF-8 text for a valid Avro schema (including all quotes as required by JSON), the following transformations will produce its Parsing Canonical Form:
- [PRIMITIVES] Convert primitive schemas to their simple form (e.g., int instead of
{"type":"int"}).
- [FULLNAMES] Replace short names with fullnames, using applicable namespaces to do so. Then eliminate namespace attributes, which are now redundant.
- [STRIP] Keep only attributes that are relevant to parsing data, which are: type, name, fields, symbols, items, values, size. Strip all others (e.g., doc and aliases).
- [ORDER] Order the appearance of fields of JSON objects as follows: name, type, fields, symbols, items, values, size. For example, if an object has type, name, and size fields, then the name field should appear first, followed by the type and then the size fields.
- [STRINGS] For all JSON string literals in the schema text, replace any escaped characters (e.g., \uXXXX escapes) with their UTF-8 equivalents.
- [INTEGERS] Eliminate quotes around and any leading zeros in front of JSON integer literals (which appear in the size attributes of fixed schemas).
- [WHITESPACE] Eliminate all whitespace in JSON outside of string literals.
Schema Fingerprints
“[A] fingerprinting algorithm is a procedure that maps an arbitrarily large data item (such as a computer file) to a much shorter bit string, its fingerprint, that uniquely identifies the original data for all practical purposes” (quoted from Wikipedia). In the Avro context, fingerprints of Parsing Canonical Form can be useful in a number of applications; for example, to cache encoder and decoder objects, to tag data items with a short substitute for the writer’s full schema, and to quickly negotiate common-case schemas between readers and writers.
In designing fingerprinting algorithms, there is a fundamental trade-off between the length of the fingerprint and the probability of collisions. To help application designers find appropriate points within this trade-off space, while encouraging interoperability and ease of implementation, we recommend using one of the following three algorithms when fingerprinting Avro schemas:
- When applications can tolerate longer fingerprints, we recommend using the SHA-256 digest algorithm to generate 256-bit fingerprints of Parsing Canonical Forms. Most languages today have SHA-256 implementations in their libraries.
- At the opposite extreme, the smallest fingerprint we recommend is a 64-bit Rabin fingerprint. Below, we provide pseudo-code for this algorithm that can be easily translated into any programming language. 64-bit fingerprints should guarantee uniqueness for schema caches of up to a million entries (for such a cache, the chance of a collision is 3E-8). We don’t recommend shorter fingerprints, as the chances of collisions is too great (for example, with 32-bit fingerprints, a cache with as few as 100,000 schemas has a 50% chance of having a collision).
- Between these two extremes, we recommend using the MD5 message digest to generate 128-bit fingerprints. These make sense only where very large numbers of schemas are being manipulated (tens of millions); otherwise, 64-bit fingerprints should be sufficient. As with SHA-256, MD5 implementations are found in most libraries today.
These fingerprints are not meant to provide any security guarantees, even the longer SHA-256-based ones. Most Avro applications should be surrounded by security measures that prevent attackers from writing random data and otherwise interfering with the consumers of schemas. We recommend that these surrounding mechanisms be used to prevent collision and pre-image attacks (i.e., “forgery”) on schema fingerprints, rather than relying on the security properties of the fingerprints themselves.
Rabin fingerprints are cyclic redundancy checks computed using irreducible polynomials. In the style of the Appendix of RFC 1952 (pg 10), which defines the CRC-32 algorithm, here’s our definition of the 64-bit AVRO fingerprinting algorithm:
long fingerprint64(byte[] buf) {
if (FP_TABLE == null) initFPTable();
long fp = EMPTY;
for (int i = 0; i < buf.length; i++)
fp = (fp >>> 8) ^ FP_TABLE[(int)(fp ^ buf[i]) & 0xff];
return fp;
}
static long EMPTY = 0xc15d213aa4d7a795L;
static long[] FP_TABLE = null;
void initFPTable() {
FP_TABLE = new long[256];
for (int i = 0; i < 256; i++) {
long fp = i;
for (int j = 0; j < 8; j++)
fp = (fp >>> 1) ^ (EMPTY & -(fp & 1L));
FP_TABLE[i] = fp;
}
}
Readers interested in the mathematics behind this algorithm may want to read this book chapter. (Unlike RFC-1952 and the book chapter, we prepend a single one bit to messages. We do this because CRCs ignore leading zero bits, which can be problematic. Our code prepends a one-bit by initializing fingerprints using EMPTY, rather than initializing using zero as in RFC-1952 and the book chapter.)
Logical Types
A logical type is an Avro primitive or complex type with extra attributes to represent a derived type. The attribute logicalType must always be present for a logical type, and is a string with the name of one of the logical types listed later in this section. Other attributes may be defined for particular logical types.
A logical type is always serialized using its underlying Avro type so that values are encoded in exactly the same way as the equivalent Avro type that does not have a logicalType attribute. Language implementations may choose to represent logical types with an appropriate native type, although this is not required.
Language implementations must ignore unknown logical types when reading, and should use the underlying Avro type. If a logical type is invalid, for example a decimal with scale greater than its precision, then implementations should ignore the logical type and use the underlying Avro type.
Decimal
The decimal logical type represents an arbitrary-precision signed decimal number of the form unscaled × 10-scale.
A decimal logical type annotates Avro bytes or fixed types. The byte array must contain the two’s-complement representation of the unscaled integer value in big-endian byte order. The scale is fixed, and is specified using an attribute.
The following attributes are supported:
- scale, a JSON integer representing the scale (optional). If not specified the scale is 0.
- precision, a JSON integer representing the (maximum) precision of decimals stored in this type (required).
For example, the following schema represents decimal numbers with a maximum precision of 4 and a scale of 2:
{
"type": "bytes",
"logicalType": "decimal",
"precision": 4,
"scale": 2
}
Precision must be a positive integer greater than zero. If the underlying type is a fixed, then the precision is limited by its size. An array of length n can store at most floor(log10(28 × n - 1 - 1)) base-10 digits of precision.
Scale must be zero or a positive integer less than or equal to the precision.
For the purposes of schema resolution, two schemas that are decimal logical types match if their scales and precisions match.
Date
The date logical type represents a date within the calendar, with no reference to a particular time zone or time of day.
A date logical type annotates an Avro int, where the int stores the number of days from the unix epoch, 1 January 1970 (ISO calendar).
Time (millisecond precision)
The time-millis logical type represents a time of day, with no reference to a particular calendar, time zone or date, with a precision of one millisecond.
A time-millis logical type annotates an Avro int, where the int stores the number of milliseconds after midnight, 00:00:00.000.
Time (microsecond precision)
The time-micros logical type represents a time of day, with no reference to a particular calendar, time zone or date, with a precision of one microsecond.
A time-micros logical type annotates an Avro long, where the long stores the number of microseconds after midnight, 00:00:00.000000.
Timestamp (millisecond precision)
The timestamp-millis logical type represents an instant on the global timeline, independent of a particular time zone or calendar, with a precision of one millisecond.
A timestamp-millis logical type annotates an Avro long, where the long stores the number of milliseconds from the unix epoch, 1 January 1970 00:00:00.000 UTC.
Timestamp (microsecond precision)
The timestamp-micros logical type represents an instant on the global timeline, independent of a particular time zone or calendar, with a precision of one microsecond.
A timestamp-micros logical type annotates an Avro long, where the long stores the number of microseconds from the unix epoch, 1 January 1970 00:00:00.000000 UTC.
Duration
The duration logical type represents an amount of time defined by a number of months, days and milliseconds. This is not equivalent to a number of milliseconds, because, depending on the moment in time from which the duration is measured, the number of days in the month and number of milliseconds in a day may differ. Other standard periods such as years, quarters, hours and minutes can be expressed through these basic periods.
A duration logical type annotates Avro fixed type of size 12, which stores three little-endian unsigned integers that represent durations at different granularities of time. The first stores a number in months, the second stores a number in days, and the third stores a number in milliseconds.
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
10.4 - Trevni
Version 0.1
DRAFT
This document is the authoritative specification of a file format. Its intent is to permit compatible, independent implementations that read and/or write files in this format.
Introduction
Data sets are often described as a table composed of rows and columns. Each record in the dataset is considered a row, with each field of the record occupying a different column. Writing records to a file one-by-one as they are created results in a row-major format, like Hadoop’s SequenceFile or Avro data files.
In many cases higher query performance may be achieved if the data is instead organized in a column-major format, where multiple values of a given column are stored adjacently. This document defines such a column-major file format for datasets.
To permit scalable, distributed query evaluation, datasets are partitioned into row groups, containing distinct collections of rows. Each row group is organized in column-major order, while row groups form a row-major partitioning of the entire dataset.
Rationale
Goals
The format is meant satisfy the following goals:
Maximize the size of row groups. Disc drives are used most efficiently when sequentially accessing data. Consider a drive that takes 10ms to seek and transfers at 100MB/second. If a 10-column dataset whose values are all the same size is split into 10MB row groups, then accessing a single column will require a sequence of seek+1MB reads, for a cost of 20ms/MB processed. If the same dataset is split into 100MB row groups then this drops to 11ms/MB processed. This effect is exaggerated for datasets with larger numbers of columns and with columns whose values are smaller than average. So we’d prefer row groups that are 100MB or greater.
Permit random access within a row group. Some queries will first examine one column, and, only when certain relatively rare criteria are met, examine other columns. Rather than iterating through selected columns of the row-group in parallel, one might iterate through one column and randomly access another. This is called support for WHERE clauses, after the SQL operator of that name.
Minimize the number of files per dataset. HDFS is a primary intended deployment platform for these files. The HDFS Namenode requires memory for each file in the filesystem, thus for a format to be HDFS-friendly it should strive to require the minimum number of distinct files.
Support co-location of columns within row-groups. Row groups are the unit of parallel operation on a column dataset. For efficient file i/o, the entirety of a row-group should ideally reside on the host that is evaluating the query in order to avoid network latencies and bottlenecks.
Data integrity. The format should permit applications to detect data corruption. Many file systems may prevent corruption, but files may be moved between filesystems and be subject to corruption at points in that process. It is best if the data in a file can be validated independently.
Extensibility. The format should permit applications to store additional annotations about a datasets in the files, such as type information, origin, etc. Some environments may have metadata stores for such information, but not all do, and files might be moved among systems with different metadata systems. The ability to keep such information within the file simplifies the coordination of such information.
Minimal overhead. The column format should not make datasets appreciably larger. Storage is a primary cost and a choice to use this format should not require additional storage.
Primary format. The column format should be usable as a primary format for datasets, not as an auxiliary, accelerated format. Applications that process a dataset in row-major order should be able to easily consume column files and applications that produce datasets in row-major order should be able to easily generate column files.
Design
To meet these goals we propose the following design.
Each row group is a separate file. All values of a column in a file are written contiguously. This maximizes the row group size, optimizing performance when querying few and small columns.
Each file occupies a single HDFS block. A larger than normal block size may be specified, e.g., ~1GB instead of the typical ~100MB. This guarantees co-location and eliminates network use when query processing can be co-located with the file. This also moderates the memory impact on the HDFS Namenode since no small files are written.
Each column in a file is written as a sequence of ~64kB compressed blocks. The sequence is prefixed by a table describing all of the blocks in the column to permit random access within the column.
Application-specific metadata may be added at the file, column, and block levels.
Checksums are included with each block, providing data integrity.
Discussion
The use of a single block per file achieves the same effect as the custom block placement policy described in the CIF paper, but while still permitting HDFS rebalancing and not increasing the number of files in the namespace.
This section formally describes the proposed column file format.
Data Model
We assume a simple data model, where a record is a set of named fields, and the value of each field is a sequence of untyped bytes. A type system may be layered on top of this, as specified in the Type Mapping section below.
Primitive Values
We define the following primitive value types:
- Signed 64-bit long values are written using a variable-length zig-zag coding, where the high-order bit in each byte determines whether subsequent bytes are present. For example:
decimal value hex bytes
| decimal value |
hex bytes |
|
| 0 |
00 |
|
| -1 |
01 |
|
| 1 |
02 |
|
| … |
|
|
| -64 |
7f |
|
| 64 |
80 01 |
|
| … |
|
|
- bytes are encoded as a long followed by that many bytes of data.
- a string is encoded as a long followed by that many bytes of UTF-8 encoded character data.
For example, the three-character string “foo” would be encoded as the long value 3 (encoded as hex 06) followed by the UTF-8 encoding of ‘f’, ‘o’, and ‘o’ (the hex bytes 66 6f 6f): 06 66 6f 6f
Type Names
The following type names are used to describe column values:
- null, requires zero bytes. Sometimes used in array columns.
- boolean, one bit, packed into bytes, little-endian;
- int, like long, but restricted to 32-bit signed values
- long 64-bit signed values, represented as above
- fixed32 32-bit values stored as four bytes, little-endian.
- fixed64 64-bit values stored as eight bytes, little-endian.
- float 32-bit IEEE floating point value, little-endian
- double 64-bit IEEE floating point value, little-endian
- string as above
- bytes as above, may be used to encapsulate more complex objects
Type names are represented as strings (UTF-8 encoded, length-prefixed).
Metadata consists of:
- A long indicating the number of metadata key/value pairs.
- For each pair, a string key and bytes value.
All metadata properties that start with “trevni.” are reserved.
The following file metadata properties are defined:
- trevni.codec the name of the default compression codec used to compress blocks, as a string. Implementations are required to support the “null” codec. Optional. If absent, it is assumed to be “null”. Codecs are described in more detail below.
- trevni.checksum the name of the checksum algorithm used in this file, as a string. Implementations are required to support the “crc-32” checksum. Optional. If absent, it is assumed to be “null”. Checksums are described in more detail below.
The following column metadata properties are defined:
- trevni.codec the name of the compression codec used to compress the blocks of this column, as a string. Implementations are required to support the “null” codec. Optional. If absent, it is assumed to be “null”. Codecs are described in more detail below.
- trevni.name the name of the column, as a string. Required.
- trevni.type the type of data in the column. One of the type names above. Required.
- trevni.values if present, indicates that the initial value of each block in this column will be stored in the block’s descriptor. Not permitted for array columns or columns that specify a parent.
- trevni.array if present, indicates that each row in this column contains a sequence of values of the named type rather than just a single value. An integer length precedes each sequence of values indicating the count of values in the sequence. If the length is negative then it indicates a sequence of zero or one lengths, where -1 indicates two zeros, -2 two ones, -3 three zeros, -4 three ones, etc.
- trevni.parent if present, the name of an array column whose lengths are also used by this column. Thus values of this column are sequences but no lengths are stored in this column.
For example, consider the following row, as JSON, where all values are primitive types, but one has multiple values.
{"id"=566, "date"=23423234234
"from"="foo@bar.com",
"to"=["bar@baz.com", "bang@foo.com"],
"content"="Hi!"}
The columns for this might be specified as:
name=id type=int
name=date type=long
name=from type=string
name=to type=string array=true
name=content type=string
If a row contains an array of records, e.g. “received” in the following:
{"id"=566, "date"=23423234234
"from"="foo@bar.com",
"to"=["bar@baz.com", "bang@foo.com"],
"content"="Hi!"
"received"=[{"date"=234234234234, "host"="192.168.0.0.1"},
{"date"=234234545645, "host"="192.168.0.0.2"}]
}
Then one can define a parent column followed by a column for each field in the record, adding the following columns:
name=received type=null array=true
name=date type=long parent=received
name=host type=string parent=received
If an array value itself contains an array, e.g. the “sigs” below:
{"id"=566, "date"=23423234234
"from"="foo@bar.com",
"to"=["bar@baz.com", "bang@foo.com"],
"content"="Hi!"
"received"=[{"date"=234234234234, "host"="192.168.0.0.1",
"sigs"=[{"algo"="weak", "value"="0af345de"}]},
{"date"=234234545645, "host"="192.168.0.0.2",
"sigs"=[]}]
}
Then a parent column may be defined that itself has a parent column.
name=sigs type=null array=true parent=received
name=algo type=string parent=sigs
name=value type=string parent=sigs
No block metadata properties are currently defined.
A file consists of:
- A file header, followed by
- one or more columns.
A file header consists of:
- Four bytes, ASCII ‘T’, ‘r’, ‘v’, followed by 0x02.
- a fixed64 indicating the number of rows in the file
- a fixed32 indicating the number of columns in the file
- file metadata.
- for each column, its column metadata
- for each column, its starting position in the file as a fixed64.
A column consists of:
- A fixed32 indicating the number of blocks in this column.
- For each block, a block descriptor
- One or more blocks.
A block descriptor consists of:
- A fixed32 indicating the number of rows in the block
- A fixed32 indicating the size in bytes of the block before the codec is applied (excluding checksum).
- A fixed32 indicating the size in bytes of the block after the codec is applied (excluding checksum).
- If this column’s metadata declares it to include values, the first value in the column, serialized according to this column’s type.
A block consists of:
- The serialized column values. If a column is an array column then value sequences are preceded by their length, as an int. If a codec is specified, the values and lengths are compressed by that codec.
- The checksum, as determined by the file metadata.
Codecs
null
The “null” codec simply passes data through uncompressed.
deflate
The “deflate” codec writes the data block using the deflate algorithm as specified in RFC 1951.
snappy
The “snappy” codec uses Google’s Snappy compression library.
Checksum algorithms
null
The “null” checksum contains zero bytes.
crc-32
Each “crc-32” checksum contains the four bytes of an ISO 3309 CRC-32 checksum of the uncompressed block data as a fixed32.
Type Mappings
We define a standard mapping for how types defined in various serialization systems are represented in a column file. Records from these systems are shredded into columns. When records are nested, a depth-first recursive walk can assign a separate column for each primitive value.
Avro
Protocol Buffers
Thrift
Implementation Notes
Some possible techniques for writing column files include:
- Use a standard ~100MB block, buffer in memory up to the block size, then flush the file directly to HDFS. A single reduce task might create multiple output files. The namenode requires memory proportional to the number of names and blocks*replication. This would increase the number of names but not blocks, so this should still be much better than a file per column.
- Spill each column to a separate local, temporary file then, when the file is closed, append these files, writing a single file to HDFS whose block size is set to be that of the entire file. This would be a bit slower than and may have trouble when the local disk is full, but it would better use HDFS namespace and further reduce seeks when processing columns whose values are small.
- Use a separate mapreduce job to convert row-major files to column-major. The map output would output a by (row#, column#, value) tuple, partitioned by row# but sorted by column# then row#. The reducer could directly write the column file. But the column file format would need to be changed to write counts, descriptors, etc. at the end of files rather than at the front.
(1) is the simplest to implement and most implementations should start with it.
References
CIF Column-Oriented Storage Techniques for MapReduce, Floratou, Patel, Shekita, & Tata, VLDB 2011.
DREMEL Dremel: Interactive Analysis of Web-Scale Datasets, Melnik, Gubarev, Long, Romer, Shivakumar, & Tolton, VLDB 2010.
10.5 - MapReduce guide
Avro provides a convenient way to represent complex data structures within a Hadoop MapReduce job. Avro data can be used as both input to and output from a MapReduce job, as well as the intermediate format. The example in this guide uses Avro data for all three, but it’s possible to mix and match; for instance, MapReduce can be used to aggregate a particular field in an Avro record.
This guide assumes basic familiarity with both Hadoop MapReduce and Avro. See the Hadoop documentation and the Avro getting started guide for introductions to these projects. This guide uses the old MapReduce API (org.apache.hadoop.mapred) and the new MapReduce API (org.apache.hadoop.mapreduce).
Setup
The code from this guide is included in the Avro docs under examples/mr-example. The example is set up as a Maven project that includes the necessary Avro and MapReduce dependencies and the Avro Maven plugin for code generation, so no external jars are needed to run the example. In particular, the POM includes the following dependencies:
<dependency>
<groupId>org.apache.avro</groupId>
<artifactId>avro</artifactId>
<version>1.8.1</version>
</dependency>
<dependency>
<groupId>org.apache.avro</groupId>
<artifactId>avro-mapred</artifactId>
<version>1.8.1</version>
</dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-core</artifactId>
<version>1.1.0</version>
</dependency>
And the following plugin:
<plugin>
<groupId>org.apache.avro</groupId>
<artifactId>avro-maven-plugin</artifactId>
<version>1.8.1</version>
<executions>
<execution>
<phase>generate-sources</phase>
<goals>
<goal>schema</goal>
</goals>
<configuration>
<sourceDirectory>${project.basedir}/../</sourceDirectory>
<outputDirectory>${project.basedir}/target/generated-sources/</outputDirectory>
</configuration>
</execution>
</executions>
</plugin>
If you do not configure the sourceDirectory and outputDirectory properties, the defaults will be used. The sourceDirectory property defaults to src/main/avro. The outputDirectory property defaults to target/generated-sources. You can change the paths to match your project layout.
Alternatively, Avro jars can be downloaded directly from the Apache Avro™ Releases page. The relevant Avro jars for this guide are avro-1.8.1.jar and avro-mapred-1.8.1.jar, as well as avro-tools-1.8.1.jar for code generation and viewing Avro data files as JSON. In addition, you will need to install Hadoop in order to use MapReduce.
Example: ColorCount
Below is a simple example of a MapReduce that uses Avro. There is an example for both the old (org.apache.hadoop.mapred) and new (org.apache.hadoop.mapreduce) APIs under examples/mr-example/src/main/java/example/. MapredColorCount is the example for the older mapred API while MapReduceColorCount is the example for the newer mapreduce API. Both examples are below, but we will detail the mapred API in our subsequent examples.
MapredColorCount:
package example;
import java.io.IOException;
import org.apache.avro.*;
import org.apache.avro.Schema.Type;
import org.apache.avro.mapred.*;
import org.apache.hadoop.conf.*;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.mapred.*;
import org.apache.hadoop.util.*;
import example.avro.User;
public class MapredColorCount extends Configured implements Tool {
public static class ColorCountMapper extends AvroMapper<User, Pair<CharSequence, Integer>> {
@Override
public void map(User user, AvroCollector<Pair<CharSequence, Integer>> collector, Reporter reporter)
throws IOException {
CharSequence color = user.getFavoriteColor();
// We need this check because the User.favorite_color field has type ["string", "null"]
if (color == null) {
color = "none";
}
collector.collect(new Pair<CharSequence, Integer>(color, 1));
}
}
public static class ColorCountReducer extends AvroReducer<CharSequence, Integer,
Pair<CharSequence, Integer>> {
@Override
public void reduce(CharSequence key, Iterable<Integer> values,
AvroCollector<Pair<CharSequence, Integer>> collector,
Reporter reporter)
throws IOException {
int sum = 0;
for (Integer value : values) {
sum += value;
}
collector.collect(new Pair<CharSequence, Integer>(key, sum));
}
}
public int run(String[] args) throws Exception {
if (args.length != 2) {
System.err.println("Usage: MapredColorCount <input path> <output path>");
return -1;
}
JobConf conf = new JobConf(getConf(), MapredColorCount.class);
conf.setJobName("colorcount");
FileInputFormat.setInputPaths(conf, new Path(args[0]));
FileOutputFormat.setOutputPath(conf, new Path(args[1]));
AvroJob.setMapperClass(conf, ColorCountMapper.class);
AvroJob.setReducerClass(conf, ColorCountReducer.class);
// Note that AvroJob.setInputSchema and AvroJob.setOutputSchema set
// relevant config options such as input/output format, map output
// classes, and output key class.
AvroJob.setInputSchema(conf, User.getClassSchema());
AvroJob.setOutputSchema(conf, Pair.getPairSchema(Schema.create(Type.STRING),
Schema.create(Type.INT)));
JobClient.runJob(conf);
return 0;
}
public static void main(String[] args) throws Exception {
int res = ToolRunner.run(new Configuration(), new MapredColorCount(), args);
System.exit(res);
}
}
MapReduceColorCount:
package example;
import java.io.IOException;
import org.apache.avro.Schema;
import org.apache.avro.mapred.AvroKey;
import org.apache.avro.mapred.AvroValue;
import org.apache.avro.mapreduce.AvroJob;
import org.apache.avro.mapreduce.AvroKeyInputFormat;
import org.apache.avro.mapreduce.AvroKeyValueOutputFormat;
import org.apache.hadoop.conf.Configured;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.IntWritable;
import org.apache.hadoop.io.NullWritable;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.mapreduce.Job;
import org.apache.hadoop.mapreduce.Mapper;
import org.apache.hadoop.mapreduce.Reducer;
import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
import org.apache.hadoop.util.Tool;
import org.apache.hadoop.util.ToolRunner;
import example.avro.User;
public class MapReduceColorCount extends Configured implements Tool {
public static class ColorCountMapper extends
Mapper<AvroKey<User>, NullWritable, Text, IntWritable> {
@Override
public void map(AvroKey<User> key, NullWritable value, Context context)
throws IOException, InterruptedException {
CharSequence color = key.datum().getFavoriteColor();
if (color == null) {
color = "none";
}
context.write(new Text(color.toString()), new IntWritable(1));
}
}
public static class ColorCountReducer extends
Reducer<Text, IntWritable, AvroKey<CharSequence>, AvroValue<Integer>> {
@Override
public void reduce(Text key, Iterable<IntWritable> values,
Context context) throws IOException, InterruptedException {
int sum = 0;
for (IntWritable value : values) {
sum += value.get();
}
context.write(new AvroKey<CharSequence>(key.toString()), new AvroValue<Integer>(sum));
}
}
public int run(String[] args) throws Exception {
if (args.length != 2) {
System.err.println("Usage: MapReduceColorCount <input path> <output path>");
return -1;
}
Job job = new Job(getConf());
job.setJarByClass(MapReduceColorCount.class);
job.setJobName("Color Count");
FileInputFormat.setInputPaths(job, new Path(args[0]));
FileOutputFormat.setOutputPath(job, new Path(args[1]));
job.setInputFormatClass(AvroKeyInputFormat.class);
job.setMapperClass(ColorCountMapper.class);
AvroJob.setInputKeySchema(job, User.getClassSchema());
job.setMapOutputKeyClass(Text.class);
job.setMapOutputValueClass(IntWritable.class);
job.setOutputFormatClass(AvroKeyValueOutputFormat.class);
job.setReducerClass(ColorCountReducer.class);
AvroJob.setOutputKeySchema(job, Schema.create(Schema.Type.STRING));
AvroJob.setOutputValueSchema(job, Schema.create(Schema.Type.INT));
return (job.waitForCompletion(true) ? 0 : 1);
}
public static void main(String[] args) throws Exception {
int res = ToolRunner.run(new MapReduceColorCount(), args);
System.exit(res);
}
}
ColorCount reads in data files containing User records, defined in examples/user.avsc, and counts the number of instances of each favorite color. (This example draws inspiration from the canonical WordCount MapReduce application.) This example uses the old MapReduce API. See MapReduceAvroWordCount, found under doc/examples/mr-example/src/main/java/example/ to see the new MapReduce API example. The User schema is defined as follows:
{"namespace": "example.avro",
"type": "record",
"name": "User",
"fields": [
{"name": "name", "type": "string"},
{"name": "favorite_number", "type": ["int", "null"]},
{"name": "favorite_color", "type": ["string", "null"]}
]
}
This schema is compiled into the User class used by ColorCount via the Avro Maven plugin (see examples/mr-example/pom.xml for how this is set up).
ColorCountMapper essentially takes a User as input and extracts the User’s favorite color, emitting the key-value pair <favoriteColor, 1>. ColorCountReducer then adds up how many occurrences of a particular favorite color were emitted, and outputs the result as a Pair record. These Pairs are serialized to an Avro data file.
Running ColorCount
The ColorCount application is provided as a Maven project in the Avro docs under examples/mr-example. To build the project, including the code generation of the User schema, run:
Next, run GenerateData from examples/mr-examples to create an Avro data file, input/users.avro, containing 20 Users with favorite colors chosen randomly from a list:
mvn exec:java -q -Dexec.mainClass=example.GenerateData
Besides creating the data file, GenerateData prints the JSON representations of the Users generated to stdout, for example:
{"name": "user", "favorite_number": null, "favorite_color": "red"}
{"name": "user", "favorite_number": null, "favorite_color": "green"}
{"name": "user", "favorite_number": null, "favorite_color": "purple"}
{"name": "user", "favorite_number": null, "favorite_color": null}
...
Now we’re ready to run ColorCount. We specify our freshly-generated input folder as the input path and output as our output folder (note that MapReduce will not start a job if the output folder already exists):
mvn exec:java -q -Dexec.mainClass=example.MapredColorCount -Dexec.args="input output"
Once ColorCount completes, checking the contents of the new output directory should yield the following:
$ ls output/
part-00000.avro _SUCCESS
You can check the contents of the generated Avro file using the avro-tools jar:
$ java -jar /path/to/avro-tools-1.8.1.jar tojson output/part-00000.avro
{"value": 3, "key": "blue"}
{"value": 7, "key": "green"}
{"value": 1, "key": "none"}
{"value": 2, "key": "orange"}
{"value": 3, "key": "purple"}
{"value": 2, "key": "red"}
{"value": 2, "key": "yellow"}
Now let’s go over the ColorCount example in detail.
Mapper - org.apache.hadoop.mapred API
The easiest way to use Avro data files as input to a MapReduce job is to subclass AvroMapper. An AvroMapper defines a map function that takes an Avro datum as input and outputs a key/value pair represented as a Pair record. In the ColorCount example, ColorCountMapper is an AvroMapper that takes a User as input and outputs a Pair<CharSequence, Integer>>, where the CharSequence key is the user’s favorite color and the Integer value is 1.
public static class ColorCountMapper extends AvroMapper<User, Pair<CharSequence, Integer>> {
@Override
public void map(User user, AvroCollector<Pair<CharSequence, Integer>> collector, Reporter reporter)
throws IOException {
CharSequence color = user.getFavoriteColor();
// We need this check because the User.favorite_color field has type ["string", "null"]
if (color == null) {
color = "none";
}
collector.collect(new Pair<CharSequence, Integer>(color, 1));
}
}
In order to use our AvroMapper, we must call AvroJob.setMapperClass and AvroJob.setInputSchema.
AvroJob.setMapperClass(conf, ColorCountMapper.class);
AvroJob.setInputSchema(conf, User.getClassSchema());
Note that AvroMapper does not implement the Mapper interface. Under the hood, the specified Avro data files are deserialized into AvroWrappers containing the actual data, which are processed by a Mapper that calls the configured AvroMapper’s map function. AvroJob.setInputSchema sets up the relevant configuration parameters needed to make this happen, thus you should not need to call JobConf.setMapperClass, JobConf.setInputFormat, JobConf.setMapOutputKeyClass, JobConf.setMapOutputValueClass, or JobConf.setOutputKeyComparatorClass.
Mapper - org.apache.hadoop.mapreduce API
This document will not go into all the differences between the mapred and mapreduce APIs, however will describe the main differences. As you can see, ColorCountMapper is now a subclass of the Hadoop Mapper class and is passed an AvroKey as it’s key. Additionally, the AvroJob method calls were slightly changed.
public static class ColorCountMapper extends
Mapper<AvroKey<User>, NullWritable, Text, IntWritable> {
@Override
public void map(AvroKey<User> key, NullWritable value, Context context)
throws IOException, InterruptedException {
CharSequence color = key.datum().getFavoriteColor();
if (color == null) {
color = "none";
}
context.write(new Text(color.toString()), new IntWritable(1));
}
}
Reducer - org.apache.hadoop.mapred API
Analogously to AvroMapper, an AvroReducer defines a reducer function that takes the key/value types output by an AvroMapper (or any mapper that outputs Pairs) and outputs a key/value pair represented a Pair record. In the ColorCount example, ColorCountReducer is an AvroReducer that takes the CharSequence key representing a favorite color and the Iterable<Integer> representing the counts for that color (they should all be 1 in this example) and adds up the counts.
public static class ColorCountReducer extends AvroReducer<CharSequence, Integer,
Pair<CharSequence, Integer>> {
@Override
public void reduce(CharSequence key, Iterable<Integer> values,
AvroCollector<Pair<CharSequence, Integer>> collector,
Reporter reporter)
throws IOException {
int sum = 0;
for (Integer value : values) {
sum += value;
}
collector.collect(new Pair<CharSequence, Integer>(key, sum));
}
}
In order to use our AvroReducer, we must call AvroJob.setReducerClass and AvroJob.setOutputSchema.
AvroJob.setReducerClass(conf, ColorCountReducer.class);
AvroJob.setOutputSchema(conf, Pair.getPairSchema(Schema.create(Type.STRING),
Schema.create(Type.INT)));
Note that AvroReducer does not implement the Reducer interface. The intermediate Pairs output by the mapper are split into AvroKeys and AvroValues, which are processed by a Reducer that calls the configured AvroReducer’s reduce function. AvroJob.setOutputSchema sets up the relevant configuration parameters needed to make this happen, thus you should not need to call JobConf.setReducerClass, JobConf.setOutputFormat, JobConf.setOutputKeyClass, JobConf.setMapOutputKeyClass, JobConf.setMapOutputValueClass, or JobConf.setOutputKeyComparatorClass.
Reduce - org.apache.hadoop.mapreduce API
As before we not detail every difference between the APIs. As with the Mapper change ColorCountReducer is now a subclass of Reducer and AvroKey and AvroValue are emitted. Additionally, the AvroJob method calls were slightly changed.
public static class ColorCountReducer extends
Reducer<Text, IntWritable, AvroKey<CharSequence>, AvroValue<Integer>> {
@Override
public void reduce(Text key, Iterable<IntWritable> values,
Context context) throws IOException, InterruptedException {
int sum = 0;
for (IntWritable value : values) {
sum += value.get();
}
context.write(new AvroKey<CharSequence>(key.toString()), new AvroValue<Integer>(sum));
}
}
Learning more
The mapred API allows users to mix Avro AvroMappers and AvroReducers with non-Avro Mappers and Reducers and the mapreduce API allows users input Avro and output non-Avro or vice versa.
The mapred package has API org.apache.avro.mapred documentation as does the org.apache.avro.mapreduce package. MapReduce API (org.apache.hadoop.mapreduce). Similarily to the mapreduce package, it’s possible with the mapred API to implement your own Mappers and Reducers directly using the public classes provided in these libraries. See the AvroWordCount application, found under examples/mr-example/src/main/java/example/AvroWordCount.java in the Avro documentation, for an example of implementing a Reducer that outputs Avro data using the old MapReduce API. See the MapReduceAvroWordCount application, found under examples/mr-example/src/main/java/example/MapReduceAvroWordCount.java in the Avro documentation, for an example of implementing a Reducer that outputs Avro data using the new MapReduce API.
10.6 - IDL Language
Introduction
This document defines Avro IDL, a higher-level language for authoring Avro schemata. Before reading this document, you should have familiarity with the concepts of schemata and protocols, as well as the various primitive and complex types available in Avro.
Overview
Purpose
The aim of the Avro IDL language is to enable developers to author schemata in a way that feels more similar to common programming languages like Java, C++, or Python. Additionally, the Avro IDL language may feel more familiar for those users who have previously used the interface description languages (IDLs) in other frameworks like Thrift, Protocol Buffers, or CORBA.
Usage
Each Avro IDL file defines a single Avro Protocol, and thus generates as its output a JSON-format Avro Protocol file with extension .avpr.
To convert a .avdl file into a .avpr file, it may be processed by the idl tool. For example:
$ java -jar avroj-tools.jar idl src/test/idl/input/namespaces.avdl /tmp/namespaces.avpr
$ head /tmp/namespaces.avpr
{
"protocol" : "TestNamespace",
"namespace" : "avro.test.protocol",
The idl tool can also process input to and from stdin and stdout. See idl --help for full usage information.
A Maven plugin is also provided to compile .avdl files. To use it, add something like the following to your pom.xml:
<build>
<plugins>
<plugin>
<groupId>org.apache.avro</groupId>
<artifactId>avro-maven-plugin</artifactId>
<executions>
<execution>
<goals>
<goal>idl-protocol</goal>
</goals>
</execution>
</executions>
</plugin>
</plugins>
</build>
Defining a Protocol in Avro IDL
An Avro IDL file consists of exactly one protocol definition. The minimal protocol is defined by the following code:
This is equivalent to (and generates) the following JSON protocol definition:
{
"protocol" : "MyProtocol",
"types" : [ ],
"messages" : {
}
}
The namespace of the protocol may be changed using the @namespace annotation:
@namespace("mynamespace")
protocol MyProtocol {
}
This notation is used throughout Avro IDL as a way of specifying properties for the annotated element, as will be described later in this document.
Protocols in Avro IDL can contain the following items:
- Imports of external protocol and schema files.
- Definitions of named schemata, including records, errors, enums, and fixeds.
- Definitions of RPC messages
Imports
Files may be imported in one of three formats:
-
An IDL file may be imported with a statement like:
import idl "foo.avdl";
-
A JSON protocol file may be imported with a statement like:
import protocol "foo.avpr";
-
A JSON schema file may be imported with a statement like:
import schema "foo.avsc";
Messages and types in the imported file are added to this file’s protocol.
Imported file names are resolved relative to the current IDL file.
Defining an Enumeration
Enums are defined in Avro IDL using a syntax similar to C or Java.
enum Suit {
SPADES, DIAMONDS, CLUBS, HEARTS
}
Note that, unlike the JSON format, anonymous enums cannot be defined.
Defining a Fixed Length Field
Fixed fields are defined using the following syntax:
fixed MD5(16);
This example defines a fixed-length type called MD5 which contains 16 bytes.
Defining Records and Errors
Records are defined in Avro IDL using a syntax similar to a struct definition in C:
record Employee {
string name;
boolean active = true;
long salary;
}
The above example defines a record with the name “Employee” with three fields.
To define an error, simply use the keyword error instead of record. For example:
error Kaboom {
string explanation;
int result_code = -1;
}
Each field in a record or error consists of a type and a name, optional property annotations and an optional default value.
A type reference in Avro IDL must be one of:
- A primitive type
- A logical type
- A named schema defined prior to this usage in the same Protocol
- A complex type (array, map, or union)
Primitive Types
The primitive types supported by Avro IDL are the same as those supported by Avro’s JSON format. This list includes int, long, string, boolean, float, double, null, and bytes.
References to Named Schemata
If a named schema has already been defined in the same Avro IDL file, it may be referenced by name as if it were a primitive type:
record Card {
Suit suit; // refers to the enum Card defined above
int number;
}
Default Values
Default values for fields may be optionally specified by using an equals sign after the field name followed by a JSON expression indicating the default value. This JSON is interpreted as described in the spec.
Complex Types
Arrays
Array types are written in a manner that will seem familiar to C++ or Java programmers. An array of any type t is denoted array<t>. For example, an array of strings is denoted array<string>, and a multidimensional array of Foo records would be array<array<Foo>>.
Maps
Map types are written similarly to array types. An array that contains values of type t is written map<t>. As in the JSON schema format, all maps contain string-type keys.
Unions
Union types are denoted as union { typeA, typeB, typeC, … }. For example, this record contains a string field that is optional (unioned with null):
record RecordWithUnion {
union { null, string } optionalString;
}
Note that the same restrictions apply to Avro IDL unions as apply to unions defined in the JSON format; namely, a record may not contain multiple elements of the same type.
Defining RPC Messages
The syntax to define an RPC message within a Avro IDL protocol is similar to the syntax for a method declaration within a C header file or a Java interface. To define an RPC message add which takes two arguments named foo and bar, returning an int, simply include the following definition within the protocol:
int add(int foo, int bar = 0);
Message arguments, like record fields, may specify default values.
To define a message with no response, you may use the alias void, equivalent to the Avro null type:
void logMessage(string message);
If you have previously defined an error type within the same protocol, you may declare that a message can throw this error using the syntax:
void goKaboom() throws Kaboom;
To define a one-way message, use the keyword oneway after the parameter list, for example:
void fireAndForget(string message) oneway;
Other Language Features
All Java-style comments are supported within a Avro IDL file. Any text following // on a line is ignored, as is any text between /* and */, possibly spanning multiple lines.
Comments that begin with /** are used as the documentation string for the type or field definition that follows the comment.
Escaping Identifiers
Occasionally, one will need to use a reserved language keyword as an identifier. In order to do so, backticks (`) may be used to escape the identifier. For example, to define a message with the literal name error, you may write:
This syntax is allowed anywhere an identifier is expected.
Annotations for Ordering and Namespaces
Java-style annotations may be used to add additional properties to types and fields throughout Avro IDL.
For example, to specify the sort order of a field within a record, one may use the @order annotation before the field name as follows:
record MyRecord {
string @order("ascending") myAscendingSortField;
string @order("descending") myDescendingField;
string @order("ignore") myIgnoredField;
}
A field’s type may also be preceded by annotations, e.g.:
record MyRecord {
@java-class("java.util.ArrayList") array<string> myStrings;
}
This can be used to support java classes that can be serialized/deserialized via their toString/String constructor, e.g.:
record MyRecord {
@java-class("java.math.BigDecimal") string value;
@java-key-class("java.io.File") map<string> fileStates;
array<@java-class("java.math.BigDecimal") string> weights;
}
Similarly, a @namespace annotation may be used to modify the namespace when defining a named schema. For example:
@namespace("org.apache.avro.firstNamespace")
protocol MyProto {
@namespace("org.apache.avro.someOtherNamespace")
record Foo {}
record Bar {}
}
will define a protocol in the firstNamespace namespace. The record Foo will be defined in someOtherNamespace and Bar will be defined in firstNamespace as it inherits its default from its container.
Type and field aliases are specified with the @aliases annotation as follows:
@aliases(["org.old.OldRecord", "org.ancient.AncientRecord"])
record MyRecord {
string @aliases(["oldField", "ancientField"]) myNewField;
}
Some annotations like those listed above are handled specially. All other annotations are added as properties to the protocol, message, schema or field.
Complete Example
The following is a complete example of a Avro IDL file that shows most of the above features:
/**
* An example protocol in Avro IDL
*/
@namespace("org.apache.avro.test")
protocol Simple {
@aliases(["org.foo.KindOf"])
enum Kind {
FOO,
BAR, // the bar enum value
BAZ
}
fixed MD5(16);
record TestRecord {
@order("ignore")
string name;
@order("descending")
Kind kind;
MD5 hash;
union { MD5, null} @aliases(["hash"]) nullableHash;
array<long> arrayOfLongs;
}
error TestError {
string message;
}
string hello(string greeting);
TestRecord echo(TestRecord `record`);
int add(int arg1, int arg2);
bytes echoBytes(bytes data);
void `error`() throws TestError;
void ping() oneway;
}
Additional examples may be found in the Avro source tree under the src/test/idl/input directory.
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
10.7 - SASL profile
Introduction
SASL (RFC 2222) provides a framework for authentication and security of network protocols. Each protocol that uses SASL is meant to define a SASL profile. This document provides a SASL profile for connection-based Avro RPC.
Overview
SASL negotiation proceeds as a series of message interactions over a connection between a client and server using a selected SASL mechanism. The client starts this negotiation by sending its chosen mechanism name with an initial (possibly empty) message. Negotiation proceeds with the exchange of messages until either side indicates success or failure. The content of the messages is mechanism-specific. If the negotiation succeeds, then the session can proceed over the connection, otherwise it must be abandoned.
Some mechanisms continue to process session data after negotiation (e.g., encrypting it), while some specify that further session data is transmitted unmodified.
Negotiation
Commands
Avro SASL negotiation uses four one-byte commands.
- 0: START Used in a client’s initial message.
- 1: CONTINUE Used while negotiation is ongoing.
- 2: FAIL Terminates negotiation unsuccessfully.
- 3: COMPLETE Terminates negotiation successfully.
The format of a START message is:
| 0 | 4-byte mechanism name length | mechanism name | 4-byte payload length | payload data |
The format of a CONTINUE message is:
| 1 | 4-byte payload length | payload data |
The format of a FAIL message is:
| 2 | 4-byte message length | UTF-8 message |
The format of a COMPLETE message is:
| 3 | 4-byte payload length | payload data |
Process
Negotiation is initiated by a client sending a START command containing the client’s chosen mechanism name and any mechanism-specific payload data.
The server and client then interchange some number (possibly zero) of CONTINUE messages. Each message contains payload data that is processed by the security mechanism to generate the next message.
Once either the client or server send a FAIL message then negotiation has failed. UTF-8-encoded text is included in the failure message. Once either a FAIL message has been sent or received, or any other error occurs in the negotiation, further communication on this connection must cease.
Once either the client or server send a COMPLETE message then negotiation has completed successfully. Session data may now be transmitted over the connection until it is closed by either side.
Session Data
If no SASL QOP (quality of protection) is negotiated, then all subsequent writes to/reads over this connection are written/read unmodified. In particular, messages use Avro framing, and are of the form:
| 4-byte frame length | frame data | ... | 4 zero bytes |
If a SASL QOP is negotiated, then it must be used by the connection for all subsequent messages. This is done by wrapping each non-empty frame written using the security mechanism and unwrapping each non-empty frame read. The length written in each non-empty frame is the length of the wrapped data. Complete frames must be passed to the security mechanism for unwrapping. Unwrapped data is then passed to the application as the content of the frame.
If at any point processing fails due to wrapping, unwrapping or framing errors, then all further communication on this connection must cease.
Anonymous Mechanism
The SASL anonymous mechanism (RFC 2245) is quite simple to implement. In particular, an initial anonymous request may be prefixed by the following static sequence:
| 0 | 0009 | ANONYMOUS | 0000 |
If a server uses the anonymous mechanism, it should check that the mechanism name in the start message prefixing the first request received is ‘ANONYMOUS’, then simply prefix its initial response with a COMPLETE message of:
| 3 | 0000 |
If an anonymous server recieves some other mechanism name, then it may respond with a FAIL message as simple as:
| 2 | 0000 |
Note that the anonymous mechanism need add no additional round-trip messages between client and server. The START message can be piggybacked on the initial request and the COMPLETE or FAIL message can be piggybacked on the initial response.
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
10.8 - Wiki
The Wiki page can be found here.
10.9 - FAQ
What is Avro?
Avro is a data serialization system.
Avro provides:
- Rich data structures.
- A compact, fast, binary data format.
- A container file, to store persistent data.
- Remote procedure call (RPC).
- Simple integration with dynamic languages. Code generation is not required to read or write data files nor to use or implement RPC protocols. Code generation as an optional optimization, only worth implementing for statically typed languages.
How can I get started quickly with Avro?
Check out the Quick Start Guide.
How do I statically compile a schema or protocol into generated code?
In Java
- Add the avro jar, the jackson-mapper-asl.jar and jackson-core-asl.jar to your CLASSPATH.
- Run java org.apache.avro.specific.SpecificCompiler .
This appears to be out of date, the SpecificCompiler requires two arguments, presumably an input and and output file, but it isn’t clear that this does.
Or use the Schema or Protocol Ant tasks. Avro’s build.xml provides examples of how these are used.
Lastly, you can also use the “avro-tools” jar which ships with an Avro release. Just use the “compile (schema|protocol)” command.
How are Strings represented in Java?
They use org.apache.avro.util.Utf8, not java.lang.String.
More generally, how do Avro types map to Java types?
The mappings are documented in the package javadoc for generic, specific and reflect API.
From Doug Cutting:
HDFS splits files into blocks, and mapreduce runs a map task for each block. When the task starts, it needs to be able to seek into the file to the start of the block process through the block’s end. If the file were, e.g., a gzip file, this would not be possible, since gzip files must be decompressed from the start. One cannot seek into the middle of a gzip file and start decompressing. So Hadoop’s SequenceFile places a marker periodically (~64k) in the file at record and compression boundaries, where processing can be sensibly started. Then, when a map task starts processing an HDFS block, it finds the first marker after the block’s start and continues through the first marker in the next block of the file. This requires a bit of non-local access (~0.1%). Avro’s data file uses the same method as SequenceFile.
Why isn’t every value in Avro nullable?
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 of every value stored and
corresponding computational costs to create this bit on write and
interpret it on read. These costs are wasted when values may not in
fact be null, as is the case in many datasets. In Avro such costs are
only paid when values may actually be null.
Also, allowing values to be null is a well-known source of errors. In
Avro, a value declared as non-null will always be non-null and programs
need not test for null values when processing it nor will they ever fail
for lack of such tests.
Tony Hoare calls his invention of null references his “Billion Dollar
Mistake”.
http://qconlondon.com/london-2009/presentation/Null+References:+The+Billion+Dollar+Mistake
Also note that in some programming languages not all values are permitted to be null. For example, in Java, values of type boolean, byte, short, char, int, float, long, and double may not be null.
How can I serialize directly to/from a byte array?
As pointed out in the specification, Avro data should always be stored with its schema. The Avro provided classes DataFileWriter, DataFileReader, and DataFileStream all ensure this by serializing the Schema in a container header. In some special cases, such as when implementing a new storage system or writing unit tests, you may need to write and read directly with the bare Avro serialized values. The following examples use code generated for Java from an Avro Schema, specifically the User example from the Quickstart guide.
Serializing to a byte array
This example takes a User object and returns a newly allocated byte array with the Avro serialization of that user.
ByteArrayOutputStream out = new ByteArrayOutputStream();
BinaryEncoder encoder = EncoderFactory.get().binaryEncoder(out, null);
DatumWriter<User> writer = new SpecificDatumWriter<User>(User.getClassSchema());
writer.write(user, encoder);
encoder.flush();
out.close();
byte[] serializedBytes = out.toByteArray();
Deserializing from a byte array
This example takes a byte array containing the Avro serialization of a user and returns a User object.SpecificDatumReader reader = new SpecificDatumReader(User.getClassSchema());
Decoder decoder = DecoderFactory.get().binaryDecoder(bytes, null);
User user = reader.read(null, decoder);
Note: If you are serializing or deserializing in a loop or as a method, you should be reusing objects, readers and/or writers. Check the JavaDoc to see how to reuse the objects.
11 - Apache Avro™ 1.8.0 Documentation
Introduction
Apache Avro™ is a data serialization system.
Avro provides:
- Rich data structures.
- A compact, fast, binary data format.
- A container file, to store persistent data.
- Remote procedure call (RPC).
- Simple integration with dynamic languages. Code generation is not required to read or write data files nor to use or implement RPC protocols. Code generation as an optional optimization, only worth implementing for statically typed languages.
Schemas
Avro relies on schemas. When Avro data is read, the schema used when writing it is always present. This permits each datum to be written with no per-value overheads, making serialization both fast and small. This also facilitates use with dynamic, scripting languages, since data, together with its schema, is fully self-describing.
When Avro data is stored in a file, its schema is stored with it, so that files may be processed later by any program. If the program reading the data expects a different schema this can be easily resolved, since both schemas are present.
When Avro is used in RPC, the client and server exchange schemas in the connection handshake. (This can be optimized so that, for most calls, no schemas are actually transmitted.) Since both client and server both have the other’s full schema, correspondence between same named fields, missing fields, extra fields, etc. can all be easily resolved.
Avro schemas are defined with JSON . This facilitates implementation in languages that already have JSON libraries.
Comparison with other systems
Avro provides functionality similar to systems such as Thrift, Protocol Buffers, etc. Avro differs from these systems in the following fundamental aspects.
- Dynamic typing: Avro does not require that code be generated. Data is always accompanied by a schema that permits full processing of that data without code generation, static datatypes, etc. This facilitates construction of generic data-processing systems and languages.
- Untagged data: Since the schema is present when data is read, considerably less type information need be encoded with data, resulting in smaller serialization size.
- No manually-assigned field IDs: When a schema changes, both the old and new schema are always present when processing data, so differences may be resolved symbolically, using field names.
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
11.1 - Getting Started (Java)
This is a short guide for getting started with Apache Avro™ using Java. This guide only covers using Avro for data serialization; see Patrick Hunt’s Avro RPC Quick Start for a good introduction to using Avro for RPC.
Download
Avro implementations for C, C++, C#, Java, PHP, Python, and Ruby can be downloaded from the Apache Avro™ Releases page. This guide uses Avro 1.8.0, the latest version at the time of writing. For the examples in this guide, download avro-1.8.0.jar and avro-tools-1.8.0.jar. The Avro Java implementation also depends on the Jackson JSON library. From the Jackson download page, download the core-asl and mapper-asl jars. Add avro-1.8.0.jar and the Jackson jars to your project’s classpath (avro-tools will be used for code generation).
Alternatively, if you are using Maven, add the following dependency to your POM:
<dependency>
<groupId>org.apache.avro</groupId>
<artifactId>avro</artifactId>
<version>1.8.0</version>
</dependency>
As well as the Avro Maven plugin (for performing code generation):
<plugin>
<groupId>org.apache.avro</groupId>
<artifactId>avro-maven-plugin</artifactId>
<version>1.8.0</version>
<executions>
<execution>
<phase>generate-sources</phase>
<goals>
<goal>schema</goal>
</goals>
<configuration>
<sourceDirectory>${project.basedir}/src/main/avro/</sourceDirectory>
<outputDirectory>${project.basedir}/src/main/java/</outputDirectory>
</configuration>
</execution>
</executions>
</plugin>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-compiler-plugin</artifactId>
<configuration>
<source>1.6</source>
<target>1.6</target>
</configuration>
</plugin>
You may also build the required Avro jars from source. Building Avro is beyond the scope of this guide; see the Build Documentation page in the wiki for more information.
Defining a schema
Avro schemas are defined using JSON. Schemas are composed of primitive types (null, boolean, int, long, float, double, bytes, and string) and complex types (record, enum, array, map, union, and fixed). You can learn more about Avro schemas and types from the specification, but for now let’s start with a simple schema example, user.avsc:
{"namespace": "example.avro",
"type": "record",
"name": "User",
"fields": [
{"name": "name", "type": "string"},
{"name": "favorite_number", "type": ["int", "null"]},
{"name": "favorite_color", "type": ["string", "null"]}
]
}
This schema defines a record representing a hypothetical user. (Note that a schema file can only contain a single schema definition.) At minimum, a record definition must include its type (“type”: “record”), a name (“name”: “User”), and fields, in this case name, favorite_number, and favorite_color. We also define a namespace (“namespace”: “example.avro”), which together with the name attribute defines the “full name” of the schema (example.avro.User in this case).
Fields are defined via an array of objects, each of which defines a name and type (other attributes are optional, see the record specification for more details). The type attribute of a field is another schema object, which can be either a primitive or complex type. For example, the name field of our User schema is the primitive type string, whereas the favorite_number and favorite_color fields are both unions, represented by JSON arrays. unions are a complex type that can be any of the types listed in the array; e.g., favorite_number can either be an int or null, essentially making it an optional field.
Serializing and deserializing with code generation
Compiling the schema
Code generation allows us to automatically create classes based on our previously-defined schema. Once we have defined the relevant classes, there is no need to use the schema directly in our programs. We use the avro-tools jar to generate code as follows:
java -jar /path/to/avro-tools-1.8.0.jar compile schema <schema file> <destination>
This will generate the appropriate source files in a package based on the schema’s namespace in the provided destination folder. For instance, to generate a User class in package example.avro from the schema defined above, run
java -jar /path/to/avro-tools-1.8.0.jar compile schema user.avsc .
Note that if you using the Avro Maven plugin, there is no need to manually invoke the schema compiler; the plugin automatically performs code generation on any .avsc files present in the configured source directory.
Creating Users
Now that we’ve completed the code generation, let’s create some Users, serialize them to a data file on disk, and then read back the file and deserialize the User objects.
First let’s create some Users and set their fields.
User user1 = new User();
user1.setName("Alyssa");
user1.setFavoriteNumber(256);
// Leave favorite color null
// Alternate constructor
User user2 = new User("Ben", 7, "red");
// Construct via builder
User user3 = User.newBuilder()
.setName("Charlie")
.setFavoriteColor("blue")
.setFavoriteNumber(null)
.build();
As shown in this example, Avro objects can be created either by invoking a constructor directly or by using a builder. Unlike constructors, builders will automatically set any default values specified in the schema. Additionally, builders validate the data as it set, whereas objects constructed directly will not cause an error until the object is serialized. However, using constructors directly generally offers better performance, as builders create a copy of the datastructure before it is written.
Note that we do not set user1’s favorite color. Since that record is of type [“string”, “null”], we can either set it to a string or leave it null; it is essentially optional. Similarly, we set user3’s favorite number to null (using a builder requires setting all fields, even if they are null).
Serializing
Now let’s serialize our Users to disk.
/// Serialize user1, user2 and user3 to disk
DatumWriter<User> userDatumWriter = new SpecificDatumWriter<User>(User.class);
DataFileWriter<User> dataFileWriter = new DataFileWriter<User>(userDatumWriter);
dataFileWriter.create(user1.getSchema(), new File("users.avro"));
dataFileWriter.append(user1);
dataFileWriter.append(user2);
dataFileWriter.append(user3);
dataFileWriter.close();
We create a DatumWriter, which converts Java objects into an in-memory serialized format. The SpecificDatumWriter class is used with generated classes and extracts the schema from the specified generated type.
Next we create a DataFileWriter, which writes the serialized records, as well as the schema, to the file specified in the dataFileWriter.create call. We write our users to the file via calls to the dataFileWriter.append method. When we are done writing, we close the data file.
Deserializing
Finally, let’s deserialize the data file we just created.
// Deserialize Users from disk
DatumReader<User> userDatumReader = new SpecificDatumReader<User>(User.class);
DataFileReader<User> dataFileReader = new DataFileReader<User>(file, userDatumReader);
User user = null;
while (dataFileReader.hasNext()) {
// Reuse user object by passing it to next(). This saves us from
// allocating and garbage collecting many objects for files with
// many items.
user = dataFileReader.next(user);
System.out.println(user);
}
This snippet will output:
{"name": "Alyssa", "favorite_number": 256, "favorite_color": null}
{"name": "Ben", "favorite_number": 7, "favorite_color": "red"}
{"name": "Charlie", "favorite_number": null, "favorite_color": "blue"}
Deserializing is very similar to serializing. We create a SpecificDatumReader, analogous to the SpecificDatumWriter we used in serialization, which converts in-memory serialized items into instances of our generated class, in this case User. We pass the DatumReader and the previously created File to a DataFileReader, analogous to the DataFileWriter, which reads the data file on disk.
Next we use the DataFileReader to iterate through the serialized Users and print the deserialized object to stdout. Note how we perform the iteration: we create a single User object which we store the current deserialized user in, and pass this record object to every call of dataFileReader.next. This is a performance optimization that allows the DataFileReader to reuse the same User object rather than allocating a new User for every iteration, which can be very expensive in terms of object allocation and garbage collection if we deserialize a large data file. While this technique is the standard way to iterate through a data file, it’s also possible to use for (User user : dataFileReader) if performance is not a concern.
Compiling and running the example code
This example code is included as a Maven project in the examples/java-example directory in the Avro docs. From this directory, execute the following commands to build and run the example:
$ mvn compile # includes code generation via Avro Maven plugin
$ mvn -q exec:java -Dexec.mainClass=example.SpecificMain
Serializing and deserializing without code generation
Data in Avro is always stored with its corresponding schema, meaning we can always read a serialized item regardless of whether we know the schema ahead of time. This allows us to perform serialization and deserialization without code generation.
Let’s go over the same example as in the previous section, but without using code generation: we’ll create some users, serialize them to a data file on disk, and then read back the file and deserialize the users objects.
Creating users
First, we use a Parser to read our schema definition and create a Schema object.
Schema schema = new Schema.Parser().parse(new File("user.avsc"));
Using this schema, let’s create some users.
GenericRecord user1 = new GenericData.Record(schema);
user1.put("name", "Alyssa");
user1.put("favorite_number", 256);
// Leave favorite color null
GenericRecord user2 = new GenericData.Record(schema);
user2.put("name", "Ben");
user2.put("favorite_number", 7);
user2.put("favorite_color", "red");
Since we’re not using code generation, we use GenericRecords to represent users. GenericRecord uses the schema to verify that we only specify valid fields. If we try to set a non-existent field (e.g., user1.put(“favorite_animal”, “cat”)), we’ll get an AvroRuntimeException when we run the program.
Note that we do not set user1’s favorite color. Since that record is of type [“string”, “null”], we can either set it to a string or leave it null; it is essentially optional.
Serializing
Now that we’ve created our user objects, serializing and deserializing them is almost identical to the example above which uses code generation. The main difference is that we use generic instead of specific readers and writers.
First we’ll serialize our users to a data file on disk.
// Serialize user1 and user2 to disk
File file = new File("users.avro");
DatumWriter<GenericRecord> datumWriter = new GenericDatumWriter<GenericRecord>(schema);
DataFileWriter<GenericRecord> dataFileWriter = new DataFileWriter<GenericRecord>(datumWriter);
dataFileWriter.create(schema, file);
dataFileWriter.append(user1);
dataFileWriter.append(user2);
dataFileWriter.close();
We create a DatumWriter, which converts Java objects into an in-memory serialized format. Since we are not using code generation, we create a GenericDatumWriter. It requires the schema both to determine how to write the GenericRecords and to verify that all non-nullable fields are present.
As in the code generation example, we also create a DataFileWriter, which writes the serialized records, as well as the schema, to the file specified in the dataFileWriter.create call. We write our users to the file via calls to the dataFileWriter.append method. When we are done writing, we close the data file.
Deserializing
Finally, we’ll deserialize the data file we just created.
// Deserialize users from disk
DatumReader<GenericRecord> datumReader = new GenericDatumReader<GenericRecord>(schema);
DataFileReader<GenericRecord> dataFileReader = new DataFileReader<GenericRecord>(file, datumReader);
GenericRecord user = null;
while (dataFileReader.hasNext()) {
// Reuse user object by passing it to next(). This saves us from
// allocating and garbage collecting many objects for files with
// many items.
user = dataFileReader.next(user);
System.out.println(user);
This outputs:
{"name": "Alyssa", "favorite_number": 256, "favorite_color": null}
{"name": "Ben", "favorite_number": 7, "favorite_color": "red"}
Deserializing is very similar to serializing. We create a GenericDatumReader, analogous to the GenericDatumWriter we used in serialization, which converts in-memory serialized items into GenericRecords. We pass the DatumReader and the previously created File to a DataFileReader, analogous to the DataFileWriter, which reads the data file on disk.
Next, we use the DataFileReader to iterate through the serialized users and print the deserialized object to stdout. Note how we perform the iteration: we create a single GenericRecord object which we store the current deserialized user in, and pass this record object to every call of dataFileReader.next. This is a performance optimization that allows the DataFileReader to reuse the same record object rather than allocating a new GenericRecord for every iteration, which can be very expensive in terms of object allocation and garbage collection if we deserialize a large data file. While this technique is the standard way to iterate through a data file, it’s also possible to use for (GenericRecord user : dataFileReader) if performance is not a concern.
Compiling and running the example code
This example code is included as a Maven project in the examples/java-example directory in the Avro docs. From this directory, execute the following commands to build and run the example:
$ mvn compile
$ mvn -q exec:java -Dexec.mainClass=example.GenericMain
11.2 - Getting Started (Python)
This is a short guide for getting started with Apache Avro™ using Python. This guide only covers using Avro for data serialization; see Patrick Hunt’s Avro RPC Quick Start for a good introduction to using Avro for RPC.
Download
Avro implementations for C, C++, C#, Java, PHP, Python, and Ruby can be downloaded from the Apache Avro™ Releases page. This guide uses Avro 1.8.0, the latest version at the time of writing. Download and unzip avro-1.8.0.tar.gz, and install via python setup.py (this will probably require root privileges). Ensure that you can import avro from a Python prompt.
$ tar xvf avro-1.8.0.tar.gz
$ cd avro-1.8.0
$ sudo python setup.py install
$ python
>>> import avro # should not raise ImportError
Alternatively, you may build the Avro Python library from source. From your the root Avro directory, run the commands
$ cd lang/py/
$ ant
$ sudo python setup.py install
$ python
>>> import avro # should not raise ImportError
Defining a schema
Avro schemas are defined using JSON. Schemas are composed of primitive types (null, boolean, int, long, float, double, bytes, and string) and complex types (record, enum, array, map, union, and fixed). You can learn more about Avro schemas and types from the specification, but for now let’s start with a simple schema example, user.avsc:
{"namespace": "example.avro",
"type": "record",
"name": "User",
"fields": [
{"name": "name", "type": "string"},
{"name": "favorite_number", "type": ["int", "null"]},
{"name": "favorite_color", "type": ["string", "null"]}
]
}
This schema defines a record representing a hypothetical user. (Note that a schema file can only contain a single schema definition.) At minimum, a record definition must include its type (“type”: “record”), a name (“name”: “User”), and fields, in this case name, favorite_number, and favorite_color. We also define a namespace (“namespace”: “example.avro”), which together with the name attribute defines the “full name” of the schema (example.avro.User in this case).
Fields are defined via an array of objects, each of which defines a name and type (other attributes are optional, see the record specification for more details). The type attribute of a field is another schema object, which can be either a primitive or complex type. For example, the name field of our User schema is the primitive type string, whereas the favorite_number and favorite_color fields are both unions, represented by JSON arrays. unions are a complex type that can be any of the types listed in the array; e.g., favorite_number can either be an int or null, essentially making it an optional field.
Serializing and deserializing without code generation
Data in Avro is always stored with its corresponding schema, meaning we can always read a serialized item, regardless of whether we know the schema ahead of time. This allows us to perform serialization and deserialization without code generation. Note that the Avro Python library does not support code generation.
Try running the following code snippet, which serializes two users to a data file on disk, and then reads back and deserializes the data file:
import avro.schema
from avro.datafile import DataFileReader, DataFileWriter
from avro.io import DatumReader, DatumWriter
schema = avro.schema.parse(open("user.avsc", "rb").read())
writer = DataFileWriter(open("users.avro", "wb"), DatumWriter(), schema)
writer.append({"name": "Alyssa", "favorite_number": 256})
writer.append({"name": "Ben", "favorite_number": 7, "favorite_color": "red"})
writer.close()
reader = DataFileReader(open("users.avro", "rb"), DatumReader())
for user in reader:
print user
reader.close()
This outputs:
{u'favorite_color': None, u'favorite_number': 256, u'name': u'Alyssa'}
{u'favorite_color': u'red', u'favorite_number': 7, u'name': u'Ben'}
Do make sure that you open your files in binary mode (i.e. using the modes wb or rb respectively). Otherwise you might generate corrupt files due to automatic replacement of newline characters with the platform-specific representations.
Let’s take a closer look at what’s going on here.
schema = avro.schema.parse(open("user.avsc", "rb").read())
avro.schema.parse takes a string containing a JSON schema definition as input and outputs a avro.schema.Schema object (specifically a subclass of Schema, in this case RecordSchema). We’re passing in the contents of our user.avsc schema file here.
writer = DataFileWriter(open("users.avro", "wb"), DatumWriter(), schema)
We create a DataFileWriter, which we’ll use to write serialized items to a data file on disk. The DataFileWriter constructor takes three arguments:
- The file we’ll serialize to
- A DatumWriter, which is responsible for actually serializing the items to Avro’s binary format (DatumWriters can be used separately from DataFileWriters, e.g., to perform IPC Avro TODO: is this true??).
- The schema we’re using. The DataFileWriter needs the schema both to write the schema to the data file, and to verify that the items we write are valid items and write the appropriate fields.
writer.append({"name": "Alyssa", "favorite_number": 256})
writer.append({"name": "Ben", "favorite_number": 7, "favorite_color": "red"})
We use DataFileWriter.append to add items to our data file. Avro records are represented as Python dicts. Since the field favorite_color has type [“int”, “null”], we are not required to specify this field, as shown in the first append. Were we to omit the required name field, an exception would be raised. Any extra entries not corresponding to a field are present in the dict are ignored.
reader = DataFileReader(open("users.avro", "rb"), DatumReader())
We open the file again, this time for reading back from disk. We use a DataFileReader and DatumReader analagous to the DataFileWriter and DatumWriter above.
for user in reader:
print user
The DataFileReader is an iterator that returns dicts corresponding to the serialized items.
11.3 - Specification
Introduction
This document defines Apache Avro. It is intended to be the authoritative specification. Implementations of Avro must adhere to this document.
Schema Declaration
A Schema is represented in JSON by one of:
- A JSON string, naming a defined type.
- A JSON object, of the form:
{"type": "typeName" ...attributes...}
where typeName is either a primitive or derived type name, as defined below. Attributes not defined in this document are permitted as metadata, but must not affect the format of serialized data.
- A JSON array, representing a union of embedded types.
Primitive Types
The set of primitive type names is:
- null: no value
- boolean: a binary value
- int: 32-bit signed integer
- long: 64-bit signed integer
- float: single precision (32-bit) IEEE 754 floating-point number
- double: double precision (64-bit) IEEE 754 floating-point number
- bytes: sequence of 8-bit unsigned bytes
- string: unicode character sequence
Primitive types have no specified attributes.
Primitive type names are also defined type names. Thus, for example, the schema “string” is equivalent to:
Complex Types
Avro supports six kinds of complex types: records, enums, arrays, maps, unions and fixed.
Records
Records use the type name “record” and support three attributes:
- name: a JSON string providing the name of the record (required).
- namespace, a JSON string that qualifies the name;
- doc: a JSON string providing documentation to the user of this schema (optional).
- aliases: a JSON array of strings, providing alternate names for this record (optional).
- fields: a JSON array, listing fields (required). Each field is a JSON object with the following attributes:
- name: a JSON string providing the name of the field (required), and
- doc: a JSON string describing this field for users (optional).
- type: A JSON object defining a schema, or a JSON string naming a record definition (required).
- _default: A default value for this field, used when reading instances that lack this field (optional). 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. Default values for bytes and fixed fields are JSON strings, where Unicode code points 0-255 are mapped to unsigned 8-bit byte values 0-255.
field default values
| avro type |
json type |
example |
| null |
null |
null |
| boolean |
boolean |
true |
| int,long |
integer |
1 |
| float,double |
number |
1.1 |
| bytes |
string |
"\u00FF" |
| string |
string |
"foo" |
| record |
object |
{"a": 1} |
| enum |
string |
"FOO" |
| array |
array |
[1] |
| map |
object |
{"a": 1} |
| fixed |
string |
"\u00ff" |
- order: specifies how this field impacts sort ordering of this record (optional). Valid values are “ascending” (the default), “descending”, or “ignore”. For more details on how this is used, see the sort order section below.
- aliases: a JSON array of strings, providing alternate names for this field (optional).
For example, a linked-list of 64-bit values may be defined with:
{
"type": "record",
"name": "LongList",
"aliases": ["LinkedLongs"], // old name for this
"fields" : [
{"name": "value", "type": "long"}, // each element has a long
{"name": "next", "type": ["null", "LongList"]} // optional next element
]
}
Enums
Enums use the type name “enum” and support the following attributes:
- name: a JSON string providing the name of the enum (required).
- namespace, a JSON string that qualifies the name ;
- aliases: a JSON array of strings, providing alternate names for this enum (optional).
- doc: a JSON string providing documentation to the user of this schema (optional).
- symbols: a JSON array, listing symbols, as JSON strings (required). All symbols in an enum must be unique; duplicates are prohibited. Every symbol must match the regular expression [A-Za-z_][A-Za-z0-9_]* (the same requirement as for names).
- default: A default value for this enumeration, used during resolution when the reader encounters a symbol from the writer that isn’t defined in the reader’s schema (optional). The value provided here must be a JSON string that’s a member of the symbols array. See documentation on schema resolution for how this gets used.
For example, playing card suits might be defined with:
{
"type": "enum",
"name": "Suit",
"symbols" : ["SPADES", "HEARTS", "DIAMONDS", "CLUBS"]
}
Arrays
Arrays use the type name “array” and support a single attribute:
- items: the schema of the array’s items.
For example, an array of strings is declared with:
{
"type": "array",
"items" : "string"
}
Maps
Maps use the type name “map” and support one attribute:
- values: the schema of the map’s values.
Map keys are assumed to be strings.
For example, a map from string to long is declared with:
{
"type": "map",
"values" : "long"
}
Unions
Unions, as mentioned above, are represented using JSON arrays. For example, ["null", "string"] declares a schema which may be either a null or string.
(Note that when a default value is specified for a record field whose type is a union, the type of the default value must match the first element of the union. Thus, for unions containing “null”, the “null” is usually listed first, since the default value of such unions is typically null.)
Unions may not contain more than one schema with the same type, except for the named types record, fixed and enum. For example, unions containing two array types or two map types are not permitted, but two types with different names are permitted. (Names permit efficient resolution when reading and writing unions.)
Unions may not immediately contain other unions.
Fixed
Fixed uses the type name “fixed” and supports two attributes:
- name: a string naming this fixed (required).
- namespace, a string that qualifies the name;
- aliases: a JSON array of strings, providing alternate names for this enum (optional).
- size: an integer, specifying the number of bytes per value (required).
For example, 16-byte quantity may be declared with:
{"type": "fixed", "size": 16, "name": "md5"}
Names
Record, enums and fixed are named types. Each has a fullname that is composed of two parts; a name and a namespace. Equality of names is defined on the fullname.
The name portion of a fullname, record field names, and enum symbols must:
- start with [A-Za-z_]
- subsequently contain only [A-Za-z0-9_]
A namespace is a dot-separated sequence of such names. The empty string may also be used as a namespace to indicate the null namespace. Equality of names (including field names and enum symbols) as well as fullnames is case-sensitive.
In record, enum and fixed definitions, the fullname is determined in one of the following ways:
A name and namespace are both specified. For example, one might use “name”: “X”, “namespace”: “org.foo” to indicate the fullname org.foo.X.
-
A fullname is specified. If the name specified contains a dot, then it is assumed to be a fullname, and any namespace also specified is ignored. For example, use “name”: “org.foo.X” to indicate the fullname org.foo.X.
-
A name only is specified, i.e., a name that contains no dots. In this case the namespace is taken from the most tightly enclosing schema or protocol. For example, if “name”: “X” is specified, and this occurs within a field of the record definition of org.foo.Y, then the fullname is org.foo.X. If there is no enclosing namespace then the null namespace is used.
References to previously defined names are as in the latter two cases above: if they contain a dot they are a fullname, if they do not contain a dot, the namespace is the namespace of the enclosing definition.
Primitive type names have no namespace and their names may not be defined in any namespace.
A schema or protocol may not contain multiple definitions of a fullname. Further, a name must be defined before it is used (“before” in the depth-first, left-to-right traversal of the JSON parse tree, where the types attribute of a protocol is always deemed to come “before” the messages attribute.)
Aliases
Named types and fields may have aliases. An implementation may optionally use aliases to map a writer’s schema to the reader’s. This facilitates both schema evolution as well as processing disparate datasets.
Aliases function by re-writing the writer’s schema using aliases from the reader’s schema. For example, if the writer’s schema was named “Foo” and the reader’s schema is named “Bar” and has an alias of “Foo”, then the implementation would act as though “Foo” were named “Bar” when reading. Similarly, if data was written as a record with a field named “x” and is read as a record with a field named “y” with alias “x”, then the implementation would act as though “x” were named “y” when reading.
A type alias may be specified either as a fully namespace-qualified, or relative to the namespace of the name it is an alias for. For example, if a type named “a.b” has aliases of “c” and “x.y”, then the fully qualified names of its aliases are “a.c” and “x.y”.
Data Serialization
Avro data is always serialized with its schema. Files that store Avro data should always also include the schema for that data in the same file. Avro-based remote procedure call (RPC) systems must also guarantee that remote recipients of data have a copy of the schema used to write that data.
Because the schema used to write data is always available when the data is read, Avro data itself is not tagged with type information. The schema is required to parse data.
In general, both serialization and deserialization proceed as a depth-first, left-to-right traversal of the schema, serializing primitive types as they are encountered.
Encodings
Avro specifies two serialization encodings: binary and JSON. Most applications will use the binary encoding, as it is smaller and faster. But, for debugging and web-based applications, the JSON encoding may sometimes be appropriate.
Binary Encoding
Primitive Types
Primitive types are encoded in binary as follows:
- null is written as zero bytes.
- a boolean is written as a single byte whose value is either 0 (false) or 1 (true).
- int and long values are written using variable-length zig-zag coding. Some examples:
| value |
hex |
| 0 |
00 |
| -1 |
01 |
| 1 |
02 |
| -2 |
03 |
| 2 |
04 |
| … |
|
| -64 |
7f |
| 64 |
80 01 |
| … |
|
- a float is written as 4 bytes. The float is converted into a 32-bit integer using a method equivalent to Java’s floatToIntBits and then encoded in little-endian format.
- a double is written as 8 bytes. The double is converted into a 64-bit integer using a method equivalent to Java’s doubleToLongBits and then encoded in little-endian format.
- bytes are encoded as a long followed by that many bytes of data.
- a string is encoded as a long followed by that many bytes of UTF-8 encoded character data.
For example, the three-character string “foo” would be encoded as the long value 3 (encoded as hex 06) followed by the UTF-8 encoding of ‘f’, ‘o’, and ‘o’ (the hex bytes 66 6f 6f):
06 66 6f 6f
Complex Types
Complex types are encoded in binary as follows:
Records
A record is encoded by encoding the values of its fields in the order that they are declared. In other words, a record is encoded as just the concatenation of the encodings of its fields. Field values are encoded per their schema.
For example, the record schema
{
"type": "record",
"name": "test",
"fields" : [
{"name": "a", "type": "long"},
{"name": "b", "type": "string"}
]
}
An instance of this record whose a field has value 27 (encoded as hex 36) and whose b field has value “foo” (encoded as hex bytes 06 66 6f 6f), would be encoded simply as the concatenation of these, namely the hex byte sequence:
36 06 66 6f 6f
Enums
An enum is encoded by a int, representing the zero-based position of the symbol in the schema.
For example, consider the enum:
{"type": "enum", "name": "Foo", "symbols": ["A", "B", "C", "D"] }
This would be encoded by an int between zero and three, with zero indicating “A”, and 3 indicating “D”.
Arrays
Arrays are encoded as a series of blocks. Each block consists of a long count value, followed by that many array items. A block with count zero indicates the end of the array. Each item is encoded per the array’s item schema.
If a block’s count is negative, its absolute value is used, and the count is followed immediately by a long block size indicating the number of bytes in the block. This block size permits fast skipping through data, e.g., when projecting a record to a subset of its fields.
For example, the array schema
{"type": "array", "items": "long"}
an array containing the items 3 and 27 could be encoded as the long value 2 (encoded as hex 04) followed by long values 3 and 27 (encoded as hex 06 36) terminated by zero:
04 06 36 00
The blocked representation permits one to read and write arrays larger than can be buffered in memory, since one can start writing items without knowing the full length of the array.
Maps
Maps are encoded as a series of blocks. Each block consists of a long count value, followed by that many key/value pairs. A block with count zero indicates the end of the map. Each item is encoded per the map’s value schema.
If a block’s count is negative, its absolute value is used, and the count is followed immediately by a long block size indicating the number of bytes in the block. This block size permits fast skipping through data, e.g., when projecting a record to a subset of its fields.
The blocked representation permits one to read and write maps larger than can be buffered in memory, since one can start writing items without knowing the full length of the map.
Unions
A union is encoded by first writing a long value indicating the zero-based position within the union of the schema of its value. The value is then encoded per the indicated schema within the union.
For example, the union schema ["null","string"] would encode:
-
null as zero (the index of “null” in the union):
00
-
the string “a” as one (the index of “string” in the union, 1, encoded as hex 02), followed by the serialized string:
02 02 61
Fixed
Fixed instances are encoded using the number of bytes declared in the schema.
JSON Encoding
Except for unions, the JSON encoding is the same as is used to encode field default values.
The value of a union is encoded in JSON as follows:
- if its type is null, then it is encoded as a JSON null;
- otherwise it is encoded as a JSON object with one name/value pair whose name is the type’s name and whose value is the recursively encoded value. For Avro’s named types (record, fixed or enum) the user-specified name is used, for other types the type name is used.
For example, the union schema ["null","string","Foo"], where Foo is a record name, would encode:
- null as null;
- the string “a” as
{"string": "a"} and
- a Foo instance as
{"Foo": {...}}, where {...} indicates the JSON encoding of a Foo instance.
Note that a schema is still required to correctly process JSON-encoded data. For example, the JSON encoding does not distinguish between int and long, float and double, records and maps, enums and strings, etc.
Sort Order
Avro defines a standard sort order for data. This permits data written by one system to be efficiently sorted by another system. This can be an important optimization, as sort order comparisons are sometimes the most frequent per-object operation. Note also that Avro binary-encoded data can be efficiently ordered without deserializing it to objects.
Data items may only be compared if they have identical schemas. Pairwise comparisons are implemented recursively with a depth-first, left-to-right traversal of the schema. The first mismatch encountered determines the order of the items.
Two items with the same schema are compared according to the following rules.
- null data is always equal.
- boolean data is ordered with false before true.
- int, long, float and double data is ordered by ascending numeric value.
- bytes and fixed data are compared lexicographically by unsigned 8-bit values.
- string data is compared lexicographically by Unicode code point. Note that since UTF-8 is used as the binary encoding for strings, sorting of bytes and string binary data is identical.
- array data is compared lexicographically by element.
- enum data is ordered by the symbol’s position in the enum schema. For example, an enum whose symbols are
["z", "a"] would sort “z” values before “a” values.
- union data is first ordered by the branch within the union, and, within that, by the type of the branch. For example, an
["int", "string"] union would order all int values before all string values, with the ints and strings themselves ordered as defined above.
- record data is ordered lexicographically by field. If a field specifies that its order is:
- “ascending”, then the order of its values is unaltered.
- “descending”, then the order of its values is reversed.
- “ignore”, then its values are ignored when sorting.
- map data may not be compared. It is an error to attempt to compare data containing maps unless those maps are in an
"order":"ignore" record field.
Object Container Files
Avro includes a simple object container file format. A file has a schema, and all objects stored in the file must be written according to that schema, using binary encoding. Objects are stored in blocks that may be compressed. Syncronization markers are used between blocks to permit efficient splitting of files for MapReduce processing.
Files may include arbitrary user-specified metadata.
A file consists of:
- A file header, followed by
- one or more file data blocks.
A file header consists of:
- Four bytes, ASCII ‘O’, ‘b’, ‘j’, followed by 1.
- file metadata, including the schema.
- The 16-byte, randomly-generated sync marker for this file.
File metadata is written as if defined by the following map schema:
{"type": "map", "values": "bytes"}
All metadata properties that start with “avro.” are reserved. The following file metadata properties are currently used:
- avro.schema contains the schema of objects stored in the file, as JSON data (required).
- avro.codec the name of the compression codec used to compress blocks, as a string. Implementations are required to support the following codecs: “null” and “deflate”. If codec is absent, it is assumed to be “null”. The codecs are described with more detail below.
A file header is thus described by the following schema:
{"type": "record", "name": "org.apache.avro.file.Header",
"fields" : [
{"name": "magic", "type": {"type": "fixed", "name": "Magic", "size": 4}},
{"name": "meta", "type": {"type": "map", "values": "bytes"}},
{"name": "sync", "type": {"type": "fixed", "name": "Sync", "size": 16}}
]
}
A file data block consists of:
- A long indicating the count of objects in this block.
- A long indicating the size in bytes of the serialized objects in the current block, after any codec is applied
- The serialized objects. If a codec is specified, this is compressed by that codec.
- The file’s 16-byte sync marker.
Thus, each block’s binary data can be efficiently extracted or skipped without deserializing the contents. The combination of block size, object counts, and sync markers enable detection of corrupt blocks and help ensure data integrity.
Required Codecs
null
The “null” codec simply passes through data uncompressed.
deflate
The “deflate” codec writes the data block using the deflate algorithm as specified in RFC 1951, and typically implemented using the zlib library. Note that this format (unlike the “zlib format” in RFC 1950) does not have a checksum.
Optional Codecs
snappy
The “snappy” codec uses Google’s Snappy compression library. Each compressed block is followed by the 4-byte, big-endian CRC32 checksum of the uncompressed data in the block.
Protocol Declaration
Avro protocols describe RPC interfaces. Like schemas, they are defined with JSON text.
A protocol is a JSON object with the following attributes:
- protocol, a string, the name of the protocol (required);
- namespace, an optional string that qualifies the name;
- doc, an optional string describing this protocol;
- types, an optional list of definitions of named types (records, enums, fixed and errors). An error definition is just like a record definition except it uses “error” instead of “record”. Note that forward references to named types are not permitted.
- messages, an optional JSON object whose keys are message names and whose values are objects whose attributes are described below. No two messages may have the same name.
The name and namespace qualification rules defined for schema objects apply to protocols as well.
Messages
A message has attributes:
- a doc, an optional description of the message,
- a request, a list of named, typed parameter schemas (this has the same form as the fields of a record declaration);
- a response schema;
- an optional union of declared error schemas. The effective union has “string” prepended to the declared union, to permit transmission of undeclared “system” errors. For example, if the declared error union is
["AccessError"], then the effective union is ["string", "AccessError"]. When no errors are declared, the effective error union is ["string"]. Errors are serialized using the effective union; however, a protocol’s JSON declaration contains only the declared union.
- an optional one-way boolean parameter.
A request parameter list is processed equivalently to an anonymous record. Since record field lists may vary between reader and writer, request parameters may also differ between the caller and responder, and such differences are resolved in the same manner as record field differences.
The one-way parameter may only be true when the response type is "null" and no errors are listed.
Sample Protocol
For example, one may define a simple HelloWorld protocol with:
{
"namespace": "com.acme",
"protocol": "HelloWorld",
"doc": "Protocol Greetings",
"types": [
{"name": "Greeting", "type": "record", "fields": [
{"name": "message", "type": "string"}]},
{"name": "Curse", "type": "error", "fields": [
{"name": "message", "type": "string"}]}
],
"messages": {
"hello": {
"doc": "Say hello.",
"request": [{"name": "greeting", "type": "Greeting" }],
"response": "Greeting",
"errors": ["Curse"]
}
}
}
Message Transport
Messages may be transmitted via different transport mechanisms.
To the transport, a message is an opaque byte sequence.
A transport is a system that supports:
- transmission of request messages
- receipt of corresponding response messages
Servers may send a response message back to the client corresponding to a request message. The mechanism of correspondance is transport-specific. For example, in HTTP it is implicit, since HTTP directly supports requests and responses. But a transport that multiplexes many client threads over a single socket would need to tag messages with unique identifiers.
Transports may be either stateless or stateful. In a stateless transport, messaging assumes no established connection state, while stateful transports establish connections that may be used for multiple messages. This distinction is discussed further in the handshake section below.
HTTP as Transport
When HTTP is used as a transport, each Avro message exchange is an HTTP request/response pair. All messages of an Avro protocol should share a single URL at an HTTP server. Other protocols may also use that URL. Both normal and error Avro response messages should use the 200 (OK) response code. The chunked encoding may be used for requests and responses, but, regardless the Avro request and response are the entire content of an HTTP request and response. The HTTP Content-Type of requests and responses should be specified as “avro/binary”. Requests should be made using the POST method.
HTTP is used by Avro as a stateless transport.
Message Framing
Avro messages are framed as a list of buffers.
Framing is a layer between messages and the transport. It exists to optimize certain operations.
The format of framed message data is:
- a series of buffers, where each buffer consists of:
- a four-byte, big-endian buffer length, followed by
- that many bytes of buffer data.
- A message is always terminated by a zero-length buffer.
Framing is transparent to request and response message formats (described below). Any message may be presented as a single or multiple buffers.
Framing can permit readers to more efficiently get different buffers from different sources and for writers to more efficiently store different buffers to different destinations. In particular, it can reduce the number of times large binary objects are copied. For example, if an RPC parameter consists of a megabyte of file data, that data can be copied directly to a socket from a file descriptor, and, on the other end, it could be written directly to a file descriptor, never entering user space.
A simple, recommended, framing policy is for writers to create a new segment whenever a single binary object is written that is larger than a normal output buffer. Small objects are then appended in buffers, while larger objects are written as their own buffers. When a reader then tries to read a large object the runtime can hand it an entire buffer directly, without having to copy it.
Handshake
The purpose of the handshake is to ensure that the client and the server have each other’s protocol definition, so that the client can correctly deserialize responses, and the server can correctly deserialize requests. Both clients and servers should maintain a cache of recently seen protocols, so that, in most cases, a handshake will be completed without extra round-trip network exchanges or the transmission of full protocol text.
RPC requests and responses may not be processed until a handshake has been completed. With a stateless transport, all requests and responses are prefixed by handshakes. With a stateful transport, handshakes are only attached to requests and responses until a successful handshake response has been returned over a connection. After this, request and response payloads are sent without handshakes for the lifetime of that connection.
The handshake process uses the following record schemas:
{
"type": "record",
"name": "HandshakeRequest", "namespace":"org.apache.avro.ipc",
"fields": [
{"name": "clientHash",
"type": {"type": "fixed", "name": "MD5", "size": 16}},
{"name": "clientProtocol", "type": ["null", "string"]},
{"name": "serverHash", "type": "MD5"},
{"name": "meta", "type": ["null", {"type": "map", "values": "bytes"}]}
]
}
{
"type": "record",
"name": "HandshakeResponse", "namespace": "org.apache.avro.ipc",
"fields": [
{"name": "match",
"type": {"type": "enum", "name": "HandshakeMatch",
"symbols": ["BOTH", "CLIENT", "NONE"]}},
{"name": "serverProtocol",
"type": ["null", "string"]},
{"name": "serverHash",
"type": ["null", {"type": "fixed", "name": "MD5", "size": 16}]},
{"name": "meta",
"type": ["null", {"type": "map", "values": "bytes"}]}
]
}
- A client first prefixes each request with a
HandshakeRequest containing just the hash of its protocol and of the server’s protocol (clientHash!=null, clientProtocol=null, serverHash!=null), where the hashes are 128-bit MD5 hashes of the JSON protocol text. If a client has never connected to a given server, it sends its hash as a guess of the server’s hash, otherwise it sends the hash that it previously obtained from this server.
The server responds with a HandshakeResponse containing one of:
-
match=BOTH, serverProtocol=null, serverHash=null if the client sent the valid hash of the server’s protocol and the server knows what protocol corresponds to the client’s hash. In this case, the request is complete and the response data immediately follows the HandshakeResponse.
-
match=CLIENT, serverProtocol!=null, serverHash!=null if the server has previously seen the client’s protocol, but the client sent an incorrect hash of the server’s protocol. The request is complete and the response data immediately follows the HandshakeResponse. The client must use the returned protocol to process the response and should also cache that protocol and its hash for future interactions with this server.
-
match=NONE if the server has not previously seen the client’s protocol. The serverHash and serverProtocol may also be non-null if the server’s protocol hash was incorrect.
In this case the client must then re-submit its request with its protocol text (clientHash!=null, clientProtocol!=null, serverHash!=null) and the server should respond with a successful match (match=BOTH, serverProtocol=null, serverHash=null) as above.
The meta field is reserved for future handshake enhancements.
A call consists of a request message paired with its resulting response or error message. Requests and responses contain extensible metadata, and both kinds of messages are framed as described above.
The format of a call request is:
- request metadata, a map with values of type bytes
- the message name, an Avro string, followed by
- the message parameters. Parameters are serialized according to the message’s request declaration.
When the empty string is used as a message name a server should ignore the parameters and return an empty response. A client may use this to ping a server or to perform a handshake without sending a protocol message.
When a message is declared one-way and a stateful connection has been established by a successful handshake response, no response data is sent. Otherwise the format of the call response is:
- response metadata, a map with values of type bytes
- a one-byte error flag boolean, followed by either:
- if the error flag is false, the message response, serialized per the message’s response schema.
- if the error flag is true, the error, serialized per the message’s effective error union schema.
Schema Resolution
A reader of Avro data, whether from an RPC or a file, can always parse that data because the original schema must be provided along with the data. However, the reader may be programmed to read data into a different schema. For example, if the data was written with a different version of the software than it is read, then fields may have been added or removed from records. This section specifies how such schema differences should be resolved.
We refer to the schema used to write the data as the writer’s schema, and the schema that the application expects the reader’s schema. Differences between these should be resolved as follows:
- It is an error if the two schemas do not match.
To match, one of the following must hold:
-
both schemas are arrays whose item types match
-
both schemas are maps whose value types match
-
both schemas are enums whose names match
-
both schemas are fixed whose sizes and names match
-
both schemas are records with the same name
-
either schema is a union
-
both schemas have same primitive type
-
the writer’s schema may be promoted to the reader’s as follows:
- int is promotable to long, float, or double
- long is promotable to float or double
- float is promotable to double
- string is promotable to bytes
- bytes is promotable to string
-
if both are records:
- the ordering of fields may be different: fields are matched by name.
- schemas for fields with the same name in both records are resolved recursively.
- if the writer’s record contains a field with a name not present in the reader’s record, the writer’s value for that field is ignored.
- if the reader’s record schema has a field that contains a default value, and writer’s schema does not have a field with the same name, then the reader should use the default value from its field.
- if the reader’s record schema has a field with no default value, and writer’s schema does not have a field with the same name, an error is signalled.
-
if both are enums:
if the writer’s symbol is not present in the reader’s enum and the reader has a default value, then that value is used, otherwise an error is signalled.
This resolution algorithm is applied recursively to the reader’s and writer’s array item schemas.
This resolution algorithm is applied recursively to the reader’s and writer’s value schemas.
The first schema in the reader’s union that matches the selected writer’s union schema is recursively resolved against it. if none match, an error is signalled.
- if reader’s is a union, but writer’s is not
The first schema in the reader’s union that matches the writer’s schema is recursively resolved against it. If none match, an error is signalled.
- if writer’s is a union, but reader’s is not
If the reader’s schema matches the selected writer’s schema, it is recursively resolved against it. If they do not match, an error is signalled.
A schema’s doc fields are ignored for the purposes of schema resolution. Hence, the doc portion of a schema may be dropped at serialization.
One of the defining characteristics of Avro is that a reader is assumed to have the “same” schema used by the writer of the data the reader is reading. This assumption leads to a data format that’s compact and also amenable to many forms of schema evolution. However, the specification so far has not defined what it means for the reader to have the “same” schema as the writer. Does the schema need to be textually identical? Well, clearly adding or removing some whitespace to a JSON expression does not change its meaning. At the same time, reordering the fields of records clearly does change the meaning. So what does it mean for a reader to have “the same” schema as a writer?
Parsing Canonical Form is a transformation of a writer’s schema that let’s us define what it means for two schemas to be “the same” for the purpose of reading data written against the schema. It is called Parsing Canonical Form because the transformations strip away parts of the schema, like “doc” attributes, that are irrelevant to readers trying to parse incoming data. It is called Canonical Form because the transformations normalize the JSON text (such as the order of attributes) in a way that eliminates unimportant differences between schemas. If the Parsing Canonical Forms of two different schemas are textually equal, then those schemas are “the same” as far as any reader is concerned, i.e., there is no serialized data that would allow a reader to distinguish data generated by a writer using one of the original schemas from data generated by a writing using the other original schema. (We sketch a proof of this property in a companion document.)
The next subsection specifies the transformations that define Parsing Canonical Form. But with a well-defined canonical form, it can be convenient to go one step further, transforming these canonical forms into simple integers (“fingerprints”) that can be used to uniquely identify schemas. The subsection after next recommends some standard practices for generating such fingerprints.
Assuming an input schema (in JSON form) that’s already UTF-8 text for a valid Avro schema (including all quotes as required by JSON), the following transformations will produce its Parsing Canonical Form:
- [PRIMITIVES] Convert primitive schemas to their simple form (e.g., int instead of
{"type":"int"}).
- [FULLNAMES] Replace short names with fullnames, using applicable namespaces to do so. Then eliminate namespace attributes, which are now redundant.
- [STRIP] Keep only attributes that are relevant to parsing data, which are: type, name, fields, symbols, items, values, size. Strip all others (e.g., doc and aliases).
- [ORDER] Order the appearance of fields of JSON objects as follows: name, type, fields, symbols, items, values, size. For example, if an object has type, name, and size fields, then the name field should appear first, followed by the type and then the size fields.
- [STRINGS] For all JSON string literals in the schema text, replace any escaped characters (e.g., \uXXXX escapes) with their UTF-8 equivalents.
- [INTEGERS] Eliminate quotes around and any leading zeros in front of JSON integer literals (which appear in the size attributes of fixed schemas).
- [WHITESPACE] Eliminate all whitespace in JSON outside of string literals.
Schema Fingerprints
“[A] fingerprinting algorithm is a procedure that maps an arbitrarily large data item (such as a computer file) to a much shorter bit string, its fingerprint, that uniquely identifies the original data for all practical purposes” (quoted from Wikipedia). In the Avro context, fingerprints of Parsing Canonical Form can be useful in a number of applications; for example, to cache encoder and decoder objects, to tag data items with a short substitute for the writer’s full schema, and to quickly negotiate common-case schemas between readers and writers.
In designing fingerprinting algorithms, there is a fundamental trade-off between the length of the fingerprint and the probability of collisions. To help application designers find appropriate points within this trade-off space, while encouraging interoperability and ease of implementation, we recommend using one of the following three algorithms when fingerprinting Avro schemas:
- When applications can tolerate longer fingerprints, we recommend using the SHA-256 digest algorithm to generate 256-bit fingerprints of Parsing Canonical Forms. Most languages today have SHA-256 implementations in their libraries.
- At the opposite extreme, the smallest fingerprint we recommend is a 64-bit Rabin fingerprint. Below, we provide pseudo-code for this algorithm that can be easily translated into any programming language. 64-bit fingerprints should guarantee uniqueness for schema caches of up to a million entries (for such a cache, the chance of a collision is 3E-8). We don’t recommend shorter fingerprints, as the chances of collisions is too great (for example, with 32-bit fingerprints, a cache with as few as 100,000 schemas has a 50% chance of having a collision).
- Between these two extremes, we recommend using the MD5 message digest to generate 128-bit fingerprints. These make sense only where very large numbers of schemas are being manipulated (tens of millions); otherwise, 64-bit fingerprints should be sufficient. As with SHA-256, MD5 implementations are found in most libraries today.
These fingerprints are not meant to provide any security guarantees, even the longer SHA-256-based ones. Most Avro applications should be surrounded by security measures that prevent attackers from writing random data and otherwise interfering with the consumers of schemas. We recommend that these surrounding mechanisms be used to prevent collision and pre-image attacks (i.e., “forgery”) on schema fingerprints, rather than relying on the security properties of the fingerprints themselves.
Rabin fingerprints are cyclic redundancy checks computed using irreducible polynomials. In the style of the Appendix of RFC 1952 (pg 10), which defines the CRC-32 algorithm, here’s our definition of the 64-bit AVRO fingerprinting algorithm:
long fingerprint64(byte[] buf) {
if (FP_TABLE == null) initFPTable();
long fp = EMPTY;
for (int i = 0; i < buf.length; i++)
fp = (fp >>> 8) ^ FP_TABLE[(int)(fp ^ buf[i]) & 0xff];
return fp;
}
static long EMPTY = 0xc15d213aa4d7a795L;
static long[] FP_TABLE = null;
void initFPTable() {
FP_TABLE = new long[256];
for (int i = 0; i < 256; i++) {
long fp = i;
for (int j = 0; j < 8; j++)
fp = (fp >>> 1) ^ (EMPTY & -(fp & 1L));
FP_TABLE[i] = fp;
}
}
Readers interested in the mathematics behind this algorithm may want to read this book chapter. (Unlike RFC-1952 and the book chapter, we prepend a single one bit to messages. We do this because CRCs ignore leading zero bits, which can be problematic. Our code prepends a one-bit by initializing fingerprints using EMPTY, rather than initializing using zero as in RFC-1952 and the book chapter.)
Logical Types
A logical type is an Avro primitive or complex type with extra attributes to represent a derived type. The attribute logicalType must always be present for a logical type, and is a string with the name of one of the logical types listed later in this section. Other attributes may be defined for particular logical types.
A logical type is always serialized using its underlying Avro type so that values are encoded in exactly the same way as the equivalent Avro type that does not have a logicalType attribute. Language implementations may choose to represent logical types with an appropriate native type, although this is not required.
Language implementations must ignore unknown logical types when reading, and should use the underlying Avro type. If a logical type is invalid, for example a decimal with scale greater than its precision, then implementations should ignore the logical type and use the underlying Avro type.
Decimal
The decimal logical type represents an arbitrary-precision signed decimal number of the form unscaled × 10-scale.
A decimal logical type annotates Avro bytes or fixed types. The byte array must contain the two’s-complement representation of the unscaled integer value in big-endian byte order. The scale is fixed, and is specified using an attribute.
The following attributes are supported:
- scale, a JSON integer representing the scale (optional). If not specified the scale is 0.
- precision, a JSON integer representing the (maximum) precision of decimals stored in this type (required).
For example, the following schema represents decimal numbers with a maximum precision of 4 and a scale of 2:
{
"type": "bytes",
"logicalType": "decimal",
"precision": 4,
"scale": 2
}
Precision must be a positive integer greater than zero. If the underlying type is a fixed, then the precision is limited by its size. An array of length n can store at most floor(log10(28 × n - 1 - 1)) base-10 digits of precision.
Scale must be zero or a positive integer less than or equal to the precision.
For the purposes of schema resolution, two schemas that are decimal logical types match if their scales and precisions match.
Date
The date logical type represents a date within the calendar, with no reference to a particular time zone or time of day.
A date logical type annotates an Avro int, where the int stores the number of days from the unix epoch, 1 January 1970 (ISO calendar).
Time (millisecond precision)
The time-millis logical type represents a time of day, with no reference to a particular calendar, time zone or date, with a precision of one millisecond.
A time-millis logical type annotates an Avro int, where the int stores the number of milliseconds after midnight, 00:00:00.000.
Time (microsecond precision)
The time-micros logical type represents a time of day, with no reference to a particular calendar, time zone or date, with a precision of one microsecond.
A time-micros logical type annotates an Avro long, where the long stores the number of microseconds after midnight, 00:00:00.000000.
Timestamp (millisecond precision)
The timestamp-millis logical type represents an instant on the global timeline, independent of a particular time zone or calendar, with a precision of one millisecond.
A timestamp-millis logical type annotates an Avro long, where the long stores the number of milliseconds from the unix epoch, 1 January 1970 00:00:00.000 UTC.
Timestamp (microsecond precision)
The timestamp-micros logical type represents an instant on the global timeline, independent of a particular time zone or calendar, with a precision of one microsecond.
A timestamp-micros logical type annotates an Avro long, where the long stores the number of microseconds from the unix epoch, 1 January 1970 00:00:00.000000 UTC.
Duration
The duration logical type represents an amount of time defined by a number of months, days and milliseconds. This is not equivalent to a number of milliseconds, because, depending on the moment in time from which the duration is measured, the number of days in the month and number of milliseconds in a day may differ. Other standard periods such as years, quarters, hours and minutes can be expressed through these basic periods.
A duration logical type annotates Avro fixed type of size 12, which stores three little-endian unsigned integers that represent durations at different granularities of time. The first stores a number in months, the second stores a number in days, and the third stores a number in milliseconds.
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
11.4 - Trevni
Version 0.1
DRAFT
This document is the authoritative specification of a file format. Its intent is to permit compatible, independent implementations that read and/or write files in this format.
Introduction
Data sets are often described as a table composed of rows and columns. Each record in the dataset is considered a row, with each field of the record occupying a different column. Writing records to a file one-by-one as they are created results in a row-major format, like Hadoop’s SequenceFile or Avro data files.
In many cases higher query performance may be achieved if the data is instead organized in a column-major format, where multiple values of a given column are stored adjacently. This document defines such a column-major file format for datasets.
To permit scalable, distributed query evaluation, datasets are partitioned into row groups, containing distinct collections of rows. Each row group is organized in column-major order, while row groups form a row-major partitioning of the entire dataset.
Rationale
Goals
The format is meant satisfy the following goals:
Maximize the size of row groups. Disc drives are used most efficiently when sequentially accessing data. Consider a drive that takes 10ms to seek and transfers at 100MB/second. If a 10-column dataset whose values are all the same size is split into 10MB row groups, then accessing a single column will require a sequence of seek+1MB reads, for a cost of 20ms/MB processed. If the same dataset is split into 100MB row groups then this drops to 11ms/MB processed. This effect is exaggerated for datasets with larger numbers of columns and with columns whose values are smaller than average. So we’d prefer row groups that are 100MB or greater.
Permit random access within a row group. Some queries will first examine one column, and, only when certain relatively rare criteria are met, examine other columns. Rather than iterating through selected columns of the row-group in parallel, one might iterate through one column and randomly access another. This is called support for WHERE clauses, after the SQL operator of that name.
Minimize the number of files per dataset. HDFS is a primary intended deployment platform for these files. The HDFS Namenode requires memory for each file in the filesystem, thus for a format to be HDFS-friendly it should strive to require the minimum number of distinct files.
Support co-location of columns within row-groups. Row groups are the unit of parallel operation on a column dataset. For efficient file i/o, the entirety of a row-group should ideally reside on the host that is evaluating the query in order to avoid network latencies and bottlenecks.
Data integrity. The format should permit applications to detect data corruption. Many file systems may prevent corruption, but files may be moved between filesystems and be subject to corruption at points in that process. It is best if the data in a file can be validated independently.
Extensibility. The format should permit applications to store additional annotations about a datasets in the files, such as type information, origin, etc. Some environments may have metadata stores for such information, but not all do, and files might be moved among systems with different metadata systems. The ability to keep such information within the file simplifies the coordination of such information.
Minimal overhead. The column format should not make datasets appreciably larger. Storage is a primary cost and a choice to use this format should not require additional storage.
Primary format. The column format should be usable as a primary format for datasets, not as an auxiliary, accelerated format. Applications that process a dataset in row-major order should be able to easily consume column files and applications that produce datasets in row-major order should be able to easily generate column files.
Design
To meet these goals we propose the following design.
Each row group is a separate file. All values of a column in a file are written contiguously. This maximizes the row group size, optimizing performance when querying few and small columns.
Each file occupies a single HDFS block. A larger than normal block size may be specified, e.g., ~1GB instead of the typical ~100MB. This guarantees co-location and eliminates network use when query processing can be co-located with the file. This also moderates the memory impact on the HDFS Namenode since no small files are written.
Each column in a file is written as a sequence of ~64kB compressed blocks. The sequence is prefixed by a table describing all of the blocks in the column to permit random access within the column.
Application-specific metadata may be added at the file, column, and block levels.
Checksums are included with each block, providing data integrity.
Discussion
The use of a single block per file achieves the same effect as the custom block placement policy described in the CIF paper, but while still permitting HDFS rebalancing and not increasing the number of files in the namespace.
This section formally describes the proposed column file format.
Data Model
We assume a simple data model, where a record is a set of named fields, and the value of each field is a sequence of untyped bytes. A type system may be layered on top of this, as specified in the Type Mapping section below.
Primitive Values
We define the following primitive value types:
- Signed 64-bit long values are written using a variable-length zig-zag coding, where the high-order bit in each byte determines whether subsequent bytes are present. For example:
decimal value hex bytes
| decimal value |
hex bytes |
|
| 0 |
00 |
|
| -1 |
01 |
|
| 1 |
02 |
|
| … |
|
|
| -64 |
7f |
|
| 64 |
80 01 |
|
| … |
|
|
- bytes are encoded as a long followed by that many bytes of data.
- a string is encoded as a long followed by that many bytes of UTF-8 encoded character data.
For example, the three-character string “foo” would be encoded as the long value 3 (encoded as hex 06) followed by the UTF-8 encoding of ‘f’, ‘o’, and ‘o’ (the hex bytes 66 6f 6f): 06 66 6f 6f
Type Names
The following type names are used to describe column values:
- null, requires zero bytes. Sometimes used in array columns.
- boolean, one bit, packed into bytes, little-endian;
- int, like long, but restricted to 32-bit signed values
- long 64-bit signed values, represented as above
- fixed32 32-bit values stored as four bytes, little-endian.
- fixed64 64-bit values stored as eight bytes, little-endian.
- float 32-bit IEEE floating point value, little-endian
- double 64-bit IEEE floating point value, little-endian
- string as above
- bytes as above, may be used to encapsulate more complex objects
Type names are represented as strings (UTF-8 encoded, length-prefixed).
Metadata consists of:
- A long indicating the number of metadata key/value pairs.
- For each pair, a string key and bytes value.
All metadata properties that start with “trevni.” are reserved.
The following file metadata properties are defined:
- trevni.codec the name of the default compression codec used to compress blocks, as a string. Implementations are required to support the “null” codec. Optional. If absent, it is assumed to be “null”. Codecs are described in more detail below.
- trevni.checksum the name of the checksum algorithm used in this file, as a string. Implementations are required to support the “crc-32” checksum. Optional. If absent, it is assumed to be “null”. Checksums are described in more detail below.
The following column metadata properties are defined:
- trevni.codec the name of the compression codec used to compress the blocks of this column, as a string. Implementations are required to support the “null” codec. Optional. If absent, it is assumed to be “null”. Codecs are described in more detail below.
- trevni.name the name of the column, as a string. Required.
- trevni.type the type of data in the column. One of the type names above. Required.
- trevni.values if present, indicates that the initial value of each block in this column will be stored in the block’s descriptor. Not permitted for array columns or columns that specify a parent.
- trevni.array if present, indicates that each row in this column contains a sequence of values of the named type rather than just a single value. An integer length precedes each sequence of values indicating the count of values in the sequence. If the length is negative then it indicates a sequence of zero or one lengths, where -1 indicates two zeros, -2 two ones, -3 three zeros, -4 three ones, etc.
- trevni.parent if present, the name of an array column whose lengths are also used by this column. Thus values of this column are sequences but no lengths are stored in this column.
For example, consider the following row, as JSON, where all values are primitive types, but one has multiple values.
{"id"=566, "date"=23423234234
"from"="foo@bar.com",
"to"=["bar@baz.com", "bang@foo.com"],
"content"="Hi!"}
The columns for this might be specified as:
name=id type=int
name=date type=long
name=from type=string
name=to type=string array=true
name=content type=string
If a row contains an array of records, e.g. “received” in the following:
{"id"=566, "date"=23423234234
"from"="foo@bar.com",
"to"=["bar@baz.com", "bang@foo.com"],
"content"="Hi!"
"received"=[{"date"=234234234234, "host"="192.168.0.0.1"},
{"date"=234234545645, "host"="192.168.0.0.2"}]
}
Then one can define a parent column followed by a column for each field in the record, adding the following columns:
name=received type=null array=true
name=date type=long parent=received
name=host type=string parent=received
If an array value itself contains an array, e.g. the “sigs” below:
{"id"=566, "date"=23423234234
"from"="foo@bar.com",
"to"=["bar@baz.com", "bang@foo.com"],
"content"="Hi!"
"received"=[{"date"=234234234234, "host"="192.168.0.0.1",
"sigs"=[{"algo"="weak", "value"="0af345de"}]},
{"date"=234234545645, "host"="192.168.0.0.2",
"sigs"=[]}]
}
Then a parent column may be defined that itself has a parent column.
name=sigs type=null array=true parent=received
name=algo type=string parent=sigs
name=value type=string parent=sigs
No block metadata properties are currently defined.
A file consists of:
- A file header, followed by
- one or more columns.
A file header consists of:
- Four bytes, ASCII ‘T’, ‘r’, ‘v’, followed by 0x02.
- a fixed64 indicating the number of rows in the file
- a fixed32 indicating the number of columns in the file
- file metadata.
- for each column, its column metadata
- for each column, its starting position in the file as a fixed64.
A column consists of:
- A fixed32 indicating the number of blocks in this column.
- For each block, a block descriptor
- One or more blocks.
A block descriptor consists of:
- A fixed32 indicating the number of rows in the block
- A fixed32 indicating the size in bytes of the block before the codec is applied (excluding checksum).
- A fixed32 indicating the size in bytes of the block after the codec is applied (excluding checksum).
- If this column’s metadata declares it to include values, the first value in the column, serialized according to this column’s type.
A block consists of:
- The serialized column values. If a column is an array column then value sequences are preceded by their length, as an int. If a codec is specified, the values and lengths are compressed by that codec.
- The checksum, as determined by the file metadata.
Codecs
null
The “null” codec simply passes data through uncompressed.
deflate
The “deflate” codec writes the data block using the deflate algorithm as specified in RFC 1951.
snappy
The “snappy” codec uses Google’s Snappy compression library.
Checksum algorithms
null
The “null” checksum contains zero bytes.
crc-32
Each “crc-32” checksum contains the four bytes of an ISO 3309 CRC-32 checksum of the uncompressed block data as a fixed32.
Type Mappings
We define a standard mapping for how types defined in various serialization systems are represented in a column file. Records from these systems are shredded into columns. When records are nested, a depth-first recursive walk can assign a separate column for each primitive value.
Avro
Protocol Buffers
Thrift
Implementation Notes
Some possible techniques for writing column files include:
- Use a standard ~100MB block, buffer in memory up to the block size, then flush the file directly to HDFS. A single reduce task might create multiple output files. The namenode requires memory proportional to the number of names and blocks*replication. This would increase the number of names but not blocks, so this should still be much better than a file per column.
- Spill each column to a separate local, temporary file then, when the file is closed, append these files, writing a single file to HDFS whose block size is set to be that of the entire file. This would be a bit slower than and may have trouble when the local disk is full, but it would better use HDFS namespace and further reduce seeks when processing columns whose values are small.
- Use a separate mapreduce job to convert row-major files to column-major. The map output would output a by (row#, column#, value) tuple, partitioned by row# but sorted by column# then row#. The reducer could directly write the column file. But the column file format would need to be changed to write counts, descriptors, etc. at the end of files rather than at the front.
(1) is the simplest to implement and most implementations should start with it.
References
CIF Column-Oriented Storage Techniques for MapReduce, Floratou, Patel, Shekita, & Tata, VLDB 2011.
DREMEL Dremel: Interactive Analysis of Web-Scale Datasets, Melnik, Gubarev, Long, Romer, Shivakumar, & Tolton, VLDB 2010.
11.5 - MapReduce guide
Avro provides a convenient way to represent complex data structures within a Hadoop MapReduce job. Avro data can be used as both input to and output from a MapReduce job, as well as the intermediate format. The example in this guide uses Avro data for all three, but it’s possible to mix and match; for instance, MapReduce can be used to aggregate a particular field in an Avro record.
This guide assumes basic familiarity with both Hadoop MapReduce and Avro. See the Hadoop documentation and the Avro getting started guide for introductions to these projects. This guide uses the old MapReduce API (org.apache.hadoop.mapred) and the new MapReduce API (org.apache.hadoop.mapreduce).
Setup
The code from this guide is included in the Avro docs under examples/mr-example. The example is set up as a Maven project that includes the necessary Avro and MapReduce dependencies and the Avro Maven plugin for code generation, so no external jars are needed to run the example. In particular, the POM includes the following dependencies:
<dependency>
<groupId>org.apache.avro</groupId>
<artifactId>avro</artifactId>
<version>1.8.0</version>
</dependency>
<dependency>
<groupId>org.apache.avro</groupId>
<artifactId>avro-mapred</artifactId>
<version>1.8.0</version>
</dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-core</artifactId>
<version>1.1.0</version>
</dependency>
And the following plugin:
<plugin>
<groupId>org.apache.avro</groupId>
<artifactId>avro-maven-plugin</artifactId>
<version>1.8.0</version>
<executions>
<execution>
<phase>generate-sources</phase>
<goals>
<goal>schema</goal>
</goals>
<configuration>
<sourceDirectory>${project.basedir}/../</sourceDirectory>
<outputDirectory>${project.basedir}/target/generated-sources/</outputDirectory>
</configuration>
</execution>
</executions>
</plugin>
If you do not configure the sourceDirectory and outputDirectory properties, the defaults will be used. The sourceDirectory property defaults to src/main/avro. The outputDirectory property defaults to target/generated-sources. You can change the paths to match your project layout.
Alternatively, Avro jars can be downloaded directly from the Apache Avro™ Releases page. The relevant Avro jars for this guide are avro-1.8.0.jar and avro-mapred-1.8.0.jar, as well as avro-tools-1.8.0.jar for code generation and viewing Avro data files as JSON. In addition, you will need to install Hadoop in order to use MapReduce.
Example: ColorCount
Below is a simple example of a MapReduce that uses Avro. There is an example for both the old (org.apache.hadoop.mapred) and new (org.apache.hadoop.mapreduce) APIs under examples/mr-example/src/main/java/example/. MapredColorCount is the example for the older mapred API while MapReduceColorCount is the example for the newer mapreduce API. Both examples are below, but we will detail the mapred API in our subsequent examples.
MapredColorCount:
package example;
import java.io.IOException;
import org.apache.avro.*;
import org.apache.avro.Schema.Type;
import org.apache.avro.mapred.*;
import org.apache.hadoop.conf.*;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.mapred.*;
import org.apache.hadoop.util.*;
import example.avro.User;
public class MapredColorCount extends Configured implements Tool {
public static class ColorCountMapper extends AvroMapper<User, Pair<CharSequence, Integer>> {
@Override
public void map(User user, AvroCollector<Pair<CharSequence, Integer>> collector, Reporter reporter)
throws IOException {
CharSequence color = user.getFavoriteColor();
// We need this check because the User.favorite_color field has type ["string", "null"]
if (color == null) {
color = "none";
}
collector.collect(new Pair<CharSequence, Integer>(color, 1));
}
}
public static class ColorCountReducer extends AvroReducer<CharSequence, Integer,
Pair<CharSequence, Integer>> {
@Override
public void reduce(CharSequence key, Iterable<Integer> values,
AvroCollector<Pair<CharSequence, Integer>> collector,
Reporter reporter)
throws IOException {
int sum = 0;
for (Integer value : values) {
sum += value;
}
collector.collect(new Pair<CharSequence, Integer>(key, sum));
}
}
public int run(String[] args) throws Exception {
if (args.length != 2) {
System.err.println("Usage: MapredColorCount <input path> <output path>");
return -1;
}
JobConf conf = new JobConf(getConf(), MapredColorCount.class);
conf.setJobName("colorcount");
FileInputFormat.setInputPaths(conf, new Path(args[0]));
FileOutputFormat.setOutputPath(conf, new Path(args[1]));
AvroJob.setMapperClass(conf, ColorCountMapper.class);
AvroJob.setReducerClass(conf, ColorCountReducer.class);
// Note that AvroJob.setInputSchema and AvroJob.setOutputSchema set
// relevant config options such as input/output format, map output
// classes, and output key class.
AvroJob.setInputSchema(conf, User.getClassSchema());
AvroJob.setOutputSchema(conf, Pair.getPairSchema(Schema.create(Type.STRING),
Schema.create(Type.INT)));
JobClient.runJob(conf);
return 0;
}
public static void main(String[] args) throws Exception {
int res = ToolRunner.run(new Configuration(), new MapredColorCount(), args);
System.exit(res);
}
}
MapReduceColorCount:
package example;
import java.io.IOException;
import org.apache.avro.Schema;
import org.apache.avro.mapred.AvroKey;
import org.apache.avro.mapred.AvroValue;
import org.apache.avro.mapreduce.AvroJob;
import org.apache.avro.mapreduce.AvroKeyInputFormat;
import org.apache.avro.mapreduce.AvroKeyValueOutputFormat;
import org.apache.hadoop.conf.Configured;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.IntWritable;
import org.apache.hadoop.io.NullWritable;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.mapreduce.Job;
import org.apache.hadoop.mapreduce.Mapper;
import org.apache.hadoop.mapreduce.Reducer;
import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
import org.apache.hadoop.util.Tool;
import org.apache.hadoop.util.ToolRunner;
import example.avro.User;
public class MapReduceColorCount extends Configured implements Tool {
public static class ColorCountMapper extends
Mapper<AvroKey<User>, NullWritable, Text, IntWritable> {
@Override
public void map(AvroKey<User> key, NullWritable value, Context context)
throws IOException, InterruptedException {
CharSequence color = key.datum().getFavoriteColor();
if (color == null) {
color = "none";
}
context.write(new Text(color.toString()), new IntWritable(1));
}
}
public static class ColorCountReducer extends
Reducer<Text, IntWritable, AvroKey<CharSequence>, AvroValue<Integer>> {
@Override
public void reduce(Text key, Iterable<IntWritable> values,
Context context) throws IOException, InterruptedException {
int sum = 0;
for (IntWritable value : values) {
sum += value.get();
}
context.write(new AvroKey<CharSequence>(key.toString()), new AvroValue<Integer>(sum));
}
}
public int run(String[] args) throws Exception {
if (args.length != 2) {
System.err.println("Usage: MapReduceColorCount <input path> <output path>");
return -1;
}
Job job = new Job(getConf());
job.setJarByClass(MapReduceColorCount.class);
job.setJobName("Color Count");
FileInputFormat.setInputPaths(job, new Path(args[0]));
FileOutputFormat.setOutputPath(job, new Path(args[1]));
job.setInputFormatClass(AvroKeyInputFormat.class);
job.setMapperClass(ColorCountMapper.class);
AvroJob.setInputKeySchema(job, User.getClassSchema());
job.setMapOutputKeyClass(Text.class);
job.setMapOutputValueClass(IntWritable.class);
job.setOutputFormatClass(AvroKeyValueOutputFormat.class);
job.setReducerClass(ColorCountReducer.class);
AvroJob.setOutputKeySchema(job, Schema.create(Schema.Type.STRING));
AvroJob.setOutputValueSchema(job, Schema.create(Schema.Type.INT));
return (job.waitForCompletion(true) ? 0 : 1);
}
public static void main(String[] args) throws Exception {
int res = ToolRunner.run(new MapReduceColorCount(), args);
System.exit(res);
}
}
ColorCount reads in data files containing User records, defined in examples/user.avsc, and counts the number of instances of each favorite color. (This example draws inspiration from the canonical WordCount MapReduce application.) This example uses the old MapReduce API. See MapReduceAvroWordCount, found under doc/examples/mr-example/src/main/java/example/ to see the new MapReduce API example. The User schema is defined as follows:
{"namespace": "example.avro",
"type": "record",
"name": "User",
"fields": [
{"name": "name", "type": "string"},
{"name": "favorite_number", "type": ["int", "null"]},
{"name": "favorite_color", "type": ["string", "null"]}
]
}
This schema is compiled into the User class used by ColorCount via the Avro Maven plugin (see examples/mr-example/pom.xml for how this is set up).
ColorCountMapper essentially takes a User as input and extracts the User’s favorite color, emitting the key-value pair <favoriteColor, 1>. ColorCountReducer then adds up how many occurrences of a particular favorite color were emitted, and outputs the result as a Pair record. These Pairs are serialized to an Avro data file.
Running ColorCount
The ColorCount application is provided as a Maven project in the Avro docs under examples/mr-example. To build the project, including the code generation of the User schema, run:
Next, run GenerateData from examples/mr-examples to create an Avro data file, input/users.avro, containing 20 Users with favorite colors chosen randomly from a list:
mvn exec:java -q -Dexec.mainClass=example.GenerateData
Besides creating the data file, GenerateData prints the JSON representations of the Users generated to stdout, for example:
{"name": "user", "favorite_number": null, "favorite_color": "red"}
{"name": "user", "favorite_number": null, "favorite_color": "green"}
{"name": "user", "favorite_number": null, "favorite_color": "purple"}
{"name": "user", "favorite_number": null, "favorite_color": null}
...
Now we’re ready to run ColorCount. We specify our freshly-generated input folder as the input path and output as our output folder (note that MapReduce will not start a job if the output folder already exists):
mvn exec:java -q -Dexec.mainClass=example.MapredColorCount -Dexec.args="input output"
Once ColorCount completes, checking the contents of the new output directory should yield the following:
$ ls output/
part-00000.avro _SUCCESS
You can check the contents of the generated Avro file using the avro-tools jar:
$ java -jar /path/to/avro-tools-1.8.0.jar tojson output/part-00000.avro
{"value": 3, "key": "blue"}
{"value": 7, "key": "green"}
{"value": 1, "key": "none"}
{"value": 2, "key": "orange"}
{"value": 3, "key": "purple"}
{"value": 2, "key": "red"}
{"value": 2, "key": "yellow"}
Now let’s go over the ColorCount example in detail.
Mapper - org.apache.hadoop.mapred API
The easiest way to use Avro data files as input to a MapReduce job is to subclass AvroMapper. An AvroMapper defines a map function that takes an Avro datum as input and outputs a key/value pair represented as a Pair record. In the ColorCount example, ColorCountMapper is an AvroMapper that takes a User as input and outputs a Pair<CharSequence, Integer>>, where the CharSequence key is the user’s favorite color and the Integer value is 1.
public static class ColorCountMapper extends AvroMapper<User, Pair<CharSequence, Integer>> {
@Override
public void map(User user, AvroCollector<Pair<CharSequence, Integer>> collector, Reporter reporter)
throws IOException {
CharSequence color = user.getFavoriteColor();
// We need this check because the User.favorite_color field has type ["string", "null"]
if (color == null) {
color = "none";
}
collector.collect(new Pair<CharSequence, Integer>(color, 1));
}
}
In order to use our AvroMapper, we must call AvroJob.setMapperClass and AvroJob.setInputSchema.
AvroJob.setMapperClass(conf, ColorCountMapper.class);
AvroJob.setInputSchema(conf, User.getClassSchema());
Note that AvroMapper does not implement the Mapper interface. Under the hood, the specified Avro data files are deserialized into AvroWrappers containing the actual data, which are processed by a Mapper that calls the configured AvroMapper’s map function. AvroJob.setInputSchema sets up the relevant configuration parameters needed to make this happen, thus you should not need to call JobConf.setMapperClass, JobConf.setInputFormat, JobConf.setMapOutputKeyClass, JobConf.setMapOutputValueClass, or JobConf.setOutputKeyComparatorClass.
Mapper - org.apache.hadoop.mapreduce API
This document will not go into all the differences between the mapred and mapreduce APIs, however will describe the main differences. As you can see, ColorCountMapper is now a subclass of the Hadoop Mapper class and is passed an AvroKey as it’s key. Additionally, the AvroJob method calls were slightly changed.
public static class ColorCountMapper extends
Mapper<AvroKey<User>, NullWritable, Text, IntWritable> {
@Override
public void map(AvroKey<User> key, NullWritable value, Context context)
throws IOException, InterruptedException {
CharSequence color = key.datum().getFavoriteColor();
if (color == null) {
color = "none";
}
context.write(new Text(color.toString()), new IntWritable(1));
}
}
Reducer - org.apache.hadoop.mapred API
Analogously to AvroMapper, an AvroReducer defines a reducer function that takes the key/value types output by an AvroMapper (or any mapper that outputs Pairs) and outputs a key/value pair represented a Pair record. In the ColorCount example, ColorCountReducer is an AvroReducer that takes the CharSequence key representing a favorite color and the Iterable<Integer> representing the counts for that color (they should all be 1 in this example) and adds up the counts.
public static class ColorCountReducer extends AvroReducer<CharSequence, Integer,
Pair<CharSequence, Integer>> {
@Override
public void reduce(CharSequence key, Iterable<Integer> values,
AvroCollector<Pair<CharSequence, Integer>> collector,
Reporter reporter)
throws IOException {
int sum = 0;
for (Integer value : values) {
sum += value;
}
collector.collect(new Pair<CharSequence, Integer>(key, sum));
}
}
In order to use our AvroReducer, we must call AvroJob.setReducerClass and AvroJob.setOutputSchema.
AvroJob.setReducerClass(conf, ColorCountReducer.class);
AvroJob.setOutputSchema(conf, Pair.getPairSchema(Schema.create(Type.STRING),
Schema.create(Type.INT)));
Note that AvroReducer does not implement the Reducer interface. The intermediate Pairs output by the mapper are split into AvroKeys and AvroValues, which are processed by a Reducer that calls the configured AvroReducer’s reduce function. AvroJob.setOutputSchema sets up the relevant configuration parameters needed to make this happen, thus you should not need to call JobConf.setReducerClass, JobConf.setOutputFormat, JobConf.setOutputKeyClass, JobConf.setMapOutputKeyClass, JobConf.setMapOutputValueClass, or JobConf.setOutputKeyComparatorClass.
Reduce - org.apache.hadoop.mapreduce API
As before we not detail every difference between the APIs. As with the Mapper change ColorCountReducer is now a subclass of Reducer and AvroKey and AvroValue are emitted. Additionally, the AvroJob method calls were slightly changed.
public static class ColorCountReducer extends
Reducer<Text, IntWritable, AvroKey<CharSequence>, AvroValue<Integer>> {
@Override
public void reduce(Text key, Iterable<IntWritable> values,
Context context) throws IOException, InterruptedException {
int sum = 0;
for (IntWritable value : values) {
sum += value.get();
}
context.write(new AvroKey<CharSequence>(key.toString()), new AvroValue<Integer>(sum));
}
}
Learning more
The mapred API allows users to mix Avro AvroMappers and AvroReducers with non-Avro Mappers and Reducers and the mapreduce API allows users input Avro and output non-Avro or vice versa.
The mapred package has API org.apache.avro.mapred documentation as does the org.apache.avro.mapreduce package. MapReduce API (org.apache.hadoop.mapreduce). Similarily to the mapreduce package, it’s possible with the mapred API to implement your own Mappers and Reducers directly using the public classes provided in these libraries. See the AvroWordCount application, found under examples/mr-example/src/main/java/example/AvroWordCount.java in the Avro documentation, for an example of implementing a Reducer that outputs Avro data using the old MapReduce API. See the MapReduceAvroWordCount application, found under examples/mr-example/src/main/java/example/MapReduceAvroWordCount.java in the Avro documentation, for an example of implementing a Reducer that outputs Avro data using the new MapReduce API.
11.6 - IDL Language
Introduction
This document defines Avro IDL, a higher-level language for authoring Avro schemata. Before reading this document, you should have familiarity with the concepts of schemata and protocols, as well as the various primitive and complex types available in Avro.
Overview
Purpose
The aim of the Avro IDL language is to enable developers to author schemata in a way that feels more similar to common programming languages like Java, C++, or Python. Additionally, the Avro IDL language may feel more familiar for those users who have previously used the interface description languages (IDLs) in other frameworks like Thrift, Protocol Buffers, or CORBA.
Usage
Each Avro IDL file defines a single Avro Protocol, and thus generates as its output a JSON-format Avro Protocol file with extension .avpr.
To convert a .avdl file into a .avpr file, it may be processed by the idl tool. For example:
$ java -jar avroj-tools.jar idl src/test/idl/input/namespaces.avdl /tmp/namespaces.avpr
$ head /tmp/namespaces.avpr
{
"protocol" : "TestNamespace",
"namespace" : "avro.test.protocol",
The idl tool can also process input to and from stdin and stdout. See idl --help for full usage information.
A Maven plugin is also provided to compile .avdl files. To use it, add something like the following to your pom.xml:
<build>
<plugins>
<plugin>
<groupId>org.apache.avro</groupId>
<artifactId>avro-maven-plugin</artifactId>
<executions>
<execution>
<goals>
<goal>idl-protocol</goal>
</goals>
</execution>
</executions>
</plugin>
</plugins>
</build>
Defining a Protocol in Avro IDL
An Avro IDL file consists of exactly one protocol definition. The minimal protocol is defined by the following code:
This is equivalent to (and generates) the following JSON protocol definition:
{
"protocol" : "MyProtocol",
"types" : [ ],
"messages" : {
}
}
The namespace of the protocol may be changed using the @namespace annotation:
@namespace("mynamespace")
protocol MyProtocol {
}
This notation is used throughout Avro IDL as a way of specifying properties for the annotated element, as will be described later in this document.
Protocols in Avro IDL can contain the following items:
- Imports of external protocol and schema files.
- Definitions of named schemata, including records, errors, enums, and fixeds.
- Definitions of RPC messages
Imports
Files may be imported in one of three formats:
-
An IDL file may be imported with a statement like:
import idl "foo.avdl";
-
A JSON protocol file may be imported with a statement like:
import protocol "foo.avpr";
-
A JSON schema file may be imported with a statement like:
import schema "foo.avsc";
Messages and types in the imported file are added to this file’s protocol.
Imported file names are resolved relative to the current IDL file.
Defining an Enumeration
Enums are defined in Avro IDL using a syntax similar to C or Java.
enum Suit {
SPADES, DIAMONDS, CLUBS, HEARTS
}
Note that, unlike the JSON format, anonymous enums cannot be defined.
Defining a Fixed Length Field
Fixed fields are defined using the following syntax:
fixed MD5(16);
This example defines a fixed-length type called MD5 which contains 16 bytes.
Defining Records and Errors
Records are defined in Avro IDL using a syntax similar to a struct definition in C:
record Employee {
string name;
boolean active = true;
long salary;
}
The above example defines a record with the name “Employee” with three fields.
To define an error, simply use the keyword error instead of record. For example:
error Kaboom {
string explanation;
int result_code = -1;
}
Each field in a record or error consists of a type and a name, optional property annotations and an optional default value.
A type reference in Avro IDL must be one of:
- A primitive type
- A logical type
- A named schema defined prior to this usage in the same Protocol
- A complex type (array, map, or union)
Primitive Types
The primitive types supported by Avro IDL are the same as those supported by Avro’s JSON format. This list includes int, long, string, boolean, float, double, null, and bytes.
References to Named Schemata
If a named schema has already been defined in the same Avro IDL file, it may be referenced by name as if it were a primitive type:
record Card {
Suit suit; // refers to the enum Card defined above
int number;
}
Default Values
Default values for fields may be optionally specified by using an equals sign after the field name followed by a JSON expression indicating the default value. This JSON is interpreted as described in the spec.
Complex Types
Arrays
Array types are written in a manner that will seem familiar to C++ or Java programmers. An array of any type t is denoted array<t>. For example, an array of strings is denoted array<string>, and a multidimensional array of Foo records would be array<array<Foo>>.
Maps
Map types are written similarly to array types. An array that contains values of type t is written map<t>. As in the JSON schema format, all maps contain string-type keys.
Unions
Union types are denoted as union { typeA, typeB, typeC, … }. For example, this record contains a string field that is optional (unioned with null):
record RecordWithUnion {
union { null, string } optionalString;
}
Note that the same restrictions apply to Avro IDL unions as apply to unions defined in the JSON format; namely, a record may not contain multiple elements of the same type.
Defining RPC Messages
The syntax to define an RPC message within a Avro IDL protocol is similar to the syntax for a method declaration within a C header file or a Java interface. To define an RPC message add which takes two arguments named foo and bar, returning an int, simply include the following definition within the protocol:
int add(int foo, int bar = 0);
Message arguments, like record fields, may specify default values.
To define a message with no response, you may use the alias void, equivalent to the Avro null type:
void logMessage(string message);
If you have previously defined an error type within the same protocol, you may declare that a message can throw this error using the syntax:
void goKaboom() throws Kaboom;
To define a one-way message, use the keyword oneway after the parameter list, for example:
void fireAndForget(string message) oneway;
Other Language Features
All Java-style comments are supported within a Avro IDL file. Any text following // on a line is ignored, as is any text between /* and */, possibly spanning multiple lines.
Comments that begin with /** are used as the documentation string for the type or field definition that follows the comment.
Escaping Identifiers
Occasionally, one will need to use a reserved language keyword as an identifier. In order to do so, backticks (`) may be used to escape the identifier. For example, to define a message with the literal name error, you may write:
This syntax is allowed anywhere an identifier is expected.
Annotations for Ordering and Namespaces
Java-style annotations may be used to add additional properties to types and fields throughout Avro IDL.
For example, to specify the sort order of a field within a record, one may use the @order annotation before the field name as follows:
record MyRecord {
string @order("ascending") myAscendingSortField;
string @order("descending") myDescendingField;
string @order("ignore") myIgnoredField;
}
A field’s type may also be preceded by annotations, e.g.:
record MyRecord {
@java-class("java.util.ArrayList") array<string> myStrings;
}
This can be used to support java classes that can be serialized/deserialized via their toString/String constructor, e.g.:
record MyRecord {
@java-class("java.math.BigDecimal") string value;
@java-key-class("java.io.File") map<string> fileStates;
array<@java-class("java.math.BigDecimal") string> weights;
}
Similarly, a @namespace annotation may be used to modify the namespace when defining a named schema. For example:
@namespace("org.apache.avro.firstNamespace")
protocol MyProto {
@namespace("org.apache.avro.someOtherNamespace")
record Foo {}
record Bar {}
}
will define a protocol in the firstNamespace namespace. The record Foo will be defined in someOtherNamespace and Bar will be defined in firstNamespace as it inherits its default from its container.
Type and field aliases are specified with the @aliases annotation as follows:
@aliases(["org.old.OldRecord", "org.ancient.AncientRecord"])
record MyRecord {
string @aliases(["oldField", "ancientField"]) myNewField;
}
Some annotations like those listed above are handled specially. All other annotations are added as properties to the protocol, message, schema or field.
Complete Example
The following is a complete example of a Avro IDL file that shows most of the above features:
/**
* An example protocol in Avro IDL
*/
@namespace("org.apache.avro.test")
protocol Simple {
@aliases(["org.foo.KindOf"])
enum Kind {
FOO,
BAR, // the bar enum value
BAZ
}
fixed MD5(16);
record TestRecord {
@order("ignore")
string name;
@order("descending")
Kind kind;
MD5 hash;
union { MD5, null} @aliases(["hash"]) nullableHash;
array<long> arrayOfLongs;
}
error TestError {
string message;
}
string hello(string greeting);
TestRecord echo(TestRecord `record`);
int add(int arg1, int arg2);
bytes echoBytes(bytes data);
void `error`() throws TestError;
void ping() oneway;
}
Additional examples may be found in the Avro source tree under the src/test/idl/input directory.
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
11.7 - SASL profile
Introduction
SASL (RFC 2222) provides a framework for authentication and security of network protocols. Each protocol that uses SASL is meant to define a SASL profile. This document provides a SASL profile for connection-based Avro RPC.
Overview
SASL negotiation proceeds as a series of message interactions over a connection between a client and server using a selected SASL mechanism. The client starts this negotiation by sending its chosen mechanism name with an initial (possibly empty) message. Negotiation proceeds with the exchange of messages until either side indicates success or failure. The content of the messages is mechanism-specific. If the negotiation succeeds, then the session can proceed over the connection, otherwise it must be abandoned.
Some mechanisms continue to process session data after negotiation (e.g., encrypting it), while some specify that further session data is transmitted unmodified.
Negotiation
Commands
Avro SASL negotiation uses four one-byte commands.
- 0: START Used in a client’s initial message.
- 1: CONTINUE Used while negotiation is ongoing.
- 2: FAIL Terminates negotiation unsuccessfully.
- 3: COMPLETE Terminates negotiation successfully.
The format of a START message is:
| 0 | 4-byte mechanism name length | mechanism name | 4-byte payload length | payload data |
The format of a CONTINUE message is:
| 1 | 4-byte payload length | payload data |
The format of a FAIL message is:
| 2 | 4-byte message length | UTF-8 message |
The format of a COMPLETE message is:
| 3 | 4-byte payload length | payload data |
Process
Negotiation is initiated by a client sending a START command containing the client’s chosen mechanism name and any mechanism-specific payload data.
The server and client then interchange some number (possibly zero) of CONTINUE messages. Each message contains payload data that is processed by the security mechanism to generate the next message.
Once either the client or server send a FAIL message then negotiation has failed. UTF-8-encoded text is included in the failure message. Once either a FAIL message has been sent or received, or any other error occurs in the negotiation, further communication on this connection must cease.
Once either the client or server send a COMPLETE message then negotiation has completed successfully. Session data may now be transmitted over the connection until it is closed by either side.
Session Data
If no SASL QOP (quality of protection) is negotiated, then all subsequent writes to/reads over this connection are written/read unmodified. In particular, messages use Avro framing, and are of the form:
| 4-byte frame length | frame data | ... | 4 zero bytes |
If a SASL QOP is negotiated, then it must be used by the connection for all subsequent messages. This is done by wrapping each non-empty frame written using the security mechanism and unwrapping each non-empty frame read. The length written in each non-empty frame is the length of the wrapped data. Complete frames must be passed to the security mechanism for unwrapping. Unwrapped data is then passed to the application as the content of the frame.
If at any point processing fails due to wrapping, unwrapping or framing errors, then all further communication on this connection must cease.
Anonymous Mechanism
The SASL anonymous mechanism (RFC 2245) is quite simple to implement. In particular, an initial anonymous request may be prefixed by the following static sequence:
| 0 | 0009 | ANONYMOUS | 0000 |
If a server uses the anonymous mechanism, it should check that the mechanism name in the start message prefixing the first request received is ‘ANONYMOUS’, then simply prefix its initial response with a COMPLETE message of:
| 3 | 0000 |
If an anonymous server recieves some other mechanism name, then it may respond with a FAIL message as simple as:
| 2 | 0000 |
Note that the anonymous mechanism need add no additional round-trip messages between client and server. The START message can be piggybacked on the initial request and the COMPLETE or FAIL message can be piggybacked on the initial response.
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
11.8 - Wiki
The Wiki page can be found here.
11.9 - FAQ
What is Avro?
Avro is a data serialization system.
Avro provides:
- Rich data structures.
- A compact, fast, binary data format.
- A container file, to store persistent data.
- Remote procedure call (RPC).
- Simple integration with dynamic languages. Code generation is not required to read or write data files nor to use or implement RPC protocols. Code generation as an optional optimization, only worth implementing for statically typed languages.
How can I get started quickly with Avro?
Check out the Quick Start Guide.
How do I statically compile a schema or protocol into generated code?
In Java
- Add the avro jar, the jackson-mapper-asl.jar and jackson-core-asl.jar to your CLASSPATH.
- Run java org.apache.avro.specific.SpecificCompiler .
This appears to be out of date, the SpecificCompiler requires two arguments, presumably an input and and output file, but it isn’t clear that this does.
Or use the Schema or Protocol Ant tasks. Avro’s build.xml provides examples of how these are used.
Lastly, you can also use the “avro-tools” jar which ships with an Avro release. Just use the “compile (schema|protocol)” command.
How are Strings represented in Java?
They use org.apache.avro.util.Utf8, not java.lang.String.
More generally, how do Avro types map to Java types?
The mappings are documented in the package javadoc for generic, specific and reflect API.
From Doug Cutting:
HDFS splits files into blocks, and mapreduce runs a map task for each block. When the task starts, it needs to be able to seek into the file to the start of the block process through the block’s end. If the file were, e.g., a gzip file, this would not be possible, since gzip files must be decompressed from the start. One cannot seek into the middle of a gzip file and start decompressing. So Hadoop’s SequenceFile places a marker periodically (~64k) in the file at record and compression boundaries, where processing can be sensibly started. Then, when a map task starts processing an HDFS block, it finds the first marker after the block’s start and continues through the first marker in the next block of the file. This requires a bit of non-local access (~0.1%). Avro’s data file uses the same method as SequenceFile.
Why isn’t every value in Avro nullable?
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 of every value stored and
corresponding computational costs to create this bit on write and
interpret it on read. These costs are wasted when values may not in
fact be null, as is the case in many datasets. In Avro such costs are
only paid when values may actually be null.
Also, allowing values to be null is a well-known source of errors. In
Avro, a value declared as non-null will always be non-null and programs
need not test for null values when processing it nor will they ever fail
for lack of such tests.
Tony Hoare calls his invention of null references his “Billion Dollar
Mistake”.
http://qconlondon.com/london-2009/presentation/Null+References:+The+Billion+Dollar+Mistake
Also note that in some programming languages not all values are permitted to be null. For example, in Java, values of type boolean, byte, short, char, int, float, long, and double may not be null.
How can I serialize directly to/from a byte array?
As pointed out in the specification, Avro data should always be stored with its schema. The Avro provided classes DataFileWriter, DataFileReader, and DataFileStream all ensure this by serializing the Schema in a container header. In some special cases, such as when implementing a new storage system or writing unit tests, you may need to write and read directly with the bare Avro serialized values. The following examples use code generated for Java from an Avro Schema, specifically the User example from the Quickstart guide.
Serializing to a byte array
This example takes a User object and returns a newly allocated byte array with the Avro serialization of that user.
ByteArrayOutputStream out = new ByteArrayOutputStream();
BinaryEncoder encoder = EncoderFactory.get().binaryEncoder(out, null);
DatumWriter<User> writer = new SpecificDatumWriter<User>(User.getClassSchema());
writer.write(user, encoder);
encoder.flush();
out.close();
byte[] serializedBytes = out.toByteArray();
Deserializing from a byte array
This example takes a byte array containing the Avro serialization of a user and returns a User object.SpecificDatumReader reader = new SpecificDatumReader(User.getClassSchema());
Decoder decoder = DecoderFactory.get().binaryDecoder(bytes, null);
User user = reader.read(null, decoder);
Note: If you are serializing or deserializing in a loop or as a method, you should be reusing objects, readers and/or writers. Check the JavaDoc to see how to reuse the objects.
12 - Apache Avro™ 1.7.7 Documentation
Introduction
Apache Avro™ is a data serialization system.
Avro provides:
- Rich data structures.
- A compact, fast, binary data format.
- A container file, to store persistent data.
- Remote procedure call (RPC).
- Simple integration with dynamic languages. Code generation is not required to read or write data files nor to use or implement RPC protocols. Code generation as an optional optimization, only worth implementing for statically typed languages.
Schemas
Avro relies on schemas. When Avro data is read, the schema used when writing it is always present. This permits each datum to be written with no per-value overheads, making serialization both fast and small. This also facilitates use with dynamic, scripting languages, since data, together with its schema, is fully self-describing.
When Avro data is stored in a file, its schema is stored with it, so that files may be processed later by any program. If the program reading the data expects a different schema this can be easily resolved, since both schemas are present.
When Avro is used in RPC, the client and server exchange schemas in the connection handshake. (This can be optimized so that, for most calls, no schemas are actually transmitted.) Since both client and server both have the other’s full schema, correspondence between same named fields, missing fields, extra fields, etc. can all be easily resolved.
Avro schemas are defined with JSON . This facilitates implementation in languages that already have JSON libraries.
Comparison with other systems
Avro provides functionality similar to systems such as Thrift, Protocol Buffers, etc. Avro differs from these systems in the following fundamental aspects.
- Dynamic typing: Avro does not require that code be generated. Data is always accompanied by a schema that permits full processing of that data without code generation, static datatypes, etc. This facilitates construction of generic data-processing systems and languages.
- Untagged data: Since the schema is present when data is read, considerably less type information need be encoded with data, resulting in smaller serialization size.
- No manually-assigned field IDs: When a schema changes, both the old and new schema are always present when processing data, so differences may be resolved symbolically, using field names.
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
12.1 - Getting Started (Java)
This is a short guide for getting started with Apache Avro™ using Java. This guide only covers using Avro for data serialization; see Patrick Hunt’s Avro RPC Quick Start for a good introduction to using Avro for RPC.
Download
Avro implementations for C, C++, C#, Java, PHP, Python, and Ruby can be downloaded from the Apache Avro™ Releases page. This guide uses Avro 1.7.7, the latest version at the time of writing. For the examples in this guide, download avro-1.7.7.jar and avro-tools-1.7.7.jar. The Avro Java implementation also depends on the Jackson JSON library. From the Jackson download page, download the core-asl and mapper-asl jars. Add avro-1.7.7.jar and the Jackson jars to your project’s classpath (avro-tools will be used for code generation).
Alternatively, if you are using Maven, add the following dependency to your POM:
<dependency>
<groupId>org.apache.avro</groupId>
<artifactId>avro</artifactId>
<version>1.7.7</version>
</dependency>
As well as the Avro Maven plugin (for performing code generation):
<plugin>
<groupId>org.apache.avro</groupId>
<artifactId>avro-maven-plugin</artifactId>
<version>1.7.7</version>
<executions>
<execution>
<phase>generate-sources</phase>
<goals>
<goal>schema</goal>
</goals>
<configuration>
<sourceDirectory>${project.basedir}/src/main/avro/</sourceDirectory>
<outputDirectory>${project.basedir}/src/main/java/</outputDirectory>
</configuration>
</execution>
</executions>
</plugin>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-compiler-plugin</artifactId>
<configuration>
<source>1.6</source>
<target>1.6</target>
</configuration>
</plugin>
You may also build the required Avro jars from source. Building Avro is beyond the scope of this guide; see the Build Documentation page in the wiki for more information.
Defining a schema
Avro schemas are defined using JSON. Schemas are composed of primitive types (null, boolean, int, long, float, double, bytes, and string) and complex types (record, enum, array, map, union, and fixed). You can learn more about Avro schemas and types from the specification, but for now let’s start with a simple schema example, user.avsc:
{"namespace": "example.avro",
"type": "record",
"name": "User",
"fields": [
{"name": "name", "type": "string"},
{"name": "favorite_number", "type": ["int", "null"]},
{"name": "favorite_color", "type": ["string", "null"]}
]
}
This schema defines a record representing a hypothetical user. (Note that a schema file can only contain a single schema definition.) At minimum, a record definition must include its type (“type”: “record”), a name (“name”: “User”), and fields, in this case name, favorite_number, and favorite_color. We also define a namespace (“namespace”: “example.avro”), which together with the name attribute defines the “full name” of the schema (example.avro.User in this case).
Fields are defined via an array of objects, each of which defines a name and type (other attributes are optional, see the record specification for more details). The type attribute of a field is another schema object, which can be either a primitive or complex type. For example, the name field of our User schema is the primitive type string, whereas the favorite_number and favorite_color fields are both unions, represented by JSON arrays. unions are a complex type that can be any of the types listed in the array; e.g., favorite_number can either be an int or null, essentially making it an optional field.
Serializing and deserializing with code generation
Compiling the schema
Code generation allows us to automatically create classes based on our previously-defined schema. Once we have defined the relevant classes, there is no need to use the schema directly in our programs. We use the avro-tools jar to generate code as follows:
java -jar /path/to/avro-tools-1.7.7.jar compile schema <schema file> <destination>
This will generate the appropriate source files in a package based on the schema’s namespace in the provided destination folder. For instance, to generate a User class in package example.avro from the schema defined above, run
java -jar /path/to/avro-tools-1.7.7.jar compile schema user.avsc .
Note that if you using the Avro Maven plugin, there is no need to manually invoke the schema compiler; the plugin automatically performs code generation on any .avsc files present in the configured source directory.
Creating Users
Now that we’ve completed the code generation, let’s create some Users, serialize them to a data file on disk, and then read back the file and deserialize the User objects.
First let’s create some Users and set their fields.
User user1 = new User();
user1.setName("Alyssa");
user1.setFavoriteNumber(256);
// Leave favorite color null
// Alternate constructor
User user2 = new User("Ben", 7, "red");
// Construct via builder
User user3 = User.newBuilder()
.setName("Charlie")
.setFavoriteColor("blue")
.setFavoriteNumber(null)
.build();
As shown in this example, Avro objects can be created either by invoking a constructor directly or by using a builder. Unlike constructors, builders will automatically set any default values specified in the schema. Additionally, builders validate the data as it set, whereas objects constructed directly will not cause an error until the object is serialized. However, using constructors directly generally offers better performance, as builders create a copy of the datastructure before it is written.
Note that we do not set user1’s favorite color. Since that record is of type [“string”, “null”], we can either set it to a string or leave it null; it is essentially optional. Similarly, we set user3’s favorite number to null (using a builder requires setting all fields, even if they are null).
Serializing
Now let’s serialize our Users to disk.
/// Serialize user1, user2 and user3 to disk
DatumWriter<User> userDatumWriter = new SpecificDatumWriter<User>(User.class);
DataFileWriter<User> dataFileWriter = new DataFileWriter<User>(userDatumWriter);
dataFileWriter.create(user1.getSchema(), new File("users.avro"));
dataFileWriter.append(user1);
dataFileWriter.append(user2);
dataFileWriter.append(user3);
dataFileWriter.close();
We create a DatumWriter, which converts Java objects into an in-memory serialized format. The SpecificDatumWriter class is used with generated classes and extracts the schema from the specified generated type.
Next we create a DataFileWriter, which writes the serialized records, as well as the schema, to the file specified in the dataFileWriter.create call. We write our users to the file via calls to the dataFileWriter.append method. When we are done writing, we close the data file.
Deserializing
Finally, let’s deserialize the data file we just created.
// Deserialize Users from disk
DatumReader<User> userDatumReader = new SpecificDatumReader<User>(User.class);
DataFileReader<User> dataFileReader = new DataFileReader<User>(file, userDatumReader);
User user = null;
while (dataFileReader.hasNext()) {
// Reuse user object by passing it to next(). This saves us from
// allocating and garbage collecting many objects for files with
// many items.
user = dataFileReader.next(user);
System.out.println(user);
}
This snippet will output:
{"name": "Alyssa", "favorite_number": 256, "favorite_color": null}
{"name": "Ben", "favorite_number": 7, "favorite_color": "red"}
{"name": "Charlie", "favorite_number": null, "favorite_color": "blue"}
Deserializing is very similar to serializing. We create a SpecificDatumReader, analogous to the SpecificDatumWriter we used in serialization, which converts in-memory serialized items into instances of our generated class, in this case User. We pass the DatumReader and the previously created File to a DataFileReader, analogous to the DataFileWriter, which reads the data file on disk.
Next we use the DataFileReader to iterate through the serialized Users and print the deserialized object to stdout. Note how we perform the iteration: we create a single User object which we store the current deserialized user in, and pass this record object to every call of dataFileReader.next. This is a performance optimization that allows the DataFileReader to reuse the same User object rather than allocating a new User for every iteration, which can be very expensive in terms of object allocation and garbage collection if we deserialize a large data file. While this technique is the standard way to iterate through a data file, it’s also possible to use for (User user : dataFileReader) if performance is not a concern.
Compiling and running the example code
This example code is included as a Maven project in the examples/java-example directory in the Avro docs. From this directory, execute the following commands to build and run the example:
$ mvn compile # includes code generation via Avro Maven plugin
$ mvn -q exec:java -Dexec.mainClass=example.SpecificMain
Serializing and deserializing without code generation
Data in Avro is always stored with its corresponding schema, meaning we can always read a serialized item regardless of whether we know the schema ahead of time. This allows us to perform serialization and deserialization without code generation.
Let’s go over the same example as in the previous section, but without using code generation: we’ll create some users, serialize them to a data file on disk, and then read back the file and deserialize the users objects.
Creating users
First, we use a Parser to read our schema definition and create a Schema object.
Schema schema = new Schema.Parser().parse(new File("user.avsc"));
Using this schema, let’s create some users.
GenericRecord user1 = new GenericData.Record(schema);
user1.put("name", "Alyssa");
user1.put("favorite_number", 256);
// Leave favorite color null
GenericRecord user2 = new GenericData.Record(schema);
user2.put("name", "Ben");
user2.put("favorite_number", 7);
user2.put("favorite_color", "red");
Since we’re not using code generation, we use GenericRecords to represent users. GenericRecord uses the schema to verify that we only specify valid fields. If we try to set a non-existent field (e.g., user1.put(“favorite_animal”, “cat”)), we’ll get an AvroRuntimeException when we run the program.
Note that we do not set user1’s favorite color. Since that record is of type [“string”, “null”], we can either set it to a string or leave it null; it is essentially optional.
Serializing
Now that we’ve created our user objects, serializing and deserializing them is almost identical to the example above which uses code generation. The main difference is that we use generic instead of specific readers and writers.
First we’ll serialize our users to a data file on disk.
// Serialize user1 and user2 to disk
File file = new File("users.avro");
DatumWriter<GenericRecord> datumWriter = new GenericDatumWriter<GenericRecord>(schema);
DataFileWriter<GenericRecord> dataFileWriter = new DataFileWriter<GenericRecord>(datumWriter);
dataFileWriter.create(schema, file);
dataFileWriter.append(user1);
dataFileWriter.append(user2);
dataFileWriter.close();
We create a DatumWriter, which converts Java objects into an in-memory serialized format. Since we are not using code generation, we create a GenericDatumWriter. It requires the schema both to determine how to write the GenericRecords and to verify that all non-nullable fields are present.
As in the code generation example, we also create a DataFileWriter, which writes the serialized records, as well as the schema, to the file specified in the dataFileWriter.create call. We write our users to the file via calls to the dataFileWriter.append method. When we are done writing, we close the data file.
Deserializing
Finally, we’ll deserialize the data file we just created.
// Deserialize users from disk
DatumReader<GenericRecord> datumReader = new GenericDatumReader<GenericRecord>(schema);
DataFileReader<GenericRecord> dataFileReader = new DataFileReader<GenericRecord>(file, datumReader);
GenericRecord user = null;
while (dataFileReader.hasNext()) {
// Reuse user object by passing it to next(). This saves us from
// allocating and garbage collecting many objects for files with
// many items.
user = dataFileReader.next(user);
System.out.println(user);
This outputs:
{"name": "Alyssa", "favorite_number": 256, "favorite_color": null}
{"name": "Ben", "favorite_number": 7, "favorite_color": "red"}
Deserializing is very similar to serializing. We create a GenericDatumReader, analogous to the GenericDatumWriter we used in serialization, which converts in-memory serialized items into GenericRecords. We pass the DatumReader and the previously created File to a DataFileReader, analogous to the DataFileWriter, which reads the data file on disk.
Next, we use the DataFileReader to iterate through the serialized users and print the deserialized object to stdout. Note how we perform the iteration: we create a single GenericRecord object which we store the current deserialized user in, and pass this record object to every call of dataFileReader.next. This is a performance optimization that allows the DataFileReader to reuse the same record object rather than allocating a new GenericRecord for every iteration, which can be very expensive in terms of object allocation and garbage collection if we deserialize a large data file. While this technique is the standard way to iterate through a data file, it’s also possible to use for (GenericRecord user : dataFileReader) if performance is not a concern.
Compiling and running the example code
This example code is included as a Maven project in the examples/java-example directory in the Avro docs. From this directory, execute the following commands to build and run the example:
$ mvn compile
$ mvn -q exec:java -Dexec.mainClass=example.GenericMain
12.2 - Getting Started (Python)
This is a short guide for getting started with Apache Avro™ using Python. This guide only covers using Avro for data serialization; see Patrick Hunt’s Avro RPC Quick Start for a good introduction to using Avro for RPC.
Download
Avro implementations for C, C++, C#, Java, PHP, Python, and Ruby can be downloaded from the Apache Avro™ Releases page. This guide uses Avro 1.7.7, the latest version at the time of writing. Download and unzip avro-1.7.7.tar.gz, and install via python setup.py (this will probably require root privileges). Ensure that you can import avro from a Python prompt.
$ tar xvf avro-1.7.7.tar.gz
$ cd avro-1.7.7
$ sudo python setup.py install
$ python
>>> import avro # should not raise ImportError
Alternatively, you may build the Avro Python library from source. From your the root Avro directory, run the commands
$ cd lang/py/
$ ant
$ sudo python setup.py install
$ python
>>> import avro # should not raise ImportError
Defining a schema
Avro schemas are defined using JSON. Schemas are composed of primitive types (null, boolean, int, long, float, double, bytes, and string) and complex types (record, enum, array, map, union, and fixed). You can learn more about Avro schemas and types from the specification, but for now let’s start with a simple schema example, user.avsc:
{"namespace": "example.avro",
"type": "record",
"name": "User",
"fields": [
{"name": "name", "type": "string"},
{"name": "favorite_number", "type": ["int", "null"]},
{"name": "favorite_color", "type": ["string", "null"]}
]
}
This schema defines a record representing a hypothetical user. (Note that a schema file can only contain a single schema definition.) At minimum, a record definition must include its type (“type”: “record”), a name (“name”: “User”), and fields, in this case name, favorite_number, and favorite_color. We also define a namespace (“namespace”: “example.avro”), which together with the name attribute defines the “full name” of the schema (example.avro.User in this case).
Fields are defined via an array of objects, each of which defines a name and type (other attributes are optional, see the record specification for more details). The type attribute of a field is another schema object, which can be either a primitive or complex type. For example, the name field of our User schema is the primitive type string, whereas the favorite_number and favorite_color fields are both unions, represented by JSON arrays. unions are a complex type that can be any of the types listed in the array; e.g., favorite_number can either be an int or null, essentially making it an optional field.
Serializing and deserializing without code generation
Data in Avro is always stored with its corresponding schema, meaning we can always read a serialized item, regardless of whether we know the schema ahead of time. This allows us to perform serialization and deserialization without code generation. Note that the Avro Python library does not support code generation.
Try running the following code snippet, which serializes two users to a data file on disk, and then reads back and deserializes the data file:
import avro.schema
from avro.datafile import DataFileReader, DataFileWriter
from avro.io import DatumReader, DatumWriter
schema = avro.schema.parse(open("user.avsc", "rb").read())
writer = DataFileWriter(open("users.avro", "wb"), DatumWriter(), schema)
writer.append({"name": "Alyssa", "favorite_number": 256})
writer.append({"name": "Ben", "favorite_number": 7, "favorite_color": "red"})
writer.close()
reader = DataFileReader(open("users.avro", "rb"), DatumReader())
for user in reader:
print user
reader.close()
This outputs:
{u'favorite_color': None, u'favorite_number': 256, u'name': u'Alyssa'}
{u'favorite_color': u'red', u'favorite_number': 7, u'name': u'Ben'}
Let’s take a closer look at what’s going on here.
schema = avro.schema.parse(open("user.avsc", "rb").read())
avro.schema.parse takes a string containing a JSON schema definition as input and outputs a avro.schema.Schema object (specifically a subclass of Schema, in this case RecordSchema). We’re passing in the contents of our user.avsc schema file here.
writer = DataFileWriter(open("users.avro", "w"), DatumWriter(), schema)
We create a DataFileWriter, which we’ll use to write serialized items to a data file on disk. The DataFileWriter constructor takes three arguments:
- The file we’ll serialize to
- A DatumWriter, which is responsible for actually serializing the items to Avro’s binary format (DatumWriters can be used separately from DataFileWriters, e.g., to perform IPC with Avro TODO: is this true??).
- The schema we’re using. The DataFileWriter needs the schema both to write the schema to the data file, and to verify that the items we write are valid items and write the appropriate fields.
writer.append({"name": "Alyssa", "favorite_number": 256})
writer.append({"name": "Ben", "favorite_number": 7, "favorite_color": "red"})
We use DataFileWriter.append to add items to our data file. Avro records are represented as Python dicts. Since the field favorite_color has type [“int”, “null”], we are not required to specify this field, as shown in the first append. Were we to omit the required name field, an exception would be raised. Any extra entries not corresponding to a field are present in the dict are ignored.
reader = DataFileReader(open("users.avro", "r"), DatumReader())
We open the file again, this time for reading back from disk. We use a DataFileReader and DatumReader analagous to the DataFileWriter and DatumWriter above.
for user in reader:
print user
The DataFileReader is an iterator that returns dicts corresponding to the serialized items.
12.3 - Specification
Introduction
This document defines Apache Avro. It is intended to be the authoritative specification. Implementations of Avro must adhere to this document.
Schema Declaration
A Schema is represented in JSON by one of:
- A JSON string, naming a defined type.
- A JSON object, of the form:
{"type": "typeName" ...attributes...}
where typeName is either a primitive or derived type name, as defined below. Attributes not defined in this document are permitted as metadata, but must not affect the format of serialized data.
- A JSON array, representing a union of embedded types.
Primitive Types
The set of primitive type names is:
- null: no value
- boolean: a binary value
- int: 32-bit signed integer
- long: 64-bit signed integer
- float: single precision (32-bit) IEEE 754 floating-point number
- double: double precision (64-bit) IEEE 754 floating-point number
- bytes: sequence of 8-bit unsigned bytes
- string: unicode character sequence
Primitive types have no specified attributes.
Primitive type names are also defined type names. Thus, for example, the schema “string” is equivalent to:
Complex Types
Avro supports six kinds of complex types: records, enums, arrays, maps, unions and fixed.
Records
Records use the type name “record” and support three attributes:
- name: a JSON string providing the name of the record (required).
- namespace, a JSON string that qualifies the name;
- doc: a JSON string providing documentation to the user of this schema (optional).
- aliases: a JSON array of strings, providing alternate names for this record (optional).
- fields: a JSON array, listing fields (required). Each field is a JSON object with the following attributes:
- name: a JSON string providing the name of the field (required), and
- doc: a JSON string describing this field for users (optional).
- type: A JSON object defining a schema, or a JSON string naming a record definition (required).
- _default: A default value for this field, used when reading instances that lack this field (optional). 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. Default values for bytes and fixed fields are JSON strings, where Unicode code points 0-255 are mapped to unsigned 8-bit byte values 0-255.
field default values
| avro type |
json type |
example |
| null |
null |
null |
| boolean |
boolean |
true |
| int,long |
integer |
1 |
| float,double |
number |
1.1 |
| bytes |
string |
"\u00FF" |
| string |
string |
"foo" |
| record |
object |
{"a": 1} |
| enum |
string |
"FOO" |
| array |
array |
[1] |
| map |
object |
{"a": 1} |
| fixed |
string |
"\u00ff" |
- order: specifies how this field impacts sort ordering of this record (optional). Valid values are “ascending” (the default), “descending”, or “ignore”. For more details on how this is used, see the sort order section below.
- aliases: a JSON array of strings, providing alternate names for this field (optional).
For example, a linked-list of 64-bit values may be defined with:
{
"type": "record",
"name": "LongList",
"aliases": ["LinkedLongs"], // old name for this
"fields" : [
{"name": "value", "type": "long"}, // each element has a long
{"name": "next", "type": ["null", "LongList"]} // optional next element
]
}
Enums
Enums use the type name “enum” and support the following attributes:
- name: a JSON string providing the name of the enum (required).
- namespace, a JSON string that qualifies the name ;
- aliases: a JSON array of strings, providing alternate names for this enum (optional).
- doc: a JSON string providing documentation to the user of this schema (optional).
- symbols: a JSON array, listing symbols, as JSON strings (required). All symbols in an enum must be unique; duplicates are prohibited.
For example, playing card suits might be defined with:
{
"type": "enum",
"name": "Suit",
"symbols" : ["SPADES", "HEARTS", "DIAMONDS", "CLUBS"]
}
Arrays
Arrays use the type name “array” and support a single attribute:
- items: the schema of the array’s items.
For example, an array of strings is declared with:
{
"type": "array",
"items" : "string"
}
Maps
Maps use the type name “map” and support one attribute:
- values: the schema of the map’s values.
Map keys are assumed to be strings.
For example, a map from string to long is declared with:
{
"type": "map",
"values" : "long"
}
Unions
Unions, as mentioned above, are represented using JSON arrays. For example, ["null", "string"] declares a schema which may be either a null or string.
(Note that when a default value is specified for a record field whose type is a union, the type of the default value must match the first element of the union. Thus, for unions containing “null”, the “null” is usually listed first, since the default value of such unions is typically null.)
Unions may not contain more than one schema with the same type, except for the named types record, fixed and enum. For example, unions containing two array types or two map types are not permitted, but two types with different names are permitted. (Names permit efficient resolution when reading and writing unions.)
Unions may not immediately contain other unions.
Fixed
Fixed uses the type name “fixed” and supports two attributes:
- name: a string naming this fixed (required).
- namespace, a string that qualifies the name;
- aliases: a JSON array of strings, providing alternate names for this enum (optional).
- size: an integer, specifying the number of bytes per value (required).
For example, 16-byte quantity may be declared with:
{"type": "fixed", "size": 16, "name": "md5"}
Names
Record, enums and fixed are named types. Each has a fullname that is composed of two parts; a name and a namespace. Equality of names is defined on the fullname.
The name portion of a fullname, record field names, and enum symbols must:
- start with [A-Za-z_]
- subsequently contain only [A-Za-z0-9_]
A namespace is a dot-separated sequence of such names. The empty string may also be used as a namespace to indicate the null namespace. Equality of names (including field names and enum symbols) as well as fullnames is case-sensitive.
In record, enum and fixed definitions, the fullname is determined in one of the following ways:
A name and namespace are both specified. For example, one might use “name”: “X”, “namespace”: “org.foo” to indicate the fullname org.foo.X.
-
A fullname is specified. If the name specified contains a dot, then it is assumed to be a fullname, and any namespace also specified is ignored. For example, use “name”: “org.foo.X” to indicate the fullname org.foo.X.
-
A name only is specified, i.e., a name that contains no dots. In this case the namespace is taken from the most tightly enclosing schema or protocol. For example, if “name”: “X” is specified, and this occurs within a field of the record definition of org.foo.Y, then the fullname is org.foo.X. If there is no enclosing namespace then the null namespace is used.
References to previously defined names are as in the latter two cases above: if they contain a dot they are a fullname, if they do not contain a dot, the namespace is the namespace of the enclosing definition.
Primitive type names have no namespace and their names may not be defined in any namespace.
A schema or protocol may not contain multiple definitions of a fullname. Further, a name must be defined before it is used (“before” in the depth-first, left-to-right traversal of the JSON parse tree, where the types attribute of a protocol is always deemed to come “before” the messages attribute.)
Aliases
Named types and fields may have aliases. An implementation may optionally use aliases to map a writer’s schema to the reader’s. This facilitates both schema evolution as well as processing disparate datasets.
Aliases function by re-writing the writer’s schema using aliases from the reader’s schema. For example, if the writer’s schema was named “Foo” and the reader’s schema is named “Bar” and has an alias of “Foo”, then the implementation would act as though “Foo” were named “Bar” when reading. Similarly, if data was written as a record with a field named “x” and is read as a record with a field named “y” with alias “x”, then the implementation would act as though “x” were named “y” when reading.
A type alias may be specified either as a fully namespace-qualified, or relative to the namespace of the name it is an alias for. For example, if a type named “a.b” has aliases of “c” and “x.y”, then the fully qualified names of its aliases are “a.c” and “x.y”.
Data Serialization
Avro data is always serialized with its schema. Files that store Avro data should always also include the schema for that data in the same file. Avro-based remote procedure call (RPC) systems must also guarantee that remote recipients of data have a copy of the schema used to write that data.
Because the schema used to write data is always available when the data is read, Avro data itself is not tagged with type information. The schema is required to parse data.
In general, both serialization and deserialization proceed as a depth-first, left-to-right traversal of the schema, serializing primitive types as they are encountered.
Encodings
Avro specifies two serialization encodings: binary and JSON. Most applications will use the binary encoding, as it is smaller and faster. But, for debugging and web-based applications, the JSON encoding may sometimes be appropriate.
Binary Encoding
Primitive Types
Primitive types are encoded in binary as follows:
- null is written as zero bytes.
- a boolean is written as a single byte whose value is either 0 (false) or 1 (true).
- int and long values are written using variable-length zig-zag coding. Some examples:
| value |
hex |
| 0 |
00 |
| -1 |
01 |
| 1 |
02 |
| -2 |
03 |
| 2 |
04 |
| … |
|
| -64 |
7f |
| 64 |
80 01 |
| … |
|
- a float is written as 4 bytes. The float is converted into a 32-bit integer using a method equivalent to Java’s floatToIntBits and then encoded in little-endian format.
- a double is written as 8 bytes. The double is converted into a 64-bit integer using a method equivalent to Java’s doubleToLongBits and then encoded in little-endian format.
- bytes are encoded as a long followed by that many bytes of data.
- a string is encoded as a long followed by that many bytes of UTF-8 encoded character data.
For example, the three-character string “foo” would be encoded as the long value 3 (encoded as hex 06) followed by the UTF-8 encoding of ‘f’, ‘o’, and ‘o’ (the hex bytes 66 6f 6f):
06 66 6f 6f
Complex Types
Complex types are encoded in binary as follows:
Records
A record is encoded by encoding the values of its fields in the order that they are declared. In other words, a record is encoded as just the concatenation of the encodings of its fields. Field values are encoded per their schema.
For example, the record schema
{
"type": "record",
"name": "test",
"fields" : [
{"name": "a", "type": "long"},
{"name": "b", "type": "string"}
]
}
An instance of this record whose a field has value 27 (encoded as hex 36) and whose b field has value “foo” (encoded as hex bytes 06 66 6f 6f), would be encoded simply as the concatenation of these, namely the hex byte sequence:
36 06 66 6f 6f
Enums
An enum is encoded by a int, representing the zero-based position of the symbol in the schema.
For example, consider the enum:
{"type": "enum", "name": "Foo", "symbols": ["A", "B", "C", "D"] }
This would be encoded by an int between zero and three, with zero indicating “A”, and 3 indicating “D”.
Arrays
Arrays are encoded as a series of blocks. Each block consists of a long count value, followed by that many array items. A block with count zero indicates the end of the array. Each item is encoded per the array’s item schema.
If a block’s count is negative, its absolute value is used, and the count is followed immediately by a long block size indicating the number of bytes in the block. This block size permits fast skipping through data, e.g., when projecting a record to a subset of its fields.
For example, the array schema
{"type": "array", "items": "long"}
an array containing the items 3 and 27 could be encoded as the long value 2 (encoded as hex 04) followed by long values 3 and 27 (encoded as hex 06 36) terminated by zero:
04 06 36 00
The blocked representation permits one to read and write arrays larger than can be buffered in memory, since one can start writing items without knowing the full length of the array.
Maps
Maps are encoded as a series of blocks. Each block consists of a long count value, followed by that many key/value pairs. A block with count zero indicates the end of the map. Each item is encoded per the map’s value schema.
If a block’s count is negative, its absolute value is used, and the count is followed immediately by a long block size indicating the number of bytes in the block. This block size permits fast skipping through data, e.g., when projecting a record to a subset of its fields.
The blocked representation permits one to read and write maps larger than can be buffered in memory, since one can start writing items without knowing the full length of the map.
Unions
A union is encoded by first writing a long value indicating the zero-based position within the union of the schema of its value. The value is then encoded per the indicated schema within the union.
For example, the union schema ["null","string"] would encode:
-
null as zero (the index of “null” in the union):
00
-
the string “a” as one (the index of “string” in the union, 1, encoded as hex 02), followed by the serialized string:
02 02 61
Fixed
Fixed instances are encoded using the number of bytes declared in the schema.
JSON Encoding
Except for unions, the JSON encoding is the same as is used to encode field default values.
The value of a union is encoded in JSON as follows:
- if its type is null, then it is encoded as a JSON null;
- otherwise it is encoded as a JSON object with one name/value pair whose name is the type’s name and whose value is the recursively encoded value. For Avro’s named types (record, fixed or enum) the user-specified name is used, for other types the type name is used.
For example, the union schema ["null","string","Foo"], where Foo is a record name, would encode:
- null as null;
- the string “a” as
{"string": "a"} and
- a Foo instance as
{"Foo": {...}}, where {...} indicates the JSON encoding of a Foo instance.
Note that a schema is still required to correctly process JSON-encoded data. For example, the JSON encoding does not distinguish between int and long, float and double, records and maps, enums and strings, etc.
Sort Order
Avro defines a standard sort order for data. This permits data written by one system to be efficiently sorted by another system. This can be an important optimization, as sort order comparisons are sometimes the most frequent per-object operation. Note also that Avro binary-encoded data can be efficiently ordered without deserializing it to objects.
Data items may only be compared if they have identical schemas. Pairwise comparisons are implemented recursively with a depth-first, left-to-right traversal of the schema. The first mismatch encountered determines the order of the items.
Two items with the same schema are compared according to the following rules.
- null data is always equal.
- boolean data is ordered with false before true.
- int, long, float and double data is ordered by ascending numeric value.
- bytes and fixed data are compared lexicographically by unsigned 8-bit values.
- string data is compared lexicographically by Unicode code point. Note that since UTF-8 is used as the binary encoding for strings, sorting of bytes and string binary data is identical.
- array data is compared lexicographically by element.
- enum data is ordered by the symbol’s position in the enum schema. For example, an enum whose symbols are
["z", "a"] would sort “z” values before “a” values.
- union data is first ordered by the branch within the union, and, within that, by the type of the branch. For example, an
["int", "string"] union would order all int values before all string values, with the ints and strings themselves ordered as defined above.
- record data is ordered lexicographically by field. If a field specifies that its order is:
- “ascending”, then the order of its values is unaltered.
- “descending”, then the order of its values is reversed.
- “ignore”, then its values are ignored when sorting.
- map data may not be compared. It is an error to attempt to compare data containing maps unless those maps are in an
"order":"ignore" record field.
Object Container Files
Avro includes a simple object container file format. A file has a schema, and all objects stored in the file must be written according to that schema, using binary encoding. Objects are stored in blocks that may be compressed. Syncronization markers are used between blocks to permit efficient splitting of files for MapReduce processing.
Files may include arbitrary user-specified metadata.
A file consists of:
- A file header, followed by
- one or more file data blocks.
A file header consists of:
- Four bytes, ASCII ‘O’, ‘b’, ‘j’, followed by 1.
- file metadata, including the schema.
- The 16-byte, randomly-generated sync marker for this file.
File metadata is written as if defined by the following map schema:
{"type": "map", "values": "bytes"}
All metadata properties that start with “avro.” are reserved. The following file metadata properties are currently used:
- avro.schema contains the schema of objects stored in the file, as JSON data (required).
- avro.codec the name of the compression codec used to compress blocks, as a string. Implementations are required to support the following codecs: “null” and “deflate”. If codec is absent, it is assumed to be “null”. The codecs are described with more detail below.
A file header is thus described by the following schema:
{"type": "record", "name": "org.apache.avro.file.Header",
"fields" : [
{"name": "magic", "type": {"type": "fixed", "name": "Magic", "size": 4}},
{"name": "meta", "type": {"type": "map", "values": "bytes"}},
{"name": "sync", "type": {"type": "fixed", "name": "Sync", "size": 16}}
]
}
A file data block consists of:
- A long indicating the count of objects in this block.
- A long indicating the size in bytes of the serialized objects in the current block, after any codec is applied
- The serialized objects. If a codec is specified, this is compressed by that codec.
- The file’s 16-byte sync marker.
Thus, each block’s binary data can be efficiently extracted or skipped without deserializing the contents. The combination of block size, object counts, and sync markers enable detection of corrupt blocks and help ensure data integrity.
Required Codecs
null
The “null” codec simply passes through data uncompressed.
deflate
The “deflate” codec writes the data block using the deflate algorithm as specified in RFC 1951, and typically implemented using the zlib library. Note that this format (unlike the “zlib format” in RFC 1950) does not have a checksum.
Optional Codecs
snappy
The “snappy” codec uses Google’s Snappy compression library. Each compressed block is followed by the 4-byte, big-endian CRC32 checksum of the uncompressed data in the block.
Protocol Declaration
Avro protocols describe RPC interfaces. Like schemas, they are defined with JSON text.
A protocol is a JSON object with the following attributes:
- protocol, a string, the name of the protocol (required);
- namespace, an optional string that qualifies the name;
- doc, an optional string describing this protocol;
- types, an optional list of definitions of named types (records, enums, fixed and errors). An error definition is just like a record definition except it uses “error” instead of “record”. Note that forward references to named types are not permitted.
- messages, an optional JSON object whose keys are message names and whose values are objects whose attributes are described below. No two messages may have the same name.
The name and namespace qualification rules defined for schema objects apply to protocols as well.
Messages
A message has attributes:
- a doc, an optional description of the message,
- a request, a list of named, typed parameter schemas (this has the same form as the fields of a record declaration);
- a response schema;
- an optional union of declared error schemas. The effective union has “string” prepended to the declared union, to permit transmission of undeclared “system” errors. For example, if the declared error union is
["AccessError"], then the effective union is ["string", "AccessError"]. When no errors are declared, the effective error union is ["string"]. Errors are serialized using the effective union; however, a protocol’s JSON declaration contains only the declared union.
- an optional one-way boolean parameter.
A request parameter list is processed equivalently to an anonymous record. Since record field lists may vary between reader and writer, request parameters may also differ between the caller and responder, and such differences are resolved in the same manner as record field differences.
The one-way parameter may only be true when the response type is "null" and no errors are listed.
Sample Protocol
For example, one may define a simple HelloWorld protocol with:
{
"namespace": "com.acme",
"protocol": "HelloWorld",
"doc": "Protocol Greetings",
"types": [
{"name": "Greeting", "type": "record", "fields": [
{"name": "message", "type": "string"}]},
{"name": "Curse", "type": "error", "fields": [
{"name": "message", "type": "string"}]}
],
"messages": {
"hello": {
"doc": "Say hello.",
"request": [{"name": "greeting", "type": "Greeting" }],
"response": "Greeting",
"errors": ["Curse"]
}
}
}
Message Transport
Messages may be transmitted via different transport mechanisms.
To the transport, a message is an opaque byte sequence.
A transport is a system that supports:
- transmission of request messages
- receipt of corresponding response messages
Servers may send a response message back to the client corresponding to a request message. The mechanism of correspondance is transport-specific. For example, in HTTP it is implicit, since HTTP directly supports requests and responses. But a transport that multiplexes many client threads over a single socket would need to tag messages with unique identifiers.
Transports may be either stateless or stateful. In a stateless transport, messaging assumes no established connection state, while stateful transports establish connections that may be used for multiple messages. This distinction is discussed further in the handshake section below.
HTTP as Transport
When HTTP is used as a transport, each Avro message exchange is an HTTP request/response pair. All messages of an Avro protocol should share a single URL at an HTTP server. Other protocols may also use that URL. Both normal and error Avro response messages should use the 200 (OK) response code. The chunked encoding may be used for requests and responses, but, regardless the Avro request and response are the entire content of an HTTP request and response. The HTTP Content-Type of requests and responses should be specified as “avro/binary”. Requests should be made using the POST method.
HTTP is used by Avro as a stateless transport.
Message Framing
Avro messages are framed as a list of buffers.
Framing is a layer between messages and the transport. It exists to optimize certain operations.
The format of framed message data is:
- a series of buffers, where each buffer consists of:
- a four-byte, big-endian buffer length, followed by
- that many bytes of buffer data.
- A message is always terminated by a zero-lenghted buffer.
Framing is transparent to request and response message formats (described below). Any message may be presented as a single or multiple buffers.
Framing can permit readers to more efficiently get different buffers from different sources and for writers to more efficiently store different buffers to different destinations. In particular, it can reduce the number of times large binary objects are copied. For example, if an RPC parameter consists of a megabyte of file data, that data can be copied directly to a socket from a file descriptor, and, on the other end, it could be written directly to a file descriptor, never entering user space.
A simple, recommended, framing policy is for writers to create a new segment whenever a single binary object is written that is larger than a normal output buffer. Small objects are then appended in buffers, while larger objects are written as their own buffers. When a reader then tries to read a large object the runtime can hand it an entire buffer directly, without having to copy it.
Handshake
The purpose of the handshake is to ensure that the client and the server have each other’s protocol definition, so that the client can correctly deserialize responses, and the server can correctly deserialize requests. Both clients and servers should maintain a cache of recently seen protocols, so that, in most cases, a handshake will be completed without extra round-trip network exchanges or the transmission of full protocol text.
RPC requests and responses may not be processed until a handshake has been completed. With a stateless transport, all requests and responses are prefixed by handshakes. With a stateful transport, handshakes are only attached to requests and responses until a successful handshake response has been returned over a connection. After this, request and response payloads are sent without handshakes for the lifetime of that connection.
The handshake process uses the following record schemas:
{
"type": "record",
"name": "HandshakeRequest", "namespace":"org.apache.avro.ipc",
"fields": [
{"name": "clientHash",
"type": {"type": "fixed", "name": "MD5", "size": 16}},
{"name": "clientProtocol", "type": ["null", "string"]},
{"name": "serverHash", "type": "MD5"},
{"name": "meta", "type": ["null", {"type": "map", "values": "bytes"}]}
]
}
{
"type": "record",
"name": "HandshakeResponse", "namespace": "org.apache.avro.ipc",
"fields": [
{"name": "match",
"type": {"type": "enum", "name": "HandshakeMatch",
"symbols": ["BOTH", "CLIENT", "NONE"]}},
{"name": "serverProtocol",
"type": ["null", "string"]},
{"name": "serverHash",
"type": ["null", {"type": "fixed", "name": "MD5", "size": 16}]},
{"name": "meta",
"type": ["null", {"type": "map", "values": "bytes"}]}
]
}
- A client first prefixes each request with a
HandshakeRequest containing just the hash of its protocol and of the server’s protocol (clientHash!=null, clientProtocol=null, serverHash!=null), where the hashes are 128-bit MD5 hashes of the JSON protocol text. If a client has never connected to a given server, it sends its hash as a guess of the server’s hash, otherwise it sends the hash that it previously obtained from this server.
The server responds with a HandshakeResponse containing one of:
-
match=BOTH, serverProtocol=null, serverHash=null if the client sent the valid hash of the server’s protocol and the server knows what protocol corresponds to the client’s hash. In this case, the request is complete and the response data immediately follows the HandshakeResponse.
-
match=CLIENT, serverProtocol!=null, serverHash!=null if the server has previously seen the client’s protocol, but the client sent an incorrect hash of the server’s protocol. The request is complete and the response data immediately follows the HandshakeResponse. The client must use the returned protocol to process the response and should also cache that protocol and its hash for future interactions with this server.
-
match=NONE if the server has not previously seen the client’s protocol. The serverHash and serverProtocol may also be non-null if the server’s protocol hash was incorrect.
In this case the client must then re-submit its request with its protocol text (clientHash!=null, clientProtocol!=null, serverHash!=null) and the server should respond with a successful match (match=BOTH, serverProtocol=null, serverHash=null) as above.
The meta field is reserved for future handshake enhancements.
A call consists of a request message paired with its resulting response or error message. Requests and responses contain extensible metadata, and both kinds of messages are framed as described above.
The format of a call request is:
- request metadata, a map with values of type bytes
- the message name, an Avro string, followed by
- the message parameters. Parameters are serialized according to the message’s request declaration.
When the empty string is used as a message name a server should ignore the parameters and return an empty response. A client may use this to ping a server or to perform a handshake without sending a protocol message.
When a message is declared one-way and a stateful connection has been established by a successful handshake response, no response data is sent. Otherwise the format of the call response is:
- response metadata, a map with values of type bytes
- a one-byte error flag boolean, followed by either:
- if the error flag is false, the message response, serialized per the message’s response schema.
- if the error flag is true, the error, serialized per the message’s effective error union schema.
Schema Resolution
A reader of Avro data, whether from an RPC or a file, can always parse that data because the original schema must be provided along with the data. However, the reader may be programmed to read data into a different schema. For example, if the data was written with a different version of the software than it is read, then fields may have been added or removed from records. This section specifies how such schema differences should be resolved.
We refer to the schema used to write the data as the writer’s schema, and the schema that the application expects the reader’s schema. Differences between these should be resolved as follows:
- It is an error if the two schemas do not match.
To match, one of the following must hold:
-
both schemas are arrays whose item types match
-
both schemas are maps whose value types match
-
both schemas are enums whose names match
-
both schemas are fixed whose sizes and names match
-
both schemas are records with the same name
-
either schema is a union
-
both schemas have same primitive type
-
the writer’s schema may be promoted to the reader’s as follows:
- int is promotable to long, float, or double
- long is promotable to float or double
- float is promotable to double
- string is promotable to bytes
- bytes is promotable to string
-
if both are records:
- the ordering of fields may be different: fields are matched by name.
- schemas for fields with the same name in both records are resolved recursively.
- if the writer’s record contains a field with a name not present in the reader’s record, the writer’s value for that field is ignored.
- if the reader’s record schema has a field that contains a default value, and writer’s schema does not have a field with the same name, then the reader should use the default value from its field.
- if the reader’s record schema has a field with no default value, and writer’s schema does not have a field with the same name, an error is signalled.
-
if both are enums:
if the writer’s symbol is not present in the reader’s enum and the reader has a default value, then that value is used, otherwise an error is signalled.
This resolution algorithm is applied recursively to the reader’s and writer’s array item schemas.
This resolution algorithm is applied recursively to the reader’s and writer’s value schemas.
The first schema in the reader’s union that matches the selected writer’s union schema is recursively resolved against it. if none match, an error is signalled.
- if reader’s is a union, but writer’s is not
The first schema in the reader’s union that matches the writer’s schema is recursively resolved against it. If none match, an error is signalled.
- if writer’s is a union, but reader’s is not
If the reader’s schema matches the selected writer’s schema, it is recursively resolved against it. If they do not match, an error is signalled.
A schema’s doc fields are ignored for the purposes of schema resolution. Hence, the doc portion of a schema may be dropped at serialization.
One of the defining characteristics of Avro is that a reader is assumed to have the “same” schema used by the writer of the data the reader is reading. This assumption leads to a data format that’s compact and also amenable to many forms of schema evolution. However, the specification so far has not defined what it means for the reader to have the “same” schema as the writer. Does the schema need to be textually identical? Well, clearly adding or removing some whitespace to a JSON expression does not change its meaning. At the same time, reordering the fields of records clearly does change the meaning. So what does it mean for a reader to have “the same” schema as a writer?
Parsing Canonical Form is a transformation of a writer’s schema that let’s us define what it means for two schemas to be “the same” for the purpose of reading data written against the schema. It is called Parsing Canonical Form because the transformations strip away parts of the schema, like “doc” attributes, that are irrelevant to readers trying to parse incoming data. It is called Canonical Form because the transformations normalize the JSON text (such as the order of attributes) in a way that eliminates unimportant differences between schemas. If the Parsing Canonical Forms of two different schemas are textually equal, then those schemas are “the same” as far as any reader is concerned, i.e., there is no serialized data that would allow a reader to distinguish data generated by a writer using one of the original schemas from data generated by a writing using the other original schema. (We sketch a proof of this property in a companion document.)
The next subsection specifies the transformations that define Parsing Canonical Form. But with a well-defined canonical form, it can be convenient to go one step further, transforming these canonical forms into simple integers (“fingerprints”) that can be used to uniquely identify schemas. The subsection after next recommends some standard practices for generating such fingerprints.
Assuming an input schema (in JSON form) that’s already UTF-8 text for a valid Avro schema (including all quotes as required by JSON), the following transformations will produce its Parsing Canonical Form:
- [PRIMITIVES] Convert primitive schemas to their simple form (e.g., int instead of
{"type":"int"}).
- [FULLNAMES] Replace short names with fullnames, using applicable namespaces to do so. Then eliminate namespace attributes, which are now redundant.
- [STRIP] Keep only attributes that are relevant to parsing data, which are: type, name, fields, symbols, items, values, size. Strip all others (e.g., doc and aliases).
- [ORDER] Order the appearance of fields of JSON objects as follows: name, type, fields, symbols, items, values, size. For example, if an object has type, name, and size fields, then the name field should appear first, followed by the type and then the size fields.
- [STRINGS] For all JSON string literals in the schema text, replace any escaped characters (e.g., \uXXXX escapes) with their UTF-8 equivalents.
- [INTEGERS] Eliminate quotes around and any leading zeros in front of JSON integer literals (which appear in the size attributes of fixed schemas).
- [WHITESPACE] Eliminate all whitespace in JSON outside of string literals.
Schema Fingerprints
“[A] fingerprinting algorithm is a procedure that maps an arbitrarily large data item (such as a computer file) to a much shorter bit string, its fingerprint, that uniquely identifies the original data for all practical purposes” (quoted from Wikipedia). In the Avro context, fingerprints of Parsing Canonical Form can be useful in a number of applications; for example, to cache encoder and decoder objects, to tag data items with a short substitute for the writer’s full schema, and to quickly negotiate common-case schemas between readers and writers.
In designing fingerprinting algorithms, there is a fundamental trade-off between the length of the fingerprint and the probability of collisions. To help application designers find appropriate points within this trade-off space, while encouraging interoperability and ease of implementation, we recommend using one of the following three algorithms when fingerprinting Avro schemas:
- When applications can tolerate longer fingerprints, we recommend using the SHA-256 digest algorithm to generate 256-bit fingerprints of Parsing Canonical Forms. Most languages today have SHA-256 implementations in their libraries.
- At the opposite extreme, the smallest fingerprint we recommend is a 64-bit Rabin fingerprint. Below, we provide pseudo-code for this algorithm that can be easily translated into any programming language. 64-bit fingerprints should guarantee uniqueness for schema caches of up to a million entries (for such a cache, the chance of a collision is 3E-8). We don’t recommend shorter fingerprints, as the chances of collisions is too great (for example, with 32-bit fingerprints, a cache with as few as 100,000 schemas has a 50% chance of having a collision).
- Between these two extremes, we recommend using the MD5 message digest to generate 128-bit fingerprints. These make sense only where very large numbers of schemas are being manipulated (tens of millions); otherwise, 64-bit fingerprints should be sufficient. As with SHA-256, MD5 implementations are found in most libraries today.
These fingerprints are not meant to provide any security guarantees, even the longer SHA-256-based ones. Most Avro applications should be surrounded by security measures that prevent attackers from writing random data and otherwise interfering with the consumers of schemas. We recommend that these surrounding mechanisms be used to prevent collision and pre-image attacks (i.e., “forgery”) on schema fingerprints, rather than relying on the security properties of the fingerprints themselves.
Rabin fingerprints are cyclic redundancy checks computed using irreducible polynomials. In the style of the Appendix of RFC 1952 (pg 10), which defines the CRC-32 algorithm, here’s our definition of the 64-bit AVRO fingerprinting algorithm:
long fingerprint64(byte[] buf) {
if (FP_TABLE == null) initFPTable();
long fp = EMPTY;
for (int i = 0; i < buf.length; i++)
fp = (fp >>> 8) ^ FP_TABLE[(int)(fp ^ buf[i]) & 0xff];
return fp;
}
static long EMPTY = 0xc15d213aa4d7a795L;
static long[] FP_TABLE = null;
void initFPTable() {
FP_TABLE = new long[256];
for (int i = 0; i < 256; i++) {
long fp = i;
for (int j = 0; j < 8; j++)
fp = (fp >>> 1) ^ (EMPTY & -(fp & 1L));
FP_TABLE[i] = fp;
}
}
Readers interested in the mathematics behind this algorithm may want to read this book chapter. (Unlike RFC-1952 and the book chapter, we prepend a single one bit to messages. We do this because CRCs ignore leading zero bits, which can be problematic. Our code prepends a one-bit by initializing fingerprints using EMPTY, rather than initializing using zero as in RFC-1952 and the book chapter.)
Logical Types
A logical type is an Avro primitive or complex type with extra attributes to represent a derived type. The attribute logicalType must always be present for a logical type, and is a string with the name of one of the logical types listed later in this section. Other attributes may be defined for particular logical types.
A logical type is always serialized using its underlying Avro type so that values are encoded in exactly the same way as the equivalent Avro type that does not have a logicalType attribute. Language implementations may choose to represent logical types with an appropriate native type, although this is not required.
Language implementations must ignore unknown logical types when reading, and should use the underlying Avro type. If a logical type is invalid, for example a decimal with scale greater than its precision, then implementations should ignore the logical type and use the underlying Avro type.
Decimal
The decimal logical type represents an arbitrary-precision signed decimal number of the form unscaled × 10-scale.
A decimal logical type annotates Avro bytes or fixed types. The byte array must contain the two’s-complement representation of the unscaled integer value in big-endian byte order. The scale is fixed, and is specified using an attribute.
The following attributes are supported:
- scale, a JSON integer representing the scale (optional). If not specified the scale is 0.
- precision, a JSON integer representing the (maximum) precision of decimals stored in this type (required).
For example, the following schema represents decimal numbers with a maximum precision of 4 and a scale of 2:
{
"type": "bytes",
"logicalType": "decimal",
"precision": 4,
"scale": 2
}
Precision must be a positive integer greater than zero. If the underlying type is a fixed, then the precision is limited by its size. An array of length n can store at most floor(log10(28 × n - 1 - 1)) base-10 digits of precision.
Scale must be zero or a positive integer less than or equal to the precision.
For the purposes of schema resolution, two schemas that are decimal logical types match if their scales and precisions match.
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
12.4 - Trevni
Version 0.1
DRAFT
This document is the authoritative specification of a file format. Its intent is to permit compatible, independent implementations that read and/or write files in this format.
Introduction
Data sets are often described as a table composed of rows and columns. Each record in the dataset is considered a row, with each field of the record occupying a different column. Writing records to a file one-by-one as they are created results in a row-major format, like Hadoop’s SequenceFile or Avro data files.
In many cases higher query performance may be achieved if the data is instead organized in a column-major format, where multiple values of a given column are stored adjacently. This document defines such a column-major file format for datasets.
To permit scalable, distributed query evaluation, datasets are partitioned into row groups, containing distinct collections of rows. Each row group is organized in column-major order, while row groups form a row-major partitioning of the entire dataset.
Rationale
Goals
The format is meant satisfy the following goals:
Maximize the size of row groups. Disc drives are used most efficiently when sequentially accessing data. Consider a drive that takes 10ms to seek and transfers at 100MB/second. If a 10-column dataset whose values are all the same size is split into 10MB row groups, then accessing a single column will require a sequence of seek+1MB reads, for a cost of 20ms/MB processed. If the same dataset is split into 100MB row groups then this drops to 11ms/MB processed. This effect is exaggerated for datasets with larger numbers of columns and with columns whose values are smaller than average. So we’d prefer row groups that are 100MB or greater.
Permit random access within a row group. Some queries will first examine one column, and, only when certain relatively rare criteria are met, examine other columns. Rather than iterating through selected columns of the row-group in parallel, one might iterate through one column and randomly access another. This is called support for WHERE clauses, after the SQL operator of that name.
Minimize the number of files per dataset. HDFS is a primary intended deployment platform for these files. The HDFS Namenode requires memory for each file in the filesystem, thus for a format to be HDFS-friendly it should strive to require the minimum number of distinct files.
Support co-location of columns within row-groups. Row groups are the unit of parallel operation on a column dataset. For efficient file i/o, the entirety of a row-group should ideally reside on the host that is evaluating the query in order to avoid network latencies and bottlenecks.
Data integrity. The format should permit applications to detect data corruption. Many file systems may prevent corruption, but files may be moved between filesystems and be subject to corruption at points in that process. It is best if the data in a file can be validated independently.
Extensibility. The format should permit applications to store additional annotations about a datasets in the files, such as type information, origin, etc. Some environments may have metadata stores for such information, but not all do, and files might be moved among systems with different metadata systems. The ability to keep such information within the file simplifies the coordination of such information.
Minimal overhead. The column format should not make datasets appreciably larger. Storage is a primary cost and a choice to use this format should not require additional storage.
Primary format. The column format should be usable as a primary format for datasets, not as an auxiliary, accelerated format. Applications that process a dataset in row-major order should be able to easily consume column files and applications that produce datasets in row-major order should be able to easily generate column files.
Design
To meet these goals we propose the following design.
Each row group is a separate file. All values of a column in a file are written contiguously. This maximizes the row group size, optimizing performance when querying few and small columns.
Each file occupies a single HDFS block. A larger than normal block size may be specified, e.g., ~1GB instead of the typical ~100MB. This guarantees co-location and eliminates network use when query processing can be co-located with the file. This also moderates the memory impact on the HDFS Namenode since no small files are written.
Each column in a file is written as a sequence of ~64kB compressed blocks. The sequence is prefixed by a table describing all of the blocks in the column to permit random access within the column.
Application-specific metadata may be added at the file, column, and block levels.
Checksums are included with each block, providing data integrity.
Discussion
The use of a single block per file achieves the same effect as the custom block placement policy described in the CIF paper, but while still permitting HDFS rebalancing and not increasing the number of files in the namespace.
This section formally describes the proposed column file format.
Data Model
We assume a simple data model, where a record is a set of named fields, and the value of each field is a sequence of untyped bytes. A type system may be layered on top of this, as specified in the Type Mapping section below.
Primitive Values
We define the following primitive value types:
- Signed 64-bit long values are written using a variable-length zig-zag coding, where the high-order bit in each byte determines whether subsequent bytes are present. For example:
decimal value hex bytes
| decimal value |
hex bytes |
|
| 0 |
00 |
|
| -1 |
01 |
|
| 1 |
02 |
|
| … |
|
|
| -64 |
7f |
|
| 64 |
80 01 |
|
| … |
|
|
- bytes are encoded as a long followed by that many bytes of data.
- a string is encoded as a long followed by that many bytes of UTF-8 encoded character data.
For example, the three-character string “foo” would be encoded as the long value 3 (encoded as hex 06) followed by the UTF-8 encoding of ‘f’, ‘o’, and ‘o’ (the hex bytes 66 6f 6f): 06 66 6f 6f
Type Names
The following type names are used to describe column values:
- null, requires zero bytes. Sometimes used in array columns.
- boolean, one bit, packed into bytes, little-endian;
- int, like long, but restricted to 32-bit signed values
- long 64-bit signed values, represented as above
- fixed32 32-bit values stored as four bytes, little-endian.
- fixed64 64-bit values stored as eight bytes, little-endian.
- float 32-bit IEEE floating point value, little-endian
- double 64-bit IEEE floating point value, little-endian
- string as above
- bytes as above, may be used to encapsulate more complex objects
Type names are represented as strings (UTF-8 encoded, length-prefixed).
Metadata consists of:
- A long indicating the number of metadata key/value pairs.
- For each pair, a string key and bytes value.
All metadata properties that start with “trevni.” are reserved.
The following file metadata properties are defined:
- trevni.codec the name of the default compression codec used to compress blocks, as a string. Implementations are required to support the “null” codec. Optional. If absent, it is assumed to be “null”. Codecs are described in more detail below.
- trevni.checksum the name of the checksum algorithm used in this file, as a string. Implementations are required to support the “crc-32” checksum. Optional. If absent, it is assumed to be “null”. Checksums are described in more detail below.
The following column metadata properties are defined:
- trevni.codec the name of the compression codec used to compress the blocks of this column, as a string. Implementations are required to support the “null” codec. Optional. If absent, it is assumed to be “null”. Codecs are described in more detail below.
- trevni.name the name of the column, as a string. Required.
- trevni.type the type of data in the column. One of the type names above. Required.
- trevni.values if present, indicates that the initial value of each block in this column will be stored in the block’s descriptor. Not permitted for array columns or columns that specify a parent.
- trevni.array if present, indicates that each row in this column contains a sequence of values of the named type rather than just a single value. An integer length precedes each sequence of values indicating the count of values in the sequence. If the length is negative then it indicates a sequence of zero or one lengths, where -1 indicates two zeros, -2 two ones, -3 three zeros, -4 three ones, etc.
- trevni.parent if present, the name of an array column whose lengths are also used by this column. Thus values of this column are sequences but no lengths are stored in this column.
For example, consider the following row, as JSON, where all values are primitive types, but one has multiple values.
{"id"=566, "date"=23423234234
"from"="foo@bar.com",
"to"=["bar@baz.com", "bang@foo.com"],
"content"="Hi!"}
The columns for this might be specified as:
name=id type=int
name=date type=long
name=from type=string
name=to type=string array=true
name=content type=string
If a row contains an array of records, e.g. “received” in the following:
{"id"=566, "date"=23423234234
"from"="foo@bar.com",
"to"=["bar@baz.com", "bang@foo.com"],
"content"="Hi!"
"received"=[{"date"=234234234234, "host"="192.168.0.0.1"},
{"date"=234234545645, "host"="192.168.0.0.2"}]
}
Then one can define a parent column followed by a column for each field in the record, adding the following columns:
name=received type=null array=true
name=date type=long parent=received
name=host type=string parent=received
If an array value itself contains an array, e.g. the “sigs” below:
{"id"=566, "date"=23423234234
"from"="foo@bar.com",
"to"=["bar@baz.com", "bang@foo.com"],
"content"="Hi!"
"received"=[{"date"=234234234234, "host"="192.168.0.0.1",
"sigs"=[{"algo"="weak", "value"="0af345de"}]},
{"date"=234234545645, "host"="192.168.0.0.2",
"sigs"=[]}]
}
Then a parent column may be defined that itself has a parent column.
name=sigs type=null array=true parent=received
name=algo type=string parent=sigs
name=value type=string parent=sigs
No block metadata properties are currently defined.
A file consists of:
- A file header, followed by
- one or more columns.
A file header consists of:
- Four bytes, ASCII ‘T’, ‘r’, ‘v’, followed by 0x02.
- a fixed64 indicating the number of rows in the file
- a fixed32 indicating the number of columns in the file
- file metadata.
- for each column, its column metadata
- for each column, its starting position in the file as a fixed64.
A column consists of:
- A fixed32 indicating the number of blocks in this column.
- For each block, a block descriptor
- One or more blocks.
A block descriptor consists of:
- A fixed32 indicating the number of rows in the block
- A fixed32 indicating the size in bytes of the block before the codec is applied (excluding checksum).
- A fixed32 indicating the size in bytes of the block after the codec is applied (excluding checksum).
- If this column’s metadata declares it to include values, the first value in the column, serialized according to this column’s type.
A block consists of:
- The serialized column values. If a column is an array column then value sequences are preceded by their length, as an int. If a codec is specified, the values and lengths are compressed by that codec.
- The checksum, as determined by the file metadata.
Codecs
null
The “null” codec simply passes data through uncompressed.
deflate
The “deflate” codec writes the data block using the deflate algorithm as specified in RFC 1951.
snappy
The “snappy” codec uses Google’s Snappy compression library.
Checksum algorithms
null
The “null” checksum contains zero bytes.
crc-32
Each “crc-32” checksum contains the four bytes of an ISO 3309 CRC-32 checksum of the uncompressed block data as a fixed32.
Type Mappings
We define a standard mapping for how types defined in various serialization systems are represented in a column file. Records from these systems are shredded into columns. When records are nested, a depth-first recursive walk can assign a separate column for each primitive value.
Avro
Protocol Buffers
Thrift
Implementation Notes
Some possible techniques for writing column files include:
- Use a standard ~100MB block, buffer in memory up to the block size, then flush the file directly to HDFS. A single reduce task might create multiple output files. The namenode requires memory proportional to the number of names and blocks*replication. This would increase the number of names but not blocks, so this should still be much better than a file per column.
- Spill each column to a separate local, temporary file then, when the file is closed, append these files, writing a single file to HDFS whose block size is set to be that of the entire file. This would be a bit slower than and may have trouble when the local disk is full, but it would better use HDFS namespace and further reduce seeks when processing columns whose values are small.
- Use a separate mapreduce job to convert row-major files to column-major. The map output would output a by (row#, column#, value) tuple, partitioned by row# but sorted by column# then row#. The reducer could directly write the column file. But the column file format would need to be changed to write counts, descriptors, etc. at the end of files rather than at the front.
(1) is the simplest to implement and most implementations should start with it.
References
CIF Column-Oriented Storage Techniques for MapReduce, Floratou, Patel, Shekita, & Tata, VLDB 2011.
DREMEL Dremel: Interactive Analysis of Web-Scale Datasets, Melnik, Gubarev, Long, Romer, Shivakumar, & Tolton, VLDB 2010.
12.5 - MapReduce guide
Avro provides a convenient way to represent complex data structures within a Hadoop MapReduce job. Avro data can be used as both input to and output from a MapReduce job, as well as the intermediate format. The example in this guide uses Avro data for all three, but it’s possible to mix and match; for instance, MapReduce can be used to aggregate a particular field in an Avro record.
This guide assumes basic familiarity with both Hadoop MapReduce and Avro. See the Hadoop documentation and the Avro getting started guide for introductions to these projects. This guide uses the old MapReduce API (org.apache.hadoop.mapred) and the new MapReduce API (org.apache.hadoop.mapreduce).
Setup
The code from this guide is included in the Avro docs under examples/mr-example. The example is set up as a Maven project that includes the necessary Avro and MapReduce dependencies and the Avro Maven plugin for code generation, so no external jars are needed to run the example. In particular, the POM includes the following dependencies:
<dependency>
<groupId>org.apache.avro</groupId>
<artifactId>avro</artifactId>
<version>1.7.7</version>
</dependency>
<dependency>
<groupId>org.apache.avro</groupId>
<artifactId>avro-mapred</artifactId>
<version>1.7.7</version>
</dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-core</artifactId>
<version>1.1.0</version>
</dependency>
And the following plugin:
<plugin>
<groupId>org.apache.avro</groupId>
<artifactId>avro-maven-plugin</artifactId>
<version>1.7.7</version>
<executions>
<execution>
<phase>generate-sources</phase>
<goals>
<goal>schema</goal>
</goals>
<configuration>
<sourceDirectory>${project.basedir}/../</sourceDirectory>
<outputDirectory>${project.basedir}/target/generated-sources/</outputDirectory>
</configuration>
</execution>
</executions>
</plugin>
If you do not configure the sourceDirectory and outputDirectory properties, the defaults will be used. The sourceDirectory property defaults to src/main/avro. The outputDirectory property defaults to target/generated-sources. You can change the paths to match your project layout.
Alternatively, Avro jars can be downloaded directly from the Apache Avro™ Releases page. The relevant Avro jars for this guide are avro-1.7.7.jar and avro-mapred-1.7.7.jar, as well as avro-tools-1.7.7.jar for code generation and viewing Avro data files as JSON. In addition, you will need to install Hadoop in order to use MapReduce.
Example: ColorCount
Below is a simple example of a MapReduce that uses Avro. There is an example for both the old (org.apache.hadoop.mapred) and new (org.apache.hadoop.mapreduce) APIs under examples/mr-example/src/main/java/example/. MapredColorCount is the example for the older mapred API while MapReduceColorCount is the example for the newer mapreduce API. Both examples are below, but we will detail the mapred API in our subsequent examples.
MapredColorCount:
package example;
import java.io.IOException;
import org.apache.avro.*;
import org.apache.avro.Schema.Type;
import org.apache.avro.mapred.*;
import org.apache.hadoop.conf.*;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.mapred.*;
import org.apache.hadoop.util.*;
import example.avro.User;
public class MapredColorCount extends Configured implements Tool {
public static class ColorCountMapper extends AvroMapper<User, Pair<CharSequence, Integer>> {
@Override
public void map(User user, AvroCollector<Pair<CharSequence, Integer>> collector, Reporter reporter)
throws IOException {
CharSequence color = user.getFavoriteColor();
// We need this check because the User.favorite_color field has type ["string", "null"]
if (color == null) {
color = "none";
}
collector.collect(new Pair<CharSequence, Integer>(color, 1));
}
}
public static class ColorCountReducer extends AvroReducer<CharSequence, Integer,
Pair<CharSequence, Integer>> {
@Override
public void reduce(CharSequence key, Iterable<Integer> values,
AvroCollector<Pair<CharSequence, Integer>> collector,
Reporter reporter)
throws IOException {
int sum = 0;
for (Integer value : values) {
sum += value;
}
collector.collect(new Pair<CharSequence, Integer>(key, sum));
}
}
public int run(String[] args) throws Exception {
if (args.length != 2) {
System.err.println("Usage: MapredColorCount <input path> <output path>");
return -1;
}
JobConf conf = new JobConf(getConf(), MapredColorCount.class);
conf.setJobName("colorcount");
FileInputFormat.setInputPaths(conf, new Path(args[0]));
FileOutputFormat.setOutputPath(conf, new Path(args[1]));
AvroJob.setMapperClass(conf, ColorCountMapper.class);
AvroJob.setReducerClass(conf, ColorCountReducer.class);
// Note that AvroJob.setInputSchema and AvroJob.setOutputSchema set
// relevant config options such as input/output format, map output
// classes, and output key class.
AvroJob.setInputSchema(conf, User.getClassSchema());
AvroJob.setOutputSchema(conf, Pair.getPairSchema(Schema.create(Type.STRING),
Schema.create(Type.INT)));
JobClient.runJob(conf);
return 0;
}
public static void main(String[] args) throws Exception {
int res = ToolRunner.run(new Configuration(), new MapredColorCount(), args);
System.exit(res);
}
}
MapReduceColorCount:
package example;
import java.io.IOException;
import org.apache.avro.Schema;
import org.apache.avro.mapred.AvroKey;
import org.apache.avro.mapred.AvroValue;
import org.apache.avro.mapreduce.AvroJob;
import org.apache.avro.mapreduce.AvroKeyInputFormat;
import org.apache.avro.mapreduce.AvroKeyValueOutputFormat;
import org.apache.hadoop.conf.Configured;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.IntWritable;
import org.apache.hadoop.io.NullWritable;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.mapreduce.Job;
import org.apache.hadoop.mapreduce.Mapper;
import org.apache.hadoop.mapreduce.Reducer;
import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
import org.apache.hadoop.util.Tool;
import org.apache.hadoop.util.ToolRunner;
import example.avro.User;
public class MapReduceColorCount extends Configured implements Tool {
public static class ColorCountMapper extends
Mapper<AvroKey<User>, NullWritable, Text, IntWritable> {
@Override
public void map(AvroKey<User> key, NullWritable value, Context context)
throws IOException, InterruptedException {
CharSequence color = key.datum().getFavoriteColor();
if (color == null) {
color = "none";
}
context.write(new Text(color.toString()), new IntWritable(1));
}
}
public static class ColorCountReducer extends
Reducer<Text, IntWritable, AvroKey<CharSequence>, AvroValue<Integer>> {
@Override
public void reduce(Text key, Iterable<IntWritable> values,
Context context) throws IOException, InterruptedException {
int sum = 0;
for (IntWritable value : values) {
sum += value.get();
}
context.write(new AvroKey<CharSequence>(key.toString()), new AvroValue<Integer>(sum));
}
}
public int run(String[] args) throws Exception {
if (args.length != 2) {
System.err.println("Usage: MapReduceColorCount <input path> <output path>");
return -1;
}
Job job = new Job(getConf());
job.setJarByClass(MapReduceColorCount.class);
job.setJobName("Color Count");
FileInputFormat.setInputPaths(job, new Path(args[0]));
FileOutputFormat.setOutputPath(job, new Path(args[1]));
job.setInputFormatClass(AvroKeyInputFormat.class);
job.setMapperClass(ColorCountMapper.class);
AvroJob.setInputKeySchema(job, User.getClassSchema());
job.setMapOutputKeyClass(Text.class);
job.setMapOutputValueClass(IntWritable.class);
job.setOutputFormatClass(AvroKeyValueOutputFormat.class);
job.setReducerClass(ColorCountReducer.class);
AvroJob.setOutputKeySchema(job, Schema.create(Schema.Type.STRING));
AvroJob.setOutputValueSchema(job, Schema.create(Schema.Type.INT));
return (job.waitForCompletion(true) ? 0 : 1);
}
public static void main(String[] args) throws Exception {
int res = ToolRunner.run(new MapReduceColorCount(), args);
System.exit(res);
}
}
ColorCount reads in data files containing User records, defined in examples/user.avsc, and counts the number of instances of each favorite color. (This example draws inspiration from the canonical WordCount MapReduce application.) This example uses the old MapReduce API. See MapReduceAvroWordCount, found under doc/examples/mr-example/src/main/java/example/ to see the new MapReduce API example. The User schema is defined as follows:
{"namespace": "example.avro",
"type": "record",
"name": "User",
"fields": [
{"name": "name", "type": "string"},
{"name": "favorite_number", "type": ["int", "null"]},
{"name": "favorite_color", "type": ["string", "null"]}
]
}
This schema is compiled into the User class used by ColorCount via the Avro Maven plugin (see examples/mr-example/pom.xml for how this is set up).
ColorCountMapper essentially takes a User as input and extracts the User’s favorite color, emitting the key-value pair <favoriteColor, 1>. ColorCountReducer then adds up how many occurrences of a particular favorite color were emitted, and outputs the result as a Pair record. These Pairs are serialized to an Avro data file.
Running ColorCount
The ColorCount application is provided as a Maven project in the Avro docs under examples/mr-example. To build the project, including the code generation of the User schema, run:
Next, run GenerateData from examples/mr-examples to create an Avro data file, input/users.avro, containing 20 Users with favorite colors chosen randomly from a list:
mvn exec:java -q -Dexec.mainClass=example.GenerateData
Besides creating the data file, GenerateData prints the JSON representations of the Users generated to stdout, for example:
{"name": "user", "favorite_number": null, "favorite_color": "red"}
{"name": "user", "favorite_number": null, "favorite_color": "green"}
{"name": "user", "favorite_number": null, "favorite_color": "purple"}
{"name": "user", "favorite_number": null, "favorite_color": null}
...
Now we’re ready to run ColorCount. We specify our freshly-generated input folder as the input path and output as our output folder (note that MapReduce will not start a job if the output folder already exists):
mvn exec:java -q -Dexec.mainClass=example.MapredColorCount -Dexec.args="input output"
Once ColorCount completes, checking the contents of the new output directory should yield the following:
$ ls output/
part-00000.avro _SUCCESS
You can check the contents of the generated Avro file using the avro-tools jar:
$ java -jar /path/to/avro-tools-1.7.7.jar tojson output/part-00000.avro
{"value": 3, "key": "blue"}
{"value": 7, "key": "green"}
{"value": 1, "key": "none"}
{"value": 2, "key": "orange"}
{"value": 3, "key": "purple"}
{"value": 2, "key": "red"}
{"value": 2, "key": "yellow"}
Now let’s go over the ColorCount example in detail.
Mapper - org.apache.hadoop.mapred API
The easiest way to use Avro data files as input to a MapReduce job is to subclass AvroMapper. An AvroMapper defines a map function that takes an Avro datum as input and outputs a key/value pair represented as a Pair record. In the ColorCount example, ColorCountMapper is an AvroMapper that takes a User as input and outputs a Pair<CharSequence, Integer>>, where the CharSequence key is the user’s favorite color and the Integer value is 1.
public static class ColorCountMapper extends AvroMapper<User, Pair<CharSequence, Integer>> {
@Override
public void map(User user, AvroCollector<Pair<CharSequence, Integer>> collector, Reporter reporter)
throws IOException {
CharSequence color = user.getFavoriteColor();
// We need this check because the User.favorite_color field has type ["string", "null"]
if (color == null) {
color = "none";
}
collector.collect(new Pair<CharSequence, Integer>(color, 1));
}
}
In order to use our AvroMapper, we must call AvroJob.setMapperClass and AvroJob.setInputSchema.
AvroJob.setMapperClass(conf, ColorCountMapper.class);
AvroJob.setInputSchema(conf, User.getClassSchema());
Note that AvroMapper does not implement the Mapper interface. Under the hood, the specified Avro data files are deserialized into AvroWrappers containing the actual data, which are processed by a Mapper that calls the configured AvroMapper’s map function. AvroJob.setInputSchema sets up the relevant configuration parameters needed to make this happen, thus you should not need to call JobConf.setMapperClass, JobConf.setInputFormat, JobConf.setMapOutputKeyClass, JobConf.setMapOutputValueClass, or JobConf.setOutputKeyComparatorClass.
Mapper - org.apache.hadoop.mapreduce API
This document will not go into all the differences between the mapred and mapreduce APIs, however will describe the main differences. As you can see, ColorCountMapper is now a subclass of the Hadoop Mapper class and is passed an AvroKey as it’s key. Additionally, the AvroJob method calls were slightly changed.
public static class ColorCountMapper extends
Mapper<AvroKey<User>, NullWritable, Text, IntWritable> {
@Override
public void map(AvroKey<User> key, NullWritable value, Context context)
throws IOException, InterruptedException {
CharSequence color = key.datum().getFavoriteColor();
if (color == null) {
color = "none";
}
context.write(new Text(color.toString()), new IntWritable(1));
}
}
Reducer - org.apache.hadoop.mapred API
Analogously to AvroMapper, an AvroReducer defines a reducer function that takes the key/value types output by an AvroMapper (or any mapper that outputs Pairs) and outputs a key/value pair represented a Pair record. In the ColorCount example, ColorCountReducer is an AvroReducer that takes the CharSequence key representing a favorite color and the Iterable<Integer> representing the counts for that color (they should all be 1 in this example) and adds up the counts.
public static class ColorCountReducer extends AvroReducer<CharSequence, Integer,
Pair<CharSequence, Integer>> {
@Override
public void reduce(CharSequence key, Iterable<Integer> values,
AvroCollector<Pair<CharSequence, Integer>> collector,
Reporter reporter)
throws IOException {
int sum = 0;
for (Integer value : values) {
sum += value;
}
collector.collect(new Pair<CharSequence, Integer>(key, sum));
}
}
In order to use our AvroReducer, we must call AvroJob.setReducerClass and AvroJob.setOutputSchema.
AvroJob.setReducerClass(conf, ColorCountReducer.class);
AvroJob.setOutputSchema(conf, Pair.getPairSchema(Schema.create(Type.STRING),
Schema.create(Type.INT)));
Note that AvroReducer does not implement the Reducer interface. The intermediate Pairs output by the mapper are split into AvroKeys and AvroValues, which are processed by a Reducer that calls the configured AvroReducer’s reduce function. AvroJob.setOutputSchema sets up the relevant configuration parameters needed to make this happen, thus you should not need to call JobConf.setReducerClass, JobConf.setOutputFormat, JobConf.setOutputKeyClass, JobConf.setMapOutputKeyClass, JobConf.setMapOutputValueClass, or JobConf.setOutputKeyComparatorClass.
Reduce - org.apache.hadoop.mapreduce API
As before we not detail every difference between the APIs. As with the Mapper change ColorCountReducer is now a subclass of Reducer and AvroKey and AvroValue are emitted. Additionally, the AvroJob method calls were slightly changed.
public static class ColorCountReducer extends
Reducer<Text, IntWritable, AvroKey<CharSequence>, AvroValue<Integer>> {
@Override
public void reduce(Text key, Iterable<IntWritable> values,
Context context) throws IOException, InterruptedException {
int sum = 0;
for (IntWritable value : values) {
sum += value.get();
}
context.write(new AvroKey<CharSequence>(key.toString()), new AvroValue<Integer>(sum));
}
}
Learning more
The mapred API allows users to mix Avro AvroMappers and AvroReducers with non-Avro Mappers and Reducers and the mapreduce API allows users input Avro and output non-Avro or vice versa.
The mapred package has API org.apache.avro.mapred documentation as does the org.apache.avro.mapreduce package. MapReduce API (org.apache.hadoop.mapreduce). Similarily to the mapreduce package, it’s possible with the mapred API to implement your own Mappers and Reducers directly using the public classes provided in these libraries. See the AvroWordCount application, found under examples/mr-example/src/main/java/example/AvroWordCount.java in the Avro documentation, for an example of implementing a Reducer that outputs Avro data using the old MapReduce API. See the MapReduceAvroWordCount application, found under examples/mr-example/src/main/java/example/MapReduceAvroWordCount.java in the Avro documentation, for an example of implementing a Reducer that outputs Avro data using the new MapReduce API.
12.6 - IDL Language
Introduction
This document defines Avro IDL, a higher-level language for authoring Avro schemata. Before reading this document, you should have familiarity with the concepts of schemata and protocols, as well as the various primitive and complex types available in Avro.
Overview
Purpose
The aim of the Avro IDL language is to enable developers to author schemata in a way that feels more similar to common programming languages like Java, C++, or Python. Additionally, the Avro IDL language may feel more familiar for those users who have previously used the interface description languages (IDLs) in other frameworks like Thrift, Protocol Buffers, or CORBA.
Usage
Each Avro IDL file defines a single Avro Protocol, and thus generates as its output a JSON-format Avro Protocol file with extension .avpr.
To convert a .avdl file into a .avpr file, it may be processed by the idl tool. For example:
$ java -jar avroj-tools.jar idl src/test/idl/input/namespaces.avdl /tmp/namespaces.avpr
$ head /tmp/namespaces.avpr
{
"protocol" : "TestNamespace",
"namespace" : "avro.test.protocol",
The idl tool can also process input to and from stdin and stdout. See idl --help for full usage information.
A Maven plugin is also provided to compile .avdl files. To use it, add something like the following to your pom.xml:
<build>
<plugins>
<plugin>
<groupId>org.apache.avro</groupId>
<artifactId>avro-maven-plugin</artifactId>
<executions>
<execution>
<goals>
<goal>idl-protocol</goal>
</goals>
</execution>
</executions>
</plugin>
</plugins>
</build>
Defining a Protocol in Avro IDL
An Avro IDL file consists of exactly one protocol definition. The minimal protocol is defined by the following code:
This is equivalent to (and generates) the following JSON protocol definition:
{
"protocol" : "MyProtocol",
"types" : [ ],
"messages" : {
}
}
The namespace of the protocol may be changed using the @namespace annotation:
@namespace("mynamespace")
protocol MyProtocol {
}
This notation is used throughout Avro IDL as a way of specifying properties for the annotated element, as will be described later in this document.
Protocols in Avro IDL can contain the following items:
- Imports of external protocol and schema files.
- Definitions of named schemata, including records, errors, enums, and fixeds.
- Definitions of RPC messages
Imports
Files may be imported in one of three formats:
-
An IDL file may be imported with a statement like:
import idl "foo.avdl";
-
A JSON protocol file may be imported with a statement like:
import protocol "foo.avpr";
-
A JSON schema file may be imported with a statement like:
import schema "foo.avsc";
Messages and types in the imported file are added to this file’s protocol.
Imported file names are resolved relative to the current IDL file.
Defining an Enumeration
Enums are defined in Avro IDL using a syntax similar to C or Java:
enum Suit {
SPADES, DIAMONDS, CLUBS, HEARTS
}
Note that, unlike the JSON format, anonymous enums cannot be defined.
Defining a Fixed Length Field
Fixed fields are defined using the following syntax:
fixed MD5(16);
This example defines a fixed-length type called MD5 which contains 16 bytes.
Defining Records and Errors
Records are defined in Avro IDL using a syntax similar to a struct definition in C:
record Employee {
string name;
boolean active = true;
long salary;
}
The above example defines a record with the name “Employee” with three fields.
To define an error, simply use the keyword error instead of record. For example:
error Kaboom {
string explanation;
int result_code = -1;
}
Each field in a record or error consists of a type and a name, optional property annotations and an optional default value.
A type reference in Avro IDL must be one of:
- A primitive type
- A logical type
- A named schema defined prior to this usage in the same Protocol
- A complex type (array, map, or union)
Primitive Types
The primitive types supported by Avro IDL are the same as those supported by Avro’s JSON format. This list includes int, long, string, boolean, float, double, null, and bytes.
References to Named Schemata
If a named schema has already been defined in the same Avro IDL file, it may be referenced by name as if it were a primitive type:
record Card {
Suit suit; // refers to the enum Card defined above
int number;
}
Default Values
Default values for fields may be optionally specified by using an equals sign after the field name followed by a JSON expression indicating the default value. This JSON is interpreted as described in the spec.
Complex Types
Arrays
Array types are written in a manner that will seem familiar to C++ or Java programmers. An array of any type t is denoted array<t>. For example, an array of strings is denoted array<string>, and a multidimensional array of Foo records would be array<array<Foo>>.
Maps
Map types are written similarly to array types. An array that contains values of type t is written map<t>. As in the JSON schema format, all maps contain string-type keys.
Unions
Union types are denoted as union { typeA, typeB, typeC, … }. For example, this record contains a string field that is optional (unioned with null):
record RecordWithUnion {
union { null, string } optionalString;
}
Note that the same restrictions apply to Avro IDL unions as apply to unions defined in the JSON format; namely, a record may not contain multiple elements of the same type.
Defining RPC Messages
The syntax to define an RPC message within a Avro IDL protocol is similar to the syntax for a method declaration within a C header file or a Java interface. To define an RPC message add which takes two arguments named foo and bar, returning an int, simply include the following definition within the protocol:
int add(int foo, int bar = 0);
Message arguments, like record fields, may specify default values.
To define a message with no response, you may use the alias void, equivalent to the Avro null type:
void logMessage(string message);
If you have previously defined an error type within the same protocol, you may declare that a message can throw this error using the syntax:
void goKaboom() throws Kaboom;
To define a one-way message, use the keyword oneway after the parameter list, for example:
void fireAndForget(string message) oneway;
Other Language Features
All Java-style comments are supported within a Avro IDL file. Any text following // on a line is ignored, as is any text between /* and */, possibly spanning multiple lines.
Comments that begin with /** are used as the documentation string for the type or field definition that follows the comment.
Escaping Identifiers
Occasionally, one will need to use a reserved language keyword as an identifier. In order to do so, backticks (`) may be used to escape the identifier. For example, to define a message with the literal name error, you may write:
This syntax is allowed anywhere an identifier is expected.
Annotations for Ordering and Namespaces
Java-style annotations may be used to add additional properties to types and fields throughout Avro IDL.
For example, to specify the sort order of a field within a record, one may use the @order annotation before the field name as follows:
record MyRecord {
string @order("ascending") myAscendingSortField;
string @order("descending") myDescendingField;
string @order("ignore") myIgnoredField;
}
A field’s type may also be preceded by annotations, e.g.:
record MyRecord {
@java-class("java.util.ArrayList") array<string> myStrings;
}
This can be used to support java classes that can be serialized/deserialized via their toString/String constructor, e.g.:
record MyRecord {
@java-class("java.math.BigDecimal") string value;
@java-key-class("java.io.File") map<string> fileStates;
array<@java-class("java.math.BigDecimal") string> weights;
}
Similarly, a @namespace annotation may be used to modify the namespace when defining a named schema. For example:
@namespace("org.apache.avro.firstNamespace")
protocol MyProto {
@namespace("org.apache.avro.someOtherNamespace")
record Foo {}
record Bar {}
}
will define a protocol in the firstNamespace namespace. The record Foo will be defined in someOtherNamespace and Bar will be defined in firstNamespace as it inherits its default from its container.
Type and field aliases are specified with the @aliases annotation as follows:
@aliases(["org.old.OldRecord", "org.ancient.AncientRecord"])
record MyRecord {
string @aliases(["oldField", "ancientField"]) myNewField;
}
Some annotations like those listed above are handled specially. All other annotations are added as properties to the protocol, message, schema or field.
Complete Example
The following is a complete example of a Avro IDL file that shows most of the above features:
/**
* An example protocol in Avro IDL
*/
@namespace("org.apache.avro.test")
protocol Simple {
@aliases(["org.foo.KindOf"])
enum Kind {
FOO,
BAR, // the bar enum value
BAZ
}
fixed MD5(16);
record TestRecord {
@order("ignore")
string name;
@order("descending")
Kind kind;
MD5 hash;
union { MD5, null} @aliases(["hash"]) nullableHash;
array<long> arrayOfLongs;
}
error TestError {
string message;
}
string hello(string greeting);
TestRecord echo(TestRecord `record`);
int add(int arg1, int arg2);
bytes echoBytes(bytes data);
void `error`() throws TestError;
void ping() oneway;
}
Additional examples may be found in the Avro source tree under the src/test/idl/input directory.
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
12.7 - SASL profile
Introduction
SASL (RFC 2222) provides a framework for authentication and security of network protocols. Each protocol that uses SASL is meant to define a SASL profile. This document provides a SASL profile for connection-based Avro RPC.
Overview
SASL negotiation proceeds as a series of message interactions over a connection between a client and server using a selected SASL mechanism. The client starts this negotiation by sending its chosen mechanism name with an initial (possibly empty) message. Negotiation proceeds with the exchange of messages until either side indicates success or failure. The content of the messages is mechanism-specific. If the negotiation succeeds, then the session can proceed over the connection, otherwise it must be abandoned.
Some mechanisms continue to process session data after negotiation (e.g., encrypting it), while some specify that further session data is transmitted unmodified.
Negotiation
Commands
Avro SASL negotiation uses four one-byte commands.
- 0: START Used in a client’s initial message.
- 1: CONTINUE Used while negotiation is ongoing.
- 2: FAIL Terminates negotiation unsuccessfully.
- 3: COMPLETE Terminates negotiation successfully.
The format of a START message is:
| 0 | 4-byte mechanism name length | mechanism name | 4-byte payload length | payload data |
The format of a CONTINUE message is:
| 1 | 4-byte payload length | payload data |
The format of a FAIL message is:
| 2 | 4-byte message length | UTF-8 message |
The format of a COMPLETE message is:
| 3 | 4-byte payload length | payload data |
Process
Negotiation is initiated by a client sending a START command containing the client’s chosen mechanism name and any mechanism-specific payload data.
The server and client then interchange some number (possibly zero) of CONTINUE messages. Each message contains payload data that is processed by the security mechanism to generate the next message.
Once either the client or server send a FAIL message then negotiation has failed. UTF-8-encoded text is included in the failure message. Once either a FAIL message has been sent or received, or any other error occurs in the negotiation, further communication on this connection must cease.
Once either the client or server send a COMPLETE message then negotiation has completed successfully. Session data may now be transmitted over the connection until it is closed by either side.
Session Data
If no SASL QOP (quality of protection) is negotiated, then all subsequent writes to/reads over this connection are written/read unmodified. In particular, messages use Avro framing, and are of the form:
| 4-byte frame length | frame data | ... | 4 zero bytes |
If a SASL QOP is negotiated, then it must be used by the connection for all subsequent messages. This is done by wrapping each non-empty frame written using the security mechanism and unwrapping each non-empty frame read. The length written in each non-empty frame is the length of the wrapped data. Complete frames must be passed to the security mechanism for unwrapping. Unwrapped data is then passed to the application as the content of the frame.
If at any point processing fails due to wrapping, unwrapping or framing errors, then all further communication on this connection must cease.
Anonymous Mechanism
The SASL anonymous mechanism (RFC 2245) is quite simple to implement. In particular, an initial anonymous request may be prefixed by the following static sequence:
| 0 | 0009 | ANONYMOUS | 0000 |
If a server uses the anonymous mechanism, it should check that the mechanism name in the start message prefixing the first request received is ‘ANONYMOUS’, then simply prefix its initial response with a COMPLETE message of:
| 3 | 0000 |
If an anonymous server recieves some other mechanism name, then it may respond with a FAIL message as simple as:
| 2 | 0000 |
Note that the anonymous mechanism need add no additional round-trip messages between client and server. The START message can be piggybacked on the initial request and the COMPLETE or FAIL message can be piggybacked on the initial response.
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
12.8 - Wiki
The Wiki page can be found here.
12.9 - FAQ
What is Avro?
Avro is a data serialization system.
Avro provides:
- Rich data structures.
- A compact, fast, binary data format.
- A container file, to store persistent data.
- Remote procedure call (RPC).
- Simple integration with dynamic languages. Code generation is not required to read or write data files nor to use or implement RPC protocols. Code generation as an optional optimization, only worth implementing for statically typed languages.
How can I get started quickly with Avro?
Check out the Quick Start Guide.
How do I statically compile a schema or protocol into generated code?
In Java
- Add the avro jar, the jackson-mapper-asl.jar and jackson-core-asl.jar to your CLASSPATH.
- Run java org.apache.avro.specific.SpecificCompiler .
This appears to be out of date, the SpecificCompiler requires two arguments, presumably an input and and output file, but it isn’t clear that this does.
Or use the Schema or Protocol Ant tasks. Avro’s build.xml provides examples of how these are used.
Lastly, you can also use the “avro-tools” jar which ships with an Avro release. Just use the “compile (schema|protocol)” command.
How are Strings represented in Java?
They use org.apache.avro.util.Utf8, not java.lang.String.
More generally, how do Avro types map to Java types?
The mappings are documented in the package javadoc for generic, specific and reflect API.
From Doug Cutting:
HDFS splits files into blocks, and mapreduce runs a map task for each block. When the task starts, it needs to be able to seek into the file to the start of the block process through the block’s end. If the file were, e.g., a gzip file, this would not be possible, since gzip files must be decompressed from the start. One cannot seek into the middle of a gzip file and start decompressing. So Hadoop’s SequenceFile places a marker periodically (~64k) in the file at record and compression boundaries, where processing can be sensibly started. Then, when a map task starts processing an HDFS block, it finds the first marker after the block’s start and continues through the first marker in the next block of the file. This requires a bit of non-local access (~0.1%). Avro’s data file uses the same method as SequenceFile.
Why isn’t every value in Avro nullable?
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 of every value stored and
corresponding computational costs to create this bit on write and
interpret it on read. These costs are wasted when values may not in
fact be null, as is the case in many datasets. In Avro such costs are
only paid when values may actually be null.
Also, allowing values to be null is a well-known source of errors. In
Avro, a value declared as non-null will always be non-null and programs
need not test for null values when processing it nor will they ever fail
for lack of such tests.
Tony Hoare calls his invention of null references his “Billion Dollar
Mistake”.
http://qconlondon.com/london-2009/presentation/Null+References:+The+Billion+Dollar+Mistake
Also note that in some programming languages not all values are permitted to be null. For example, in Java, values of type boolean, byte, short, char, int, float, long, and double may not be null.
How can I serialize directly to/from a byte array?
As pointed out in the specification, Avro data should always be stored with its schema. The Avro provided classes DataFileWriter, DataFileReader, and DataFileStream all ensure this by serializing the Schema in a container header. In some special cases, such as when implementing a new storage system or writing unit tests, you may need to write and read directly with the bare Avro serialized values. The following examples use code generated for Java from an Avro Schema, specifically the User example from the Quickstart guide.
Serializing to a byte array
This example takes a User object and returns a newly allocated byte array with the Avro serialization of that user.
ByteArrayOutputStream out = new ByteArrayOutputStream();
BinaryEncoder encoder = EncoderFactory.get().binaryEncoder(out, null);
DatumWriter<User> writer = new SpecificDatumWriter<User>(User.getClassSchema());
writer.write(user, encoder);
encoder.flush();
out.close();
byte[] serializedBytes = out.toByteArray();
Deserializing from a byte array
This example takes a byte array containing the Avro serialization of a user and returns a User object.SpecificDatumReader reader = new SpecificDatumReader(User.getClassSchema());
Decoder decoder = DecoderFactory.get().binaryDecoder(bytes, null);
User user = reader.read(null, decoder);
Note: If you are serializing or deserializing in a loop or as a method, you should be reusing objects, readers and/or writers. Check the JavaDoc to see how to reuse the objects.
13 - Apache Avro™ 1.7.6 Documentation
Introduction
Apache Avro™ is a data serialization system.
Avro provides:
- Rich data structures.
- A compact, fast, binary data format.
- A container file, to store persistent data.
- Remote procedure call (RPC).
- Simple integration with dynamic languages. Code generation is not required to read or write data files nor to use or implement RPC protocols. Code generation as an optional optimization, only worth implementing for statically typed languages.
Schemas
Avro relies on schemas. When Avro data is read, the schema used when writing it is always present. This permits each datum to be written with no per-value overheads, making serialization both fast and small. This also facilitates use with dynamic, scripting languages, since data, together with its schema, is fully self-describing.
When Avro data is stored in a file, its schema is stored with it, so that files may be processed later by any program. If the program reading the data expects a different schema this can be easily resolved, since both schemas are present.
When Avro is used in RPC, the client and server exchange schemas in the connection handshake. (This can be optimized so that, for most calls, no schemas are actually transmitted.) Since both client and server both have the other’s full schema, correspondence between same named fields, missing fields, extra fields, etc. can all be easily resolved.
Avro schemas are defined with JSON . This facilitates implementation in languages that already have JSON libraries.
Comparison with other systems
Avro provides functionality similar to systems such as Thrift, Protocol Buffers, etc. Avro differs from these systems in the following fundamental aspects.
- Dynamic typing: Avro does not require that code be generated. Data is always accompanied by a schema that permits full processing of that data without code generation, static datatypes, etc. This facilitates construction of generic data-processing systems and languages.
- Untagged data: Since the schema is present when data is read, considerably less type information need be encoded with data, resulting in smaller serialization size.
- No manually-assigned field IDs: When a schema changes, both the old and new schema are always present when processing data, so differences may be resolved symbolically, using field names.
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
13.1 - Getting Started (Java)
This is a short guide for getting started with Apache Avro™ using Java. This guide only covers using Avro for data serialization; see Patrick Hunt’s Avro RPC Quick Start for a good introduction to using Avro for RPC.
Download
Avro implementations for C, C++, C#, Java, PHP, Python, and Ruby can be downloaded from the Apache Avro™ Releases page. This guide uses Avro 1.7.6, the latest version at the time of writing. For the examples in this guide, download avro-1.7.6.jar and avro-tools-1.7.6.jar. The Avro Java implementation also depends on the Jackson JSON library. From the Jackson download page, download the core-asl and mapper-asl jars. Add avro-1.7.6.jar and the Jackson jars to your project’s classpath (avro-tools will be used for code generation).
Alternatively, if you are using Maven, add the following dependency to your POM:
<dependency>
<groupId>org.apache.avro</groupId>
<artifactId>avro</artifactId>
<version>1.7.6</version>
</dependency>
As well as the Avro Maven plugin (for performing code generation):
<plugin>
<groupId>org.apache.avro</groupId>
<artifactId>avro-maven-plugin</artifactId>
<version>1.7.6</version>
<executions>
<execution>
<phase>generate-sources</phase>
<goals>
<goal>schema</goal>
</goals>
<configuration>
<sourceDirectory>${project.basedir}/src/main/avro/</sourceDirectory>
<outputDirectory>${project.basedir}/src/main/java/</outputDirectory>
</configuration>
</execution>
</executions>
</plugin>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-compiler-plugin</artifactId>
<configuration>
<source>1.6</source>
<target>1.6</target>
</configuration>
</plugin>
You may also build the required Avro jars from source. Building Avro is beyond the scope of this guide; see the Build Documentation page in the wiki for more information.
Defining a schema
Avro schemas are defined using JSON. Schemas are composed of primitive types (null, boolean, int, long, float, double, bytes, and string) and complex types (record, enum, array, map, union, and fixed). You can learn more about Avro schemas and types from the specification, but for now let’s start with a simple schema example, user.avsc:
{"namespace": "example.avro",
"type": "record",
"name": "User",
"fields": [
{"name": "name", "type": "string"},
{"name": "favorite_number", "type": ["int", "null"]},
{"name": "favorite_color", "type": ["string", "null"]}
]
}
This schema defines a record representing a hypothetical user. (Note that a schema file can only contain a single schema definition.) At minimum, a record definition must include its type (“type”: “record”), a name (“name”: “User”), and fields, in this case name, favorite_number, and favorite_color. We also define a namespace (“namespace”: “example.avro”), which together with the name attribute defines the “full name” of the schema (example.avro.User in this case).
Fields are defined via an array of objects, each of which defines a name and type (other attributes are optional, see the record specification for more details). The type attribute of a field is another schema object, which can be either a primitive or complex type. For example, the name field of our User schema is the primitive type string, whereas the favorite_number and favorite_color fields are both unions, represented by JSON arrays. unions are a complex type that can be any of the types listed in the array; e.g., favorite_number can either be an int or null, essentially making it an optional field.
Serializing and deserializing with code generation
Compiling the schema
Code generation allows us to automatically create classes based on our previously-defined schema. Once we have defined the relevant classes, there is no need to use the schema directly in our programs. We use the avro-tools jar to generate code as follows:
java -jar /path/to/avro-tools-1.7.6.jar compile schema <schema file> <destination>
This will generate the appropriate source files in a package based on the schema’s namespace in the provided destination folder. For instance, to generate a User class in package example.avro from the schema defined above, run
java -jar /path/to/avro-tools-1.7.6.jar compile schema user.avsc .
Note that if you using the Avro Maven plugin, there is no need to manually invoke the schema compiler; the plugin automatically performs code generation on any .avsc files present in the configured source directory.
Creating Users
Now that we’ve completed the code generation, let’s create some Users, serialize them to a data file on disk, and then read back the file and deserialize the User objects.
First let’s create some Users and set their fields.
User user1 = new User();
user1.setName("Alyssa");
user1.setFavoriteNumber(256);
// Leave favorite color null
// Alternate constructor
User user2 = new User("Ben", 7, "red");
// Construct via builder
User user3 = User.newBuilder()
.setName("Charlie")
.setFavoriteColor("blue")
.setFavoriteNumber(null)
.build();
As shown in this example, Avro objects can be created either by invoking a constructor directly or by using a builder. Unlike constructors, builders will automatically set any default values specified in the schema. Additionally, builders validate the data as it set, whereas objects constructed directly will not cause an error until the object is serialized. However, using constructors directly generally offers better performance, as builders create a copy of the datastructure before it is written.
Note that we do not set user1’s favorite color. Since that record is of type [“string”, “null”], we can either set it to a string or leave it null; it is essentially optional. Similarly, we set user3’s favorite number to null (using a builder requires setting all fields, even if they are null).
Serializing
Now let’s serialize our Users to disk.
/// Serialize user1, user2 and user3 to disk
DatumWriter<User> userDatumWriter = new SpecificDatumWriter<User>(User.class);
DataFileWriter<User> dataFileWriter = new DataFileWriter<User>(userDatumWriter);
dataFileWriter.create(user1.getSchema(), new File("users.avro"));
dataFileWriter.append(user1);
dataFileWriter.append(user2);
dataFileWriter.append(user3);
dataFileWriter.close();
We create a DatumWriter, which converts Java objects into an in-memory serialized format. The SpecificDatumWriter class is used with generated classes and extracts the schema from the specified generated type.
Next we create a DataFileWriter, which writes the serialized records, as well as the schema, to the file specified in the dataFileWriter.create call. We write our users to the file via calls to the dataFileWriter.append method. When we are done writing, we close the data file.
Deserializing
Finally, let’s deserialize the data file we just created.
// Deserialize Users from disk
DatumReader<User> userDatumReader = new SpecificDatumReader<User>(User.class);
DataFileReader<User> dataFileReader = new DataFileReader<User>(file, userDatumReader);
User user = null;
while (dataFileReader.hasNext()) {
// Reuse user object by passing it to next(). This saves us from
// allocating and garbage collecting many objects for files with
// many items.
user = dataFileReader.next(user);
System.out.println(user);
}
This snippet will output:
{"name": "Alyssa", "favorite_number": 256, "favorite_color": null}
{"name": "Ben", "favorite_number": 7, "favorite_color": "red"}
{"name": "Charlie", "favorite_number": null, "favorite_color": "blue"}
Deserializing is very similar to serializing. We create a SpecificDatumReader, analogous to the SpecificDatumWriter we used in serialization, which converts in-memory serialized items into instances of our generated class, in this case User. We pass the DatumReader and the previously created File to a DataFileReader, analogous to the DataFileWriter, which reads the data file on disk.
Next we use the DataFileReader to iterate through the serialized Users and print the deserialized object to stdout. Note how we perform the iteration: we create a single User object which we store the current deserialized user in, and pass this record object to every call of dataFileReader.next. This is a performance optimization that allows the DataFileReader to reuse the same User object rather than allocating a new User for every iteration, which can be very expensive in terms of object allocation and garbage collection if we deserialize a large data file. While this technique is the standard way to iterate through a data file, it’s also possible to use for (User user : dataFileReader) if performance is not a concern.
Compiling and running the example code
This example code is included as a Maven project in the examples/java-example directory in the Avro docs. From this directory, execute the following commands to build and run the example:
$ mvn compile # includes code generation via Avro Maven plugin
$ mvn -q exec:java -Dexec.mainClass=example.SpecificMain
Serializing and deserializing without code generation
Data in Avro is always stored with its corresponding schema, meaning we can always read a serialized item regardless of whether we know the schema ahead of time. This allows us to perform serialization and deserialization without code generation.
Let’s go over the same example as in the previous section, but without using code generation: we’ll create some users, serialize them to a data file on disk, and then read back the file and deserialize the users objects.
Creating users
First, we use a Parser to read our schema definition and create a Schema object.
Schema schema = new Parser().parse(new File("user.avsc"));
Using this schema, let’s create some users.
GenericRecord user1 = new GenericData.Record(schema);
user1.put("name", "Alyssa");
user1.put("favorite_number", 256);
// Leave favorite color null
GenericRecord user2 = new GenericData.Record(schema);
user2.put("name", "Ben");
user2.put("favorite_number", 7);
user2.put("favorite_color", "red");
Since we’re not using code generation, we use GenericRecords to represent users. GenericRecord uses the schema to verify that we only specify valid fields. If we try to set a non-existent field (e.g., user1.put(“favorite_animal”, “cat”)), we’ll get an AvroRuntimeException when we run the program.
Note that we do not set user1’s favorite color. Since that record is of type [“string”, “null”], we can either set it to a string or leave it null; it is essentially optional.
Serializing
Now that we’ve created our user objects, serializing and deserializing them is almost identical to the example above which uses code generation. The main difference is that we use generic instead of specific readers and writers.
First we’ll serialize our users to a data file on disk.
// Serialize user1 and user2 to disk
File file = new File("users.avro");
DatumWriter<GenericRecord> datumWriter = new GenericDatumWriter<GenericRecord>(schema);
DataFileWriter<GenericRecord> dataFileWriter = new DataFileWriter<GenericRecord>(datumWriter);
dataFileWriter.create(schema, file);
dataFileWriter.append(user1);
dataFileWriter.append(user2);
dataFileWriter.close();
We create a DatumWriter, which converts Java objects into an in-memory serialized format. Since we are not using code generation, we create a GenericDatumWriter. It requires the schema both to determine how to write the GenericRecords and to verify that all non-nullable fields are present.
As in the code generation example, we also create a DataFileWriter, which writes the serialized records, as well as the schema, to the file specified in the dataFileWriter.create call. We write our users to the file via calls to the dataFileWriter.append method. When we are done writing, we close the data file.
Deserializing
Finally, we’ll deserialize the data file we just created.
// Deserialize users from disk
DatumReader<GenericRecord> datumReader = new GenericDatumReader<GenericRecord>(schema);
DataFileReader<GenericRecord> dataFileReader = new DataFileReader<GenericRecord>(file, datumReader);
GenericRecord user = null;
while (dataFileReader.hasNext()) {
// Reuse user object by passing it to next(). This saves us from
// allocating and garbage collecting many objects for files with
// many items.
user = dataFileReader.next(user);
System.out.println(user);
This outputs:
{"name": "Alyssa", "favorite_number": 256, "favorite_color": null}
{"name": "Ben", "favorite_number": 7, "favorite_color": "red"}
Deserializing is very similar to serializing. We create a GenericDatumReader, analogous to the GenericDatumWriter we used in serialization, which converts in-memory serialized items into GenericRecords. We pass the DatumReader and the previously created File to a DataFileReader, analogous to the DataFileWriter, which reads the data file on disk.
Next, we use the DataFileReader to iterate through the serialized users and print the deserialized object to stdout. Note how we perform the iteration: we create a single GenericRecord object which we store the current deserialized user in, and pass this record object to every call of dataFileReader.next. This is a performance optimization that allows the DataFileReader to reuse the same record object rather than allocating a new GenericRecord for every iteration, which can be very expensive in terms of object allocation and garbage collection if we deserialize a large data file. While this technique is the standard way to iterate through a data file, it’s also possible to use for (GenericRecord user : dataFileReader) if performance is not a concern.
Compiling and running the example code
This example code is included as a Maven project in the examples/java-example directory in the Avro docs. From this directory, execute the following commands to build and run the example:
$ mvn compile
$ mvn -q exec:java -Dexec.mainClass=example.GenericMain
13.2 - Getting Started (Python)
This is a short guide for getting started with Apache Avro™ using Python. This guide only covers using Avro for data serialization; see Patrick Hunt’s Avro RPC Quick Start for a good introduction to using Avro for RPC.
Download
Avro implementations for C, C++, C#, Java, PHP, Python, and Ruby can be downloaded from the Apache Avro™ Releases page. This guide uses Avro 1.7.6, the latest version at the time of writing. Download and unzip avro-1.7.6.tar.gz, and install via python setup.py (this will probably require root privileges). Ensure that you can import avro from a Python prompt.
$ tar xvf avro-1.7.6.tar.gz
$ cd avro-1.7.6
$ sudo python setup.py install
$ python
>>> import avro # should not raise ImportError
Alternatively, you may build the Avro Python library from source. From your the root Avro directory, run the commands
$ cd lang/py/
$ ant
$ sudo python setup.py install
$ python
>>> import avro # should not raise ImportError
Defining a schema
Avro schemas are defined using JSON. Schemas are composed of primitive types (null, boolean, int, long, float, double, bytes, and string) and complex types (record, enum, array, map, union, and fixed). You can learn more about Avro schemas and types from the specification, but for now let’s start with a simple schema example, user.avsc:
{"namespace": "example.avro",
"type": "record",
"name": "User",
"fields": [
{"name": "name", "type": "string"},
{"name": "favorite_number", "type": ["int", "null"]},
{"name": "favorite_color", "type": ["string", "null"]}
]
}
This schema defines a record representing a hypothetical user. (Note that a schema file can only contain a single schema definition.) At minimum, a record definition must include its type (“type”: “record”), a name (“name”: “User”), and fields, in this case name, favorite_number, and favorite_color. We also define a namespace (“namespace”: “example.avro”), which together with the name attribute defines the “full name” of the schema (example.avro.User in this case).
Fields are defined via an array of objects, each of which defines a name and type (other attributes are optional, see the record specification for more details). The type attribute of a field is another schema object, which can be either a primitive or complex type. For example, the name field of our User schema is the primitive type string, whereas the favorite_number and favorite_color fields are both unions, represented by JSON arrays. unions are a complex type that can be any of the types listed in the array; e.g., favorite_number can either be an int or null, essentially making it an optional field.
Serializing and deserializing without code generation
Data in Avro is always stored with its corresponding schema, meaning we can always read a serialized item, regardless of whether we know the schema ahead of time. This allows us to perform serialization and deserialization without code generation. Note that the Avro Python library does not support code generation.
Try running the following code snippet, which serializes two users to a data file on disk, and then reads back and deserializes the data file:
import avro.schema
from avro.datafile import DataFileReader, DataFileWriter
from avro.io import DatumReader, DatumWriter
schema = avro.schema.parse(open("user.avsc").read())
writer = DataFileWriter(open("users.avro", "w"), DatumWriter(), schema)
writer.append({"name": "Alyssa", "favorite_number": 256})
writer.append({"name": "Ben", "favorite_number": 7, "favorite_color": "red"})
writer.close()
reader = DataFileReader(open("users.avro", "r"), DatumReader())
for user in reader:
print user
reader.close()
This outputs:
{u'favorite_color': None, u'favorite_number': 256, u'name': u'Alyssa'}
{u'favorite_color': u'red', u'favorite_number': 7, u'name': u'Ben'}
Let’s take a closer look at what’s going on here.
schema = avro.schema.parse(open("user.avsc").read())
avro.schema.parse takes a string containing a JSON schema definition as input and outputs a avro.schema.Schema object (specifically a subclass of Schema, in this case RecordSchema). We’re passing in the contents of our user.avsc schema file here.
writer = DataFileWriter(open("users.avro", "w"), DatumWriter(), schema)
We create a DataFileWriter, which we’ll use to write serialized items to a data file on disk. The DataFileWriter constructor takes three arguments:
- The file we’ll serialize to
- A DatumWriter, which is responsible for actually serializing the items to Avro’s binary format (DatumWriters can be used separately from DataFileWriters, e.g., to perform IPC with Avro TODO: is this true??).
- The schema we’re using. The DataFileWriter needs the schema both to write the schema to the data file, and to verify that the items we write are valid items and write the appropriate fields.
writer.append({"name": "Alyssa", "favorite_number": 256})
writer.append({"name": "Ben", "favorite_number": 7, "favorite_color": "red"})
We use DataFileWriter.append to add items to our data file. Avro records are represented as Python dicts. Since the field favorite_color has type [“int”, “null”], we are not required to specify this field, as shown in the first append. Were we to omit the required name field, an exception would be raised. Any extra entries not corresponding to a field are present in the dict are ignored.
reader = DataFileReader(open("users.avro", "r"), DatumReader())
We open the file again, this time for reading back from disk. We use a DataFileReader and DatumReader analagous to the DataFileWriter and DatumWriter above.
for user in reader:
print user
The DataFileReader is an iterator that returns dicts corresponding to the serialized items.
13.3 - Specification
Introduction
This document defines Apache Avro. It is intended to be the authoritative specification. Implementations of Avro must adhere to this document.
Schema Declaration
A Schema is represented in JSON by one of:
- A JSON string, naming a defined type.
- A JSON object, of the form:
{"type": "typeName" ...attributes...}
where typeName is either a primitive or derived type name, as defined below. Attributes not defined in this document are permitted as metadata, but must not affect the format of serialized data.
- A JSON array, representing a union of embedded types.
Primitive Types
The set of primitive type names is:
- null: no value
- boolean: a binary value
- int: 32-bit signed integer
- long: 64-bit signed integer
- float: single precision (32-bit) IEEE 754 floating-point number
- double: double precision (64-bit) IEEE 754 floating-point number
- bytes: sequence of 8-bit unsigned bytes
- string: unicode character sequence
Primitive types have no specified attributes.
Primitive type names are also defined type names. Thus, for example, the schema “string” is equivalent to:
Complex Types
Avro supports six kinds of complex types: records, enums, arrays, maps, unions and fixed.
Records
Records use the type name “record” and support three attributes:
- name: a JSON string providing the name of the record (required).
- namespace, a JSON string that qualifies the name;
- doc: a JSON string providing documentation to the user of this schema (optional).
- aliases: a JSON array of strings, providing alternate names for this record (optional).
- fields: a JSON array, listing fields (required). Each field is a JSON object with the following attributes:
- name: a JSON string providing the name of the field (required), and
- doc: a JSON string describing this field for users (optional).
- type: A JSON object defining a schema, or a JSON string naming a record definition (required).
- _default: A default value for this field, used when reading instances that lack this field (optional). 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. Default values for bytes and fixed fields are JSON strings, where Unicode code points 0-255 are mapped to unsigned 8-bit byte values 0-255.
field default values
| avro type |
json type |
example |
| null |
null |
null |
| boolean |
boolean |
true |
| int,long |
integer |
1 |
| float,double |
number |
1.1 |
| bytes |
string |
"\u00FF" |
| string |
string |
"foo" |
| record |
object |
{"a": 1} |
| enum |
string |
"FOO" |
| array |
array |
[1] |
| map |
object |
{"a": 1} |
| fixed |
string |
"\u00ff" |
- order: specifies how this field impacts sort ordering of this record (optional). Valid values are “ascending” (the default), “descending”, or “ignore”. For more details on how this is used, see the sort order section below.
- aliases: a JSON array of strings, providing alternate names for this field (optional).
For example, a linked-list of 64-bit values may be defined with:
{
"type": "record",
"name": "LongList",
"aliases": ["LinkedLongs"], // old name for this
"fields" : [
{"name": "value", "type": "long"}, // each element has a long
{"name": "next", "type": ["LongList", "null"]} // optional next element
]
}
Enums
Enums use the type name “enum” and support the following attributes:
- name: a JSON string providing the name of the enum (required).
- namespace, a JSON string that qualifies the name ;
- aliases: a JSON array of strings, providing alternate names for this enum (optional).
- doc: a JSON string providing documentation to the user of this schema (optional).
- symbols: a JSON array, listing symbols, as JSON strings (required). All symbols in an enum must be unique; duplicates are prohibited.
For example, playing card suits might be defined with:
{
"type": "enum",
"name": "Suit",
"symbols" : ["SPADES", "HEARTS", "DIAMONDS", "CLUBS"]
}
Arrays
Arrays use the type name “array” and support a single attribute:
- items: the schema of the array’s items.
For example, an array of strings is declared with:
{
"type": "array",
"items" : "string"
}
Maps
Maps use the type name “map” and support one attribute:
- values: the schema of the map’s values.
Map keys are assumed to be strings.
For example, a map from string to long is declared with:
{
"type": "map",
"values" : "long"
}
Unions
Unions, as mentioned above, are represented using JSON arrays. For example, ["string", "null"] declares a schema which may be either a string or null.
Unions may not contain more than one schema with the same type, except for the named types record, fixed and enum. For example, unions containing two array types or two map types are not permitted, but two types with different names are permitted. (Names permit efficient resolution when reading and writing unions.)
Unions may not immediately contain other unions.
Fixed
Fixed uses the type name “fixed” and supports two attributes:
- name: a string naming this fixed (required).
- namespace, a string that qualifies the name;
- aliases: a JSON array of strings, providing alternate names for this enum (optional).
- size: an integer, specifying the number of bytes per value (required).
For example, 16-byte quantity may be declared with:
{"type": "fixed", "size": 16, "name": "md5"}
Names
Record, enums and fixed are named types. Each has a fullname that is composed of two parts; a name and a namespace. Equality of names is defined on the fullname.
The name portion of a fullname, record field names, and enum symbols must:
- start with [A-Za-z_]
- subsequently contain only [A-Za-z0-9_]
A namespace is a dot-separated sequence of such names. Equality of names (including field names and enum symbols) as well as fullnames is case-sensitive.
In record, enum and fixed definitions, the fullname is determined in one of the following ways:
A name and namespace are both specified. For example, one might use “name”: “X”, “namespace”: “org.foo” to indicate the fullname org.foo.X.
-
A fullname is specified. If the name specified contains a dot, then it is assumed to be a fullname, and any namespace also specified is ignored. For example, use “name”: “org.foo.X” to indicate the fullname org.foo.X.
-
A name only is specified, i.e., a name that contains no dots. In this case the namespace is taken from the most tightly enclosing schema or protocol. For example, if “name”: “X” is specified, and this occurs within a field of the record definition of org.foo.Y, then the fullname is org.foo.X.
References to previously defined names are as in the latter two cases above: if they contain a dot they are a fullname, if they do not contain a dot, the namespace is the namespace of the enclosing definition.
Primitive type names have no namespace and their names may not be defined in any namespace.
A schema or protocol may not contain multiple definitions of a fullname. Further, a name must be defined before it is used (“before” in the depth-first, left-to-right traversal of the JSON parse tree, where the types attribute of a protocol is always deemed to come “before” the messages attribute.)
Aliases
Named types and fields may have aliases. An implementation may optionally use aliases to map a writer’s schema to the reader’s. This facilitates both schema evolution as well as processing disparate datasets.
Aliases function by re-writing the writer’s schema using aliases from the reader’s schema. For example, if the writer’s schema was named “Foo” and the reader’s schema is named “Bar” and has an alias of “Foo”, then the implementation would act as though “Foo” were named “Bar” when reading. Similarly, if data was written as a record with a field named “x” and is read as a record with a field named “y” with alias “x”, then the implementation would act as though “x” were named “y” when reading.
A type alias may be specified either as a fully namespace-qualified, or relative to the namespace of the name it is an alias for. For example, if a type named “a.b” has aliases of “c” and “x.y”, then the fully qualified names of its aliases are “a.c” and “x.y”.
Data Serialization
Avro data is always serialized with its schema. Files that store Avro data should always also include the schema for that data in the same file. Avro-based remote procedure call (RPC) systems must also guarantee that remote recipients of data have a copy of the schema used to write that data.
Because the schema used to write data is always available when the data is read, Avro data itself is not tagged with type information. The schema is required to parse data.
In general, both serialization and deserialization proceed as a depth-first, left-to-right traversal of the schema, serializing primitive types as they are encountered.
Encodings
Avro specifies two serialization encodings: binary and JSON. Most applications will use the binary encoding, as it is smaller and faster. But, for debugging and web-based applications, the JSON encoding may sometimes be appropriate.
Binary Encoding
Primitive Types
Primitive types are encoded in binary as follows:
- null is written as zero bytes.
- a boolean is written as a single byte whose value is either 0 (false) or 1 (true).
- int and long values are written using variable-length zig-zag coding. Some examples:
| value |
hex |
| 0 |
00 |
| -1 |
01 |
| 1 |
02 |
| -2 |
03 |
| 2 |
04 |
| … |
|
| -64 |
7f |
| 64 |
80 01 |
| … |
|
- a float is written as 4 bytes. The float is converted into a 32-bit integer using a method equivalent to Java’s floatToIntBits and then encoded in little-endian format.
- a double is written as 8 bytes. The double is converted into a 64-bit integer using a method equivalent to Java’s doubleToLongBits and then encoded in little-endian format.
- bytes are encoded as a long followed by that many bytes of data.
- a string is encoded as a long followed by that many bytes of UTF-8 encoded character data.
For example, the three-character string “foo” would be encoded as the long value 3 (encoded as hex 06) followed by the UTF-8 encoding of ‘f’, ‘o’, and ‘o’ (the hex bytes 66 6f 6f):
06 66 6f 6f
Complex Types
Complex types are encoded in binary as follows:
Records
A record is encoded by encoding the values of its fields in the order that they are declared. In other words, a record is encoded as just the concatenation of the encodings of its fields. Field values are encoded per their schema.
For example, the record schema
{
"type": "record",
"name": "test",
"fields" : [
{"name": "a", "type": "long"},
{"name": "b", "type": "string"}
]
}
An instance of this record whose a field has value 27 (encoded as hex 36) and whose b field has value “foo” (encoded as hex bytes 06 66 6f 6f), would be encoded simply as the concatenation of these, namely the hex byte sequence:
36 06 66 6f 6f
Enums
An enum is encoded by a int, representing the zero-based position of the symbol in the schema.
For example, consider the enum:
{"type": "enum", "name": "Foo", "symbols": ["A", "B", "C", "D"] }
This would be encoded by an int between zero and three, with zero indicating “A”, and 3 indicating “D”.
Arrays
Arrays are encoded as a series of blocks. Each block consists of a long count value, followed by that many array items. A block with count zero indicates the end of the array. Each item is encoded per the array’s item schema.
If a block’s count is negative, its absolute value is used, and the count is followed immediately by a long block size indicating the number of bytes in the block. This block size permits fast skipping through data, e.g., when projecting a record to a subset of its fields.
For example, the array schema
{"type": "array", "items": "long"}
an array containing the items 3 and 27 could be encoded as the long value 2 (encoded as hex 04) followed by long values 3 and 27 (encoded as hex 06 36) terminated by zero:
04 06 36 00
The blocked representation permits one to read and write arrays larger than can be buffered in memory, since one can start writing items without knowing the full length of the array.
Maps
Maps are encoded as a series of blocks. Each block consists of a long count value, followed by that many key/value pairs. A block with count zero indicates the end of the map. Each item is encoded per the map’s value schema.
If a block’s count is negative, its absolute value is used, and the count is followed immediately by a long block size indicating the number of bytes in the block. This block size permits fast skipping through data, e.g., when projecting a record to a subset of its fields.
The blocked representation permits one to read and write maps larger than can be buffered in memory, since one can start writing items without knowing the full length of the map.
Unions
A union is encoded by first writing a long value indicating the zero-based position within the union of the schema of its value. The value is then encoded per the indicated schema within the union.
For example, the union schema ["string","null"] would encode:
-
null as the integer 1 (the index of “null” in the union, encoded as hex 02):
02
-
the string “a” as zero (the index of “string” in the union), followed by the serialized string:
02 02 61
Fixed
Fixed instances are encoded using the number of bytes declared in the schema.
JSON Encoding
Except for unions, the JSON encoding is the same as is used to encode field default values.
The value of a union is encoded in JSON as follows:
- if its type is null, then it is encoded as a JSON null;
- otherwise it is encoded as a JSON object with one name/value pair whose name is the type’s name and whose value is the recursively encoded value. For Avro’s named types (record, fixed or enum) the user-specified name is used, for other types the type name is used.
For example, the union schema ["null","string","Foo"], where Foo is a record name, would encode:
- null as null;
- the string “a” as
{"string": "a"} and
- a Foo instance as
{"Foo": {...}}, where {...} indicates the JSON encoding of a Foo instance.
Note that a schema is still required to correctly process JSON-encoded data. For example, the JSON encoding does not distinguish between int and long, float and double, records and maps, enums and strings, etc.
Sort Order
Avro defines a standard sort order for data. This permits data written by one system to be efficiently sorted by another system. This can be an important optimization, as sort order comparisons are sometimes the most frequent per-object operation. Note also that Avro binary-encoded data can be efficiently ordered without deserializing it to objects.
Data items may only be compared if they have identical schemas. Pairwise comparisons are implemented recursively with a depth-first, left-to-right traversal of the schema. The first mismatch encountered determines the order of the items.
Two items with the same schema are compared according to the following rules.
- null data is always equal.
- boolean data is ordered with false before true.
- int, long, float and double data is ordered by ascending numeric value.
- bytes and fixed data are compared lexicographically by unsigned 8-bit values.
- string data is compared lexicographically by Unicode code point. Note that since UTF-8 is used as the binary encoding for strings, sorting of bytes and string binary data is identical.
- array data is compared lexicographically by element.
- enum data is ordered by the symbol’s position in the enum schema. For example, an enum whose symbols are
["z", "a"] would sort “z” values before “a” values.
- union data is first ordered by the branch within the union, and, within that, by the type of the branch. For example, an
["int", "string"] union would order all int values before all string values, with the ints and strings themselves ordered as defined above.
- record data is ordered lexicographically by field. If a field specifies that its order is:
- “ascending”, then the order of its values is unaltered.
- “descending”, then the order of its values is reversed.
- “ignore”, then its values are ignored when sorting.
- map data may not be compared. It is an error to attempt to compare data containing maps unless those maps are in an
"order":"ignore" record field.
Object Container Files
Avro includes a simple object container file format. A file has a schema, and all objects stored in the file must be written according to that schema, using binary encoding. Objects are stored in blocks that may be compressed. Syncronization markers are used between blocks to permit efficient splitting of files for MapReduce processing.
Files may include arbitrary user-specified metadata.
A file consists of:
- A file header, followed by
- one or more file data blocks.
A file header consists of:
- Four bytes, ASCII ‘O’, ‘b’, ‘j’, followed by 1.
- file metadata, including the schema.
- The 16-byte, randomly-generated sync marker for this file.
File metadata is written as if defined by the following map schema:
{"type": "map", "values": "bytes"}
All metadata properties that start with “avro.” are reserved. The following file metadata properties are currently used:
- avro.schema contains the schema of objects stored in the file, as JSON data (required).
- avro.codec the name of the compression codec used to compress blocks, as a string. Implementations are required to support the following codecs: “null” and “deflate”. If codec is absent, it is assumed to be “null”. The codecs are described with more detail below.
A file header is thus described by the following schema:
{"type": "record", "name": "org.apache.avro.file.Header",
"fields" : [
{"name": "magic", "type": {"type": "fixed", "name": "Magic", "size": 4}},
{"name": "meta", "type": {"type": "map", "values": "bytes"}},
{"name": "sync", "type": {"type": "fixed", "name": "Sync", "size": 16}}
]
}
A file data block consists of:
- A long indicating the count of objects in this block.
- A long indicating the size in bytes of the serialized objects in the current block, after any codec is applied
- The serialized objects. If a codec is specified, this is compressed by that codec.
- The file’s 16-byte sync marker.
Thus, each block’s binary data can be efficiently extracted or skipped without deserializing the contents. The combination of block size, object counts, and sync markers enable detection of corrupt blocks and help ensure data integrity.
Required Codecs
null
The “null” codec simply passes through data uncompressed.
deflate
The “deflate” codec writes the data block using the deflate algorithm as specified in RFC 1951, and typically implemented using the zlib library. Note that this format (unlike the “zlib format” in RFC 1950) does not have a checksum.
Optional Codecs
snappy
The “snappy” codec uses Google’s Snappy compression library. Each compressed block is followed by the 4-byte, big-endian CRC32 checksum of the uncompressed data in the block.
Protocol Declaration
Avro protocols describe RPC interfaces. Like schemas, they are defined with JSON text.
A protocol is a JSON object with the following attributes:
- protocol, a string, the name of the protocol (required);
- namespace, an optional string that qualifies the name;
- doc, an optional string describing this protocol;
- types, an optional list of definitions of named types (records, enums, fixed and errors). An error definition is just like a record definition except it uses “error” instead of “record”. Note that forward references to named types are not permitted.
- messages, an optional JSON object whose keys are message names and whose values are objects whose attributes are described below. No two messages may have the same name.
The name and namespace qualification rules defined for schema objects apply to protocols as well.
Messages
A message has attributes:
- a doc, an optional description of the message,
- a request, a list of named, typed parameter schemas (this has the same form as the fields of a record declaration);
- a response schema;
- an optional union of declared error schemas. The effective union has “string” prepended to the declared union, to permit transmission of undeclared “system” errors. For example, if the declared error union is
["AccessError"], then the effective union is ["string", "AccessError"]. When no errors are declared, the effective error union is ["string"]. Errors are serialized using the effective union; however, a protocol’s JSON declaration contains only the declared union.
- an optional one-way boolean parameter.
A request parameter list is processed equivalently to an anonymous record. Since record field lists may vary between reader and writer, request parameters may also differ between the caller and responder, and such differences are resolved in the same manner as record field differences.
The one-way parameter may only be true when the response type is "null" and no errors are listed.
Sample Protocol
For example, one may define a simple HelloWorld protocol with:
{
"namespace": "com.acme",
"protocol": "HelloWorld",
"doc": "Protocol Greetings",
"types": [
{"name": "Greeting", "type": "record", "fields": [
{"name": "message", "type": "string"}]},
{"name": "Curse", "type": "error", "fields": [
{"name": "message", "type": "string"}]}
],
"messages": {
"hello": {
"doc": "Say hello.",
"request": [{"name": "greeting", "type": "Greeting" }],
"response": "Greeting",
"errors": ["Curse"]
}
}
}
Message Transport
Messages may be transmitted via different transport mechanisms.
To the transport, a message is an opaque byte sequence.
A transport is a system that supports:
- transmission of request messages
- receipt of corresponding response messages
Servers may send a response message back to the client corresponding to a request message. The mechanism of correspondance is transport-specific. For example, in HTTP it is implicit, since HTTP directly supports requests and responses. But a transport that multiplexes many client threads over a single socket would need to tag messages with unique identifiers.
Transports may be either stateless or stateful. In a stateless transport, messaging assumes no established connection state, while stateful transports establish connections that may be used for multiple messages. This distinction is discussed further in the handshake section below.
HTTP as Transport
When HTTP is used as a transport, each Avro message exchange is an HTTP request/response pair. All messages of an Avro protocol should share a single URL at an HTTP server. Other protocols may also use that URL. Both normal and error Avro response messages should use the 200 (OK) response code. The chunked encoding may be used for requests and responses, but, regardless the Avro request and response are the entire content of an HTTP request and response. The HTTP Content-Type of requests and responses should be specified as “avro/binary”. Requests should be made using the POST method.
HTTP is used by Avro as a stateless transport.
Message Framing
Avro messages are framed as a list of buffers.
Framing is a layer between messages and the transport. It exists to optimize certain operations.
The format of framed message data is:
- a series of buffers, where each buffer consists of:
- a four-byte, big-endian buffer length, followed by
- that many bytes of buffer data.
- A message is always terminated by a zero-lenghted buffer.
Framing is transparent to request and response message formats (described below). Any message may be presented as a single or multiple buffers.
Framing can permit readers to more efficiently get different buffers from different sources and for writers to more efficiently store different buffers to different destinations. In particular, it can reduce the number of times large binary objects are copied. For example, if an RPC parameter consists of a megabyte of file data, that data can be copied directly to a socket from a file descriptor, and, on the other end, it could be written directly to a file descriptor, never entering user space.
A simple, recommended, framing policy is for writers to create a new segment whenever a single binary object is written that is larger than a normal output buffer. Small objects are then appended in buffers, while larger objects are written as their own buffers. When a reader then tries to read a large object the runtime can hand it an entire buffer directly, without having to copy it.
Handshake
The purpose of the handshake is to ensure that the client and the server have each other’s protocol definition, so that the client can correctly deserialize responses, and the server can correctly deserialize requests. Both clients and servers should maintain a cache of recently seen protocols, so that, in most cases, a handshake will be completed without extra round-trip network exchanges or the transmission of full protocol text.
RPC requests and responses may not be processed until a handshake has been completed. With a stateless transport, all requests and responses are prefixed by handshakes. With a stateful transport, handshakes are only attached to requests and responses until a successful handshake response has been returned over a connection. After this, request and response payloads are sent without handshakes for the lifetime of that connection.
The handshake process uses the following record schemas:
{
"type": "record",
"name": "HandshakeRequest", "namespace":"org.apache.avro.ipc",
"fields": [
{"name": "clientHash",
"type": {"type": "fixed", "name": "MD5", "size": 16}},
{"name": "clientProtocol", "type": ["null", "string"]},
{"name": "serverHash", "type": "MD5"},
{"name": "meta", "type": ["null", {"type": "map", "values": "bytes"}]}
]
}
{
"type": "record",
"name": "HandshakeResponse", "namespace": "org.apache.avro.ipc",
"fields": [
{"name": "match",
"type": {"type": "enum", "name": "HandshakeMatch",
"symbols": ["BOTH", "CLIENT", "NONE"]}},
{"name": "serverProtocol",
"type": ["null", "string"]},
{"name": "serverHash",
"type": ["null", {"type": "fixed", "name": "MD5", "size": 16}]},
{"name": "meta",
"type": ["null", {"type": "map", "values": "bytes"}]}
]
}
- A client first prefixes each request with a
HandshakeRequest containing just the hash of its protocol and of the server’s protocol (clientHash!=null, clientProtocol=null, serverHash!=null), where the hashes are 128-bit MD5 hashes of the JSON protocol text. If a client has never connected to a given server, it sends its hash as a guess of the server’s hash, otherwise it sends the hash that it previously obtained from this server.
The server responds with a HandshakeResponse containing one of:
-
match=BOTH, serverProtocol=null, serverHash=null if the client sent the valid hash of the server’s protocol and the server knows what protocol corresponds to the client’s hash. In this case, the request is complete and the response data immediately follows the HandshakeResponse.
-
match=CLIENT, serverProtocol!=null, serverHash!=null if the server has previously seen the client’s protocol, but the client sent an incorrect hash of the server’s protocol. The request is complete and the response data immediately follows the HandshakeResponse. The client must use the returned protocol to process the response and should also cache that protocol and its hash for future interactions with this server.
-
match=NONE if the server has not previously seen the client’s protocol. The serverHash and serverProtocol may also be non-null if the server’s protocol hash was incorrect.
In this case the client must then re-submit its request with its protocol text (clientHash!=null, clientProtocol!=null, serverHash!=null) and the server should respond with a successful match (match=BOTH, serverProtocol=null, serverHash=null) as above.
The meta field is reserved for future handshake enhancements.
A call consists of a request message paired with its resulting response or error message. Requests and responses contain extensible metadata, and both kinds of messages are framed as described above.
The format of a call request is:
- request metadata, a map with values of type bytes
- the message name, an Avro string, followed by
- the message parameters. Parameters are serialized according to the message’s request declaration.
When the empty string is used as a message name a server should ignore the parameters and return an empty response. A client may use this to ping a server or to perform a handshake without sending a protocol message.
When a message is declared one-way and a stateful connection has been established by a successful handshake response, no response data is sent. Otherwise the format of the call response is:
- response metadata, a map with values of type bytes
- a one-byte error flag boolean, followed by either:
- if the error flag is false, the message response, serialized per the message’s response schema.
- if the error flag is true, the error, serialized per the message’s effective error union schema.
Schema Resolution
A reader of Avro data, whether from an RPC or a file, can always parse that data because its schema is provided. But that schema may not be exactly the schema that was expected. For example, if the data was written with a different version of the software than it is read, then records may have had fields added or removed. This section specifies how such schema differences should be resolved.
We refer to the schema used to write the data as the writer’s schema, and the schema that the application expects the reader’s schema. Differences between these should be resolved as follows:
- It is an error if the two schemas do not match.
To match, one of the following must hold:
-
both schemas are arrays whose item types match
-
both schemas are maps whose value types match
-
both schemas are enums whose names match
-
both schemas are fixed whose sizes and names match
-
both schemas are records with the same name
-
either schema is a union
-
both schemas have same primitive type
-
the writer’s schema may be promoted to the reader’s as follows:
- int is promotable to long, float, or double
- long is promotable to float or double
- float is promotable to double
- string is promotable to bytes
- bytes is promotable to string
-
if both are records:
- the ordering of fields may be different: fields are matched by name.
- schemas for fields with the same name in both records are resolved recursively.
- if the writer’s record contains a field with a name not present in the reader’s record, the writer’s value for that field is ignored.
- if the reader’s record schema has a field that contains a default value, and writer’s schema does not have a field with the same name, then the reader should use the default value from its field.
- if the reader’s record schema has a field with no default value, and writer’s schema does not have a field with the same name, an error is signalled.
-
if both are enums:
if the writer’s symbol is not present in the reader’s enum and the reader has a default value, then that value is used, otherwise an error is signalled.
This resolution algorithm is applied recursively to the reader’s and writer’s array item schemas.
This resolution algorithm is applied recursively to the reader’s and writer’s value schemas.
The first schema in the reader’s union that matches the selected writer’s union schema is recursively resolved against it. if none match, an error is signalled.
- if reader’s is a union, but writer’s is not
The first schema in the reader’s union that matches the writer’s schema is recursively resolved against it. If none match, an error is signalled.
- if writer’s is a union, but reader’s is not
If the reader’s schema matches the selected writer’s schema, it is recursively resolved against it. If they do not match, an error is signalled.
A schema’s doc fields are ignored for the purposes of schema resolution. Hence, the doc portion of a schema may be dropped at serialization.
One of the defining characteristics of Avro is that a reader is assumed to have the “same” schema used by the writer of the data the reader is reading. This assumption leads to a data format that’s compact and also amenable to many forms of schema evolution. However, the specification so far has not defined what it means for the reader to have the “same” schema as the writer. Does the schema need to be textually identical? Well, clearly adding or removing some whitespace to a JSON expression does not change its meaning. At the same time, reordering the fields of records clearly does change the meaning. So what does it mean for a reader to have “the same” schema as a writer?
Parsing Canonical Form is a transformation of a writer’s schema that let’s us define what it means for two schemas to be “the same” for the purpose of reading data written against the schema. It is called Parsing Canonical Form because the transformations strip away parts of the schema, like “doc” attributes, that are irrelevant to readers trying to parse incoming data. It is called Canonical Form because the transformations normalize the JSON text (such as the order of attributes) in a way that eliminates unimportant differences between schemas. If the Parsing Canonical Forms of two different schemas are textually equal, then those schemas are “the same” as far as any reader is concerned, i.e., there is no serialized data that would allow a reader to distinguish data generated by a writer using one of the original schemas from data generated by a writing using the other original schema. (We sketch a proof of this property in a companion document.)
The next subsection specifies the transformations that define Parsing Canonical Form. But with a well-defined canonical form, it can be convenient to go one step further, transforming these canonical forms into simple integers (“fingerprints”) that can be used to uniquely identify schemas. The subsection after next recommends some standard practices for generating such fingerprints.
Assuming an input schema (in JSON form) that’s already UTF-8 text for a valid Avro schema (including all quotes as required by JSON), the following transformations will produce its Parsing Canonical Form:
- [PRIMITIVES] Convert primitive schemas to their simple form (e.g., int instead of
{"type":"int"}).
- [FULLNAMES] Replace short names with fullnames, using applicable namespaces to do so. Then eliminate namespace attributes, which are now redundant.
- [STRIP] Keep only attributes that are relevant to parsing data, which are: type, name, fields, symbols, items, values, size. Strip all others (e.g., doc and aliases).
- [ORDER] Order the appearance of fields of JSON objects as follows: name, type, fields, symbols, items, values, size. For example, if an object has type, name, and size fields, then the name field should appear first, followed by the type and then the size fields.
- [STRINGS] For all JSON string literals in the schema text, replace any escaped characters (e.g., \uXXXX escapes) with their UTF-8 equivalents.
- [INTEGERS] Eliminate quotes around and any leading zeros in front of JSON integer literals (which appear in the size attributes of fixed schemas).
- [WHITESPACE] Eliminate all whitespace in JSON outside of string literals.
Schema Fingerprints
“[A] fingerprinting algorithm is a procedure that maps an arbitrarily large data item (such as a computer file) to a much shorter bit string, its fingerprint, that uniquely identifies the original data for all practical purposes” (quoted from Wikipedia). In the Avro context, fingerprints of Parsing Canonical Form can be useful in a number of applications; for example, to cache encoder and decoder objects, to tag data items with a short substitute for the writer’s full schema, and to quickly negotiate common-case schemas between readers and writers.
In designing fingerprinting algorithms, there is a fundamental trade-off between the length of the fingerprint and the probability of collisions. To help application designers find appropriate points within this trade-off space, while encouraging interoperability and ease of implementation, we recommend using one of the following three algorithms when fingerprinting Avro schemas:
- When applications can tolerate longer fingerprints, we recommend using the SHA-256 digest algorithm to generate 256-bit fingerprints of Parsing Canonical Forms. Most languages today have SHA-256 implementations in their libraries.
- At the opposite extreme, the smallest fingerprint we recommend is a 64-bit Rabin fingerprint. Below, we provide pseudo-code for this algorithm that can be easily translated into any programming language. 64-bit fingerprints should guarantee uniqueness for schema caches of up to a million entries (for such a cache, the chance of a collision is 3E-8). We don’t recommend shorter fingerprints, as the chances of collisions is too great (for example, with 32-bit fingerprints, a cache with as few as 100,000 schemas has a 50% chance of having a collision).
- Between these two extremes, we recommend using the MD5 message digest to generate 128-bit fingerprints. These make sense only where very large numbers of schemas are being manipulated (tens of millions); otherwise, 64-bit fingerprints should be sufficient. As with SHA-256, MD5 implementations are found in most libraries today.
These fingerprints are not meant to provide any security guarantees, even the longer SHA-256-based ones. Most Avro applications should be surrounded by security measures that prevent attackers from writing random data and otherwise interfering with the consumers of schemas. We recommend that these surrounding mechanisms be used to prevent collision and pre-image attacks (i.e., “forgery”) on schema fingerprints, rather than relying on the security properties of the fingerprints themselves.
Rabin fingerprints are cyclic redundancy checks computed using irreducible polynomials. In the style of the Appendix of RFC 1952 (pg 10), which defines the CRC-32 algorithm, here’s our definition of the 64-bit AVRO fingerprinting algorithm:
long fingerprint64(byte[] buf) {
if (FP_TABLE == null) initFPTable();
long fp = EMPTY;
for (int i = 0; i < buf.length; i++)
fp = (fp >>> 8) ^ FP_TABLE[(int)(fp ^ buf[i]) & 0xff];
return fp;
}
static long EMPTY = 0xc15d213aa4d7a795L;
static long[] FP_TABLE = null;
void initFPTable() {
FP_TABLE = new long[256];
for (int i = 0; i < 256; i++) {
long fp = i;
for (int j = 0; j < 8; j++)
fp = (fp >>> 1) ^ (EMPTY & -(fp & 1L));
FP_TABLE[i] = fp;
}
}
Readers interested in the mathematics behind this algorithm may want to read this book chapter. (Unlike RFC-1952 and the book chapter, we prepend a single one bit to messages. We do this because CRCs ignore leading zero bits, which can be problematic. Our code prepends a one-bit by initializing fingerprints using EMPTY, rather than initializing using zero as in RFC-1952 and the book chapter.)
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
13.4 - Trevni
Version 0.1
DRAFT
This document is the authoritative specification of a file format. Its intent is to permit compatible, independent implementations that read and/or write files in this format.
Introduction
Data sets are often described as a table composed of rows and columns. Each record in the dataset is considered a row, with each field of the record occupying a different column. Writing records to a file one-by-one as they are created results in a row-major format, like Hadoop’s SequenceFile or Avro data files.
In many cases higher query performance may be achieved if the data is instead organized in a column-major format, where multiple values of a given column are stored adjacently. This document defines such a column-major file format for datasets.
To permit scalable, distributed query evaluation, datasets are partitioned into row groups, containing distinct collections of rows. Each row group is organized in column-major order, while row groups form a row-major partitioning of the entire dataset.
Rationale
Goals
The format is meant satisfy the following goals:
Maximize the size of row groups. Disc drives are used most efficiently when sequentially accessing data. Consider a drive that takes 10ms to seek and transfers at 100MB/second. If a 10-column dataset whose values are all the same size is split into 10MB row groups, then accessing a single column will require a sequence of seek+1MB reads, for a cost of 20ms/MB processed. If the same dataset is split into 100MB row groups then this drops to 11ms/MB processed. This effect is exaggerated for datasets with larger numbers of columns and with columns whose values are smaller than average. So we’d prefer row groups that are 100MB or greater.
Permit random access within a row group. Some queries will first examine one column, and, only when certain relatively rare criteria are met, examine other columns. Rather than iterating through selected columns of the row-group in parallel, one might iterate through one column and randomly access another. This is called support for WHERE clauses, after the SQL operator of that name.
Minimize the number of files per dataset. HDFS is a primary intended deployment platform for these files. The HDFS Namenode requires memory for each file in the filesystem, thus for a format to be HDFS-friendly it should strive to require the minimum number of distinct files.
Support co-location of columns within row-groups. Row groups are the unit of parallel operation on a column dataset. For efficient file i/o, the entirety of a row-group should ideally reside on the host that is evaluating the query in order to avoid network latencies and bottlenecks.
Data integrity. The format should permit applications to detect data corruption. Many file systems may prevent corruption, but files may be moved between filesystems and be subject to corruption at points in that process. It is best if the data in a file can be validated independently.
Extensibility. The format should permit applications to store additional annotations about a datasets in the files, such as type information, origin, etc. Some environments may have metadata stores for such information, but not all do, and files might be moved among systems with different metadata systems. The ability to keep such information within the file simplifies the coordination of such information.
Minimal overhead. The column format should not make datasets appreciably larger. Storage is a primary cost and a choice to use this format should not require additional storage.
Primary format. The column format should be usable as a primary format for datasets, not as an auxiliary, accelerated format. Applications that process a dataset in row-major order should be able to easily consume column files and applications that produce datasets in row-major order should be able to easily generate column files.
Design
To meet these goals we propose the following design.
Each row group is a separate file. All values of a column in a file are written contiguously. This maximizes the row group size, optimizing performance when querying few and small columns.
Each file occupies a single HDFS block. A larger than normal block size may be specified, e.g., ~1GB instead of the typical ~100MB. This guarantees co-location and eliminates network use when query processing can be co-located with the file. This also moderates the memory impact on the HDFS Namenode since no small files are written.
Each column in a file is written as a sequence of ~64kB compressed blocks. The sequence is prefixed by a table describing all of the blocks in the column to permit random access within the column.
Application-specific metadata may be added at the file, column, and block levels.
Checksums are included with each block, providing data integrity.
Discussion
The use of a single block per file achieves the same effect as the custom block placement policy described in the CIF paper, but while still permitting HDFS rebalancing and not increasing the number of files in the namespace.
This section formally describes the proposed column file format.
Data Model
We assume a simple data model, where a record is a set of named fields, and the value of each field is a sequence of untyped bytes. A type system may be layered on top of this, as specified in the Type Mapping section below.
Primitive Values
We define the following primitive value types:
- Signed 64-bit long values are written using a variable-length zig-zag coding, where the high-order bit in each byte determines whether subsequent bytes are present. For example:
decimal value hex bytes
| decimal value |
hex bytes |
|
| 0 |
00 |
|
| -1 |
01 |
|
| 1 |
02 |
|
| … |
|
|
| -64 |
7f |
|
| 64 |
80 01 |
|
| … |
|
|
- bytes are encoded as a long followed by that many bytes of data.
- a string is encoded as a long followed by that many bytes of UTF-8 encoded character data.
For example, the three-character string “foo” would be encoded as the long value 3 (encoded as hex 06) followed by the UTF-8 encoding of ‘f’, ‘o’, and ‘o’ (the hex bytes 66 6f 6f): 06 66 6f 6f
Type Names
The following type names are used to describe column values:
- null, requires zero bytes. Sometimes used in array columns.
- boolean, one bit, packed into bytes, little-endian;
- int, like long, but restricted to 32-bit signed values
- long 64-bit signed values, represented as above
- fixed32 32-bit values stored as four bytes, little-endian.
- fixed64 64-bit values stored as eight bytes, little-endian.
- float 32-bit IEEE floating point value, little-endian
- double 64-bit IEEE floating point value, little-endian
- string as above
- bytes as above, may be used to encapsulate more complex objects
Type names are represented as strings (UTF-8 encoded, length-prefixed).
Metadata consists of:
- A long indicating the number of metadata key/value pairs.
- For each pair, a string key and bytes value.
All metadata properties that start with “trevni.” are reserved.
The following file metadata properties are defined:
- trevni.codec the name of the default compression codec used to compress blocks, as a string. Implementations are required to support the “null” codec. Optional. If absent, it is assumed to be “null”. Codecs are described in more detail below.
- trevni.checksum the name of the checksum algorithm used in this file, as a string. Implementations are required to support the “crc-32” checksum. Optional. If absent, it is assumed to be “null”. Checksums are described in more detail below.
The following column metadata properties are defined:
- trevni.codec the name of the compression codec used to compress the blocks of this column, as a string. Implementations are required to support the “null” codec. Optional. If absent, it is assumed to be “null”. Codecs are described in more detail below.
- trevni.name the name of the column, as a string. Required.
- trevni.type the type of data in the column. One of the type names above. Required.
- trevni.values if present, indicates that the initial value of each block in this column will be stored in the block’s descriptor. Not permitted for array columns or columns that specify a parent.
- trevni.array if present, indicates that each row in this column contains a sequence of values of the named type rather than just a single value. An integer length precedes each sequence of values indicating the count of values in the sequence. If the length is negative then it indicates a sequence of zero or one lengths, where -1 indicates two zeros, -2 two ones, -3 three zeros, -4 three ones, etc.
- trevni.parent if present, the name of an array column whose lengths are also used by this column. Thus values of this column are sequences but no lengths are stored in this column.
For example, consider the following row, as JSON, where all values are primitive types, but one has multiple values.
{"id"=566, "date"=23423234234
"from"="foo@bar.com",
"to"=["bar@baz.com", "bang@foo.com"],
"content"="Hi!"}
The columns for this might be specified as:
name=id type=int
name=date type=long
name=from type=string
name=to type=string array=true
name=content type=string
If a row contains an array of records, e.g. “received” in the following:
{"id"=566, "date"=23423234234
"from"="foo@bar.com",
"to"=["bar@baz.com", "bang@foo.com"],
"content"="Hi!"
"received"=[{"date"=234234234234, "host"="192.168.0.0.1"},
{"date"=234234545645, "host"="192.168.0.0.2"}]
}
Then one can define a parent column followed by a column for each field in the record, adding the following columns:
name=received type=null array=true
name=date type=long parent=received
name=host type=string parent=received
If an array value itself contains an array, e.g. the “sigs” below:
{"id"=566, "date"=23423234234
"from"="foo@bar.com",
"to"=["bar@baz.com", "bang@foo.com"],
"content"="Hi!"
"received"=[{"date"=234234234234, "host"="192.168.0.0.1",
"sigs"=[{"algo"="weak", "value"="0af345de"}]},
{"date"=234234545645, "host"="192.168.0.0.2",
"sigs"=[]}]
}
Then a parent column may be defined that itself has a parent column.
name=sigs type=null array=true parent=received
name=algo type=string parent=sigs
name=value type=string parent=sigs
No block metadata properties are currently defined.
A file consists of:
- A file header, followed by
- one or more columns.
A file header consists of:
- Four bytes, ASCII ‘T’, ‘r’, ‘v’, followed by 0x02.
- a fixed64 indicating the number of rows in the file
- a fixed32 indicating the number of columns in the file
- file metadata.
- for each column, its column metadata
- for each column, its starting position in the file as a fixed64.
A column consists of:
- A fixed32 indicating the number of blocks in this column.
- For each block, a block descriptor
- One or more blocks.
A block descriptor consists of:
- A fixed32 indicating the number of rows in the block
- A fixed32 indicating the size in bytes of the block before the codec is applied (excluding checksum).
- A fixed32 indicating the size in bytes of the block after the codec is applied (excluding checksum).
- If this column’s metadata declares it to include values, the first value in the column, serialized according to this column’s type.
A block consists of:
- The serialized column values. If a column is an array column then value sequences are preceded by their length, as an int. If a codec is specified, the values and lengths are compressed by that codec.
- The checksum, as determined by the file metadata.
Codecs
null
The “null” codec simply passes data through uncompressed.
deflate
The “deflate” codec writes the data block using the deflate algorithm as specified in RFC 1951.
snappy
The “snappy” codec uses Google’s Snappy compression library.
Checksum algorithms
null
The “null” checksum contains zero bytes.
crc-32
Each “crc-32” checksum contains the four bytes of an ISO 3309 CRC-32 checksum of the uncompressed block data as a fixed32.
Type Mappings
We define a standard mapping for how types defined in various serialization systems are represented in a column file. Records from these systems are shredded into columns. When records are nested, a depth-first recursive walk can assign a separate column for each primitive value.
Avro
Protocol Buffers
Thrift
Implementation Notes
Some possible techniques for writing column files include:
- Use a standard ~100MB block, buffer in memory up to the block size, then flush the file directly to HDFS. A single reduce task might create multiple output files. The namenode requires memory proportional to the number of names and blocks*replication. This would increase the number of names but not blocks, so this should still be much better than a file per column.
- Spill each column to a separate local, temporary file then, when the file is closed, append these files, writing a single file to HDFS whose block size is set to be that of the entire file. This would be a bit slower than and may have trouble when the local disk is full, but it would better use HDFS namespace and further reduce seeks when processing columns whose values are small.
- Use a separate mapreduce job to convert row-major files to column-major. The map output would output a by (row#, column#, value) tuple, partitioned by row# but sorted by column# then row#. The reducer could directly write the column file. But the column file format would need to be changed to write counts, descriptors, etc. at the end of files rather than at the front.
(1) is the simplest to implement and most implementations should start with it.
References
CIF Column-Oriented Storage Techniques for MapReduce, Floratou, Patel, Shekita, & Tata, VLDB 2011.
DREMEL Dremel: Interactive Analysis of Web-Scale Datasets, Melnik, Gubarev, Long, Romer, Shivakumar, & Tolton, VLDB 2010.
13.5 - MapReduce guide
Avro provides a convenient way to represent complex data structures within a Hadoop MapReduce job. Avro data can be used as both input to and output from a MapReduce job, as well as the intermediate format. The example in this guide uses Avro data for all three, but it’s possible to mix and match; for instance, MapReduce can be used to aggregate a particular field in an Avro record.
This guide assumes basic familiarity with both Hadoop MapReduce and Avro. See the Hadoop documentation and the Avro getting started guide for introductions to these projects. This guide uses the old MapReduce API (org.apache.hadoop.mapred) and the new MapReduce API (org.apache.hadoop.mapreduce).
Setup
The code from this guide is included in the Avro docs under examples/mr-example. The example is set up as a Maven project that includes the necessary Avro and MapReduce dependencies and the Avro Maven plugin for code generation, so no external jars are needed to run the example. In particular, the POM includes the following dependencies:
<dependency>
<groupId>org.apache.avro</groupId>
<artifactId>avro</artifactId>
<version>1.7.6</version>
</dependency>
<dependency>
<groupId>org.apache.avro</groupId>
<artifactId>avro-mapred</artifactId>
<version>1.7.6</version>
</dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-core</artifactId>
<version>1.1.0</version>
</dependency>
And the following plugin:
<plugin>
<groupId>org.apache.avro</groupId>
<artifactId>avro-maven-plugin</artifactId>
<version>1.7.6</version>
<executions>
<execution>
<phase>generate-sources</phase>
<goals>
<goal>schema</goal>
</goals>
<configuration>
<sourceDirectory>${project.basedir}/../</sourceDirectory>
<outputDirectory>${project.basedir}/src/main/java/</outputDirectory>
</configuration>
</execution>
</executions>
</plugin>
Alternatively, Avro jars can be downloaded directly from the Apache Avro™ Releases page. The relevant Avro jars for this guide are avro-1.7.6.jar and avro-mapred-1.7.6.jar, as well as avro-tools-1.7.6.jar for code generation and viewing Avro data files as JSON. In addition, you will need to install Hadoop in order to use MapReduce.
Example: ColorCount
Below is a simple example of a MapReduce that uses Avro. There is an example for both the old (org.apache.hadoop.mapred) and new (org.apache.hadoop.mapreduce) APIs under examples/mr-example/src/main/java/example/. MapredColorCount is the example for the older mapred API while MapReduceColorCount is the example for the newer mapreduce API. Both examples are below, but we will detail the mapred API in our subsequent examples.
MapredColorCount:
package example;
import java.io.IOException;
import org.apache.avro.*;
import org.apache.avro.Schema.Type;
import org.apache.avro.mapred.*;
import org.apache.hadoop.conf.*;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.mapred.*;
import org.apache.hadoop.util.*;
import example.avro.User;
public class MapredColorCount extends Configured implements Tool {
public static class ColorCountMapper extends AvroMapper<User, Pair<CharSequence, Integer>> {
@Override
public void map(User user, AvroCollector<Pair<CharSequence, Integer>> collector, Reporter reporter)
throws IOException {
CharSequence color = user.getFavoriteColor();
// We need this check because the User.favorite_color field has type ["string", "null"]
if (color == null) {
color = "none";
}
collector.collect(new Pair<CharSequence, Integer>(color, 1));
}
}
public static class ColorCountReducer extends AvroReducer<CharSequence, Integer,
Pair<CharSequence, Integer>> {
@Override
public void reduce(CharSequence key, Iterable<Integer> values,
AvroCollector<Pair<CharSequence, Integer>> collector,
Reporter reporter)
throws IOException {
int sum = 0;
for (Integer value : values) {
sum += value;
}
collector.collect(new Pair<CharSequence, Integer>(key, sum));
}
}
public int run(String[] args) throws Exception {
if (args.length != 2) {
System.err.println("Usage: MapredColorCount <input path> <output path>");
return -1;
}
JobConf conf = new JobConf(getConf(), MapredColorCount.class);
conf.setJobName("colorcount");
FileInputFormat.setInputPaths(conf, new Path(args[0]));
FileOutputFormat.setOutputPath(conf, new Path(args[1]));
AvroJob.setMapperClass(conf, ColorCountMapper.class);
AvroJob.setReducerClass(conf, ColorCountReducer.class);
// Note that AvroJob.setInputSchema and AvroJob.setOutputSchema set
// relevant config options such as input/output format, map output
// classes, and output key class.
AvroJob.setInputSchema(conf, User.getClassSchema());
AvroJob.setOutputSchema(conf, Pair.getPairSchema(Schema.create(Type.STRING),
Schema.create(Type.INT)));
JobClient.runJob(conf);
return 0;
}
public static void main(String[] args) throws Exception {
int res = ToolRunner.run(new Configuration(), new MapredColorCount(), args);
System.exit(res);
}
}
MapReduceColorCount:
package example;
import java.io.IOException;
import org.apache.avro.Schema;
import org.apache.avro.mapred.AvroKey;
import org.apache.avro.mapred.AvroValue;
import org.apache.avro.mapreduce.AvroJob;
import org.apache.avro.mapreduce.AvroKeyInputFormat;
import org.apache.avro.mapreduce.AvroKeyValueOutputFormat;
import org.apache.hadoop.conf.Configured;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.IntWritable;
import org.apache.hadoop.io.NullWritable;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.mapreduce.Job;
import org.apache.hadoop.mapreduce.Mapper;
import org.apache.hadoop.mapreduce.Reducer;
import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
import org.apache.hadoop.util.Tool;
import org.apache.hadoop.util.ToolRunner;
import example.avro.User;
public class MapReduceColorCount extends Configured implements Tool {
public static class ColorCountMapper extends
Mapper<AvroKey<User>, NullWritable, Text, IntWritable> {
@Override
public void map(AvroKey<User> key, NullWritable value, Context context)
throws IOException, InterruptedException {
CharSequence color = key.datum().getFavoriteColor();
if (color == null) {
color = "none";
}
context.write(new Text(color.toString()), new IntWritable(1));
}
}
public static class ColorCountReducer extends
Reducer<Text, IntWritable, AvroKey<CharSequence>, AvroValue<Integer>> {
@Override
public void reduce(Text key, Iterable<IntWritable> values,
Context context) throws IOException, InterruptedException {
int sum = 0;
for (IntWritable value : values) {
sum += value.get();
}
context.write(new AvroKey<CharSequence>(key.toString()), new AvroValue<Integer>(sum));
}
}
public int run(String[] args) throws Exception {
if (args.length != 2) {
System.err.println("Usage: MapReduceColorCount <input path> <output path>");
return -1;
}
Job job = new Job(getConf());
job.setJarByClass(MapReduceColorCount.class);
job.setJobName("Color Count");
FileInputFormat.setInputPaths(job, new Path(args[0]));
FileOutputFormat.setOutputPath(job, new Path(args[1]));
job.setInputFormatClass(AvroKeyInputFormat.class);
job.setMapperClass(ColorCountMapper.class);
AvroJob.setInputKeySchema(job, User.getClassSchema());
job.setMapOutputKeyClass(Text.class);
job.setMapOutputValueClass(IntWritable.class);
job.setOutputFormatClass(AvroKeyValueOutputFormat.class);
job.setReducerClass(ColorCountReducer.class);
AvroJob.setOutputKeySchema(job, Schema.create(Schema.Type.STRING));
AvroJob.setOutputValueSchema(job, Schema.create(Schema.Type.INT));
return (job.waitForCompletion(true) ? 0 : 1);
}
public static void main(String[] args) throws Exception {
int res = ToolRunner.run(new MapReduceColorCount(), args);
System.exit(res);
}
}
ColorCount reads in data files containing User records, defined in examples/user.avsc, and counts the number of instances of each favorite color. (This example draws inspiration from the canonical WordCount MapReduce application.) This example uses the old MapReduce API. See MapReduceAvroWordCount, found under doc/examples/mr-example/src/main/java/example/ to see the new MapReduce API example. The User schema is defined as follows:
{"namespace": "example.avro",
"type": "record",
"name": "User",
"fields": [
{"name": "name", "type": "string"},
{"name": "favorite_number", "type": ["int", "null"]},
{"name": "favorite_color", "type": ["string", "null"]}
]
}
This schema is compiled into the User class used by ColorCount via the Avro Maven plugin (see examples/mr-example/pom.xml for how this is set up).
ColorCountMapper essentially takes a User as input and extracts the User’s favorite color, emitting the key-value pair <favoriteColor, 1>. ColorCountReducer then adds up how many occurrences of a particular favorite color were emitted, and outputs the result as a Pair record. These Pairs are serialized to an Avro data file.
Running ColorCount
The ColorCount application is provided as a Maven project in the Avro docs under examples/mr-example. To build the project, including the code generation of the User schema, run:
Next, run GenerateData from examples/mr-examples to create an Avro data file, input/users.avro, containing 20 Users with favorite colors chosen randomly from a list:
mvn exec:java -q -Dexec.mainClass=example.GenerateData
Besides creating the data file, GenerateData prints the JSON representations of the Users generated to stdout, for example:
{"name": "user", "favorite_number": null, "favorite_color": "red"}
{"name": "user", "favorite_number": null, "favorite_color": "green"}
{"name": "user", "favorite_number": null, "favorite_color": "purple"}
{"name": "user", "favorite_number": null, "favorite_color": null}
...
Now we’re ready to run ColorCount. We specify our freshly-generated input folder as the input path and output as our output folder (note that MapReduce will not start a job if the output folder already exists):
mvn exec:java -q -Dexec.mainClass=example.MapredColorCount -Dexec.args="input output"
Once ColorCount completes, checking the contents of the new output directory should yield the following:
$ ls output/
part-00000.avro _SUCCESS
You can check the contents of the generated Avro file using the avro-tools jar:
$ java -jar /path/to/avro-tools-1.7.6.jar tojson output/part-00000.avro
{"value": 3, "key": "blue"}
{"value": 7, "key": "green"}
{"value": 1, "key": "none"}
{"value": 2, "key": "orange"}
{"value": 3, "key": "purple"}
{"value": 2, "key": "red"}
{"value": 2, "key": "yellow"}
Now let’s go over the ColorCount example in detail.
Mapper - org.apache.hadoop.mapred API
The easiest way to use Avro data files as input to a MapReduce job is to subclass AvroMapper. An AvroMapper defines a map function that takes an Avro datum as input and outputs a key/value pair represented as a Pair record. In the ColorCount example, ColorCountMapper is an AvroMapper that takes a User as input and outputs a Pair<CharSequence, Integer>>, where the CharSequence key is the user’s favorite color and the Integer value is 1.
public static class ColorCountMapper extends AvroMapper<User, Pair<CharSequence, Integer>> {
@Override
public void map(User user, AvroCollector<Pair<CharSequence, Integer>> collector, Reporter reporter)
throws IOException {
CharSequence color = user.getFavoriteColor();
// We need this check because the User.favorite_color field has type ["string", "null"]
if (color == null) {
color = "none";
}
collector.collect(new Pair<CharSequence, Integer>(color, 1));
}
}
In order to use our AvroMapper, we must call AvroJob.setMapperClass and AvroJob.setInputSchema.
AvroJob.setMapperClass(conf, ColorCountMapper.class);
AvroJob.setInputSchema(conf, User.getClassSchema());
Note that AvroMapper does not implement the Mapper interface. Under the hood, the specified Avro data files are deserialized into AvroWrappers containing the actual data, which are processed by a Mapper that calls the configured AvroMapper’s map function. AvroJob.setInputSchema sets up the relevant configuration parameters needed to make this happen, thus you should not need to call JobConf.setMapperClass, JobConf.setInputFormat, JobConf.setMapOutputKeyClass, JobConf.setMapOutputValueClass, or JobConf.setOutputKeyComparatorClass.
Mapper - org.apache.hadoop.mapreduce API
This document will not go into all the differences between the mapred and mapreduce APIs, however will describe the main differences. As you can see, ColorCountMapper is now a subclass of the Hadoop Mapper class and is passed an AvroKey as it’s key. Additionally, the AvroJob method calls were slightly changed.
public static class ColorCountMapper extends
Mapper<AvroKey<User>, NullWritable, Text, IntWritable> {
@Override
public void map(AvroKey<User> key, NullWritable value, Context context)
throws IOException, InterruptedException {
CharSequence color = key.datum().getFavoriteColor();
if (color == null) {
color = "none";
}
context.write(new Text(color.toString()), new IntWritable(1));
}
}
Reducer - org.apache.hadoop.mapred API
Analogously to AvroMapper, an AvroReducer defines a reducer function that takes the key/value types output by an AvroMapper (or any mapper that outputs Pairs) and outputs a key/value pair represented a Pair record. In the ColorCount example, ColorCountReducer is an AvroReducer that takes the CharSequence key representing a favorite color and the Iterable<Integer> representing the counts for that color (they should all be 1 in this example) and adds up the counts.
public static class ColorCountReducer extends AvroReducer<CharSequence, Integer,
Pair<CharSequence, Integer>> {
@Override
public void reduce(CharSequence key, Iterable<Integer> values,
AvroCollector<Pair<CharSequence, Integer>> collector,
Reporter reporter)
throws IOException {
int sum = 0;
for (Integer value : values) {
sum += value;
}
collector.collect(new Pair<CharSequence, Integer>(key, sum));
}
}
In order to use our AvroReducer, we must call AvroJob.setReducerClass and AvroJob.setOutputSchema.
AvroJob.setReducerClass(conf, ColorCountReducer.class);
AvroJob.setOutputSchema(conf, Pair.getPairSchema(Schema.create(Type.STRING),
Schema.create(Type.INT)));
Note that AvroReducer does not implement the Reducer interface. The intermediate Pairs output by the mapper are split into AvroKeys and AvroValues, which are processed by a Reducer that calls the configured AvroReducer’s reduce function. AvroJob.setOutputSchema sets up the relevant configuration parameters needed to make this happen, thus you should not need to call JobConf.setReducerClass, JobConf.setOutputFormat, JobConf.setOutputKeyClass, JobConf.setMapOutputKeyClass, JobConf.setMapOutputValueClass, or JobConf.setOutputKeyComparatorClass.
Reduce - org.apache.hadoop.mapreduce API
As before we not detail every difference between the APIs. As with the Mapper change ColorCountReducer is now a subclass of Reducer and AvroKey and AvroValue are emitted. Additionally, the AvroJob method calls were slightly changed.
public static class ColorCountReducer extends
Reducer<Text, IntWritable, AvroKey<CharSequence>, AvroValue<Integer>> {
@Override
public void reduce(Text key, Iterable<IntWritable> values,
Context context) throws IOException, InterruptedException {
int sum = 0;
for (IntWritable value : values) {
sum += value.get();
}
context.write(new AvroKey<CharSequence>(key.toString()), new AvroValue<Integer>(sum));
}
}
Learning more
The mapred API allows users to mix Avro AvroMappers and AvroReducers with non-Avro Mappers and Reducers and the mapreduce API allows users input Avro and output non-Avro or vice versa.
The mapred package has API org.apache.avro.mapred documentation as does the org.apache.avro.mapreduce package. MapReduce API (org.apache.hadoop.mapreduce). Similarily to the mapreduce package, it’s possible with the mapred API to implement your own Mappers and Reducers directly using the public classes provided in these libraries. See the AvroWordCount application, found under examples/mr-example/src/main/java/example/AvroWordCount.java in the Avro documentation, for an example of implementing a Reducer that outputs Avro data using the old MapReduce API. See the MapReduceAvroWordCount application, found under examples/mr-example/src/main/java/example/MapReduceAvroWordCount.java in the Avro documentation, for an example of implementing a Reducer that outputs Avro data using the new MapReduce API.
13.6 - IDL Language
Introduction
This document defines Avro IDL, a higher-level language for authoring Avro schemata. Before reading this document, you should have familiarity with the concepts of schemata and protocols, as well as the various primitive and complex types available in Avro.
Overview
Purpose
The aim of the Avro IDL language is to enable developers to author schemata in a way that feels more similar to common programming languages like Java, C++, or Python. Additionally, the Avro IDL language may feel more familiar for those users who have previously used the interface description languages (IDLs) in other frameworks like Thrift, Protocol Buffers, or CORBA.
Usage
Each Avro IDL file defines a single Avro Protocol, and thus generates as its output a JSON-format Avro Protocol file with extension .avpr.
To convert a .avdl file into a .avpr file, it may be processed by the idl tool. For example:
$ java -jar avroj-tools.jar idl src/test/idl/input/namespaces.avdl /tmp/namespaces.avpr
$ head /tmp/namespaces.avpr
{
"protocol" : "TestNamespace",
"namespace" : "avro.test.protocol",
The idl tool can also process input to and from stdin and stdout. See idl --help for full usage information.
A Maven plugin is also provided to compile .avdl files. To use it, add something like the following to your pom.xml:
<build>
<plugins>
<plugin>
<groupId>org.apache.avro</groupId>
<artifactId>avro-maven-plugin</artifactId>
<executions>
<execution>
<goals>
<goal>idl-protocol</goal>
</goals>
</execution>
</executions>
</plugin>
</plugins>
</build>
Defining a Protocol in Avro IDL
An Avro IDL file consists of exactly one protocol definition. The minimal protocol is defined by the following code:
This is equivalent to (and generates) the following JSON protocol definition:
{
"protocol" : "MyProtocol",
"types" : [ ],
"messages" : {
}
}
The namespace of the protocol may be changed using the @namespace annotation:
@namespace("mynamespace")
protocol MyProtocol {
}
This notation is used throughout Avro IDL as a way of specifying properties for the annotated element, as will be described later in this document.
Protocols in Avro IDL can contain the following items:
- Imports of external protocol and schema files.
- Definitions of named schemata, including records, errors, enums, and fixeds.
- Definitions of RPC messages
Imports
Files may be imported in one of three formats:
-
An IDL file may be imported with a statement like:
import idl "foo.avdl";
-
A JSON protocol file may be imported with a statement like:
import protocol "foo.avpr";
-
A JSON schema file may be imported with a statement like:
import schema "foo.avsc";
Messages and types in the imported file are added to this file’s protocol.
Imported file names are resolved relative to the current IDL file.
Defining an Enumeration
Enums are defined in Avro IDL using a syntax similar to C or Java:
enum Suit {
SPADES, DIAMONDS, CLUBS, HEARTS
}
Note that, unlike the JSON format, anonymous enums cannot be defined.
Defining a Fixed Length Field
Fixed fields are defined using the following syntax:
fixed MD5(16);
This example defines a fixed-length type called MD5 which contains 16 bytes.
Defining Records and Errors
Records are defined in Avro IDL using a syntax similar to a struct definition in C:
record Employee {
string name;
boolean active = true;
long salary;
}
The above example defines a record with the name “Employee” with three fields.
To define an error, simply use the keyword error instead of record. For example:
error Kaboom {
string explanation;
int result_code = -1;
}
Each field in a record or error consists of a type and a name, optional property annotations and an optional default value.
A type reference in Avro IDL must be one of:
- A primitive type
- A logical type
- A named schema defined prior to this usage in the same Protocol
- A complex type (array, map, or union)
Primitive Types
The primitive types supported by Avro IDL are the same as those supported by Avro’s JSON format. This list includes int, long, string, boolean, float, double, null, and bytes.
References to Named Schemata
If a named schema has already been defined in the same Avro IDL file, it may be referenced by name as if it were a primitive type:
record Card {
Suit suit; // refers to the enum Card defined above
int number;
}
Default Values
Default values for fields may be optionally specified by using an equals sign after the field name followed by a JSON expression indicating the default value. This JSON is interpreted as described in the spec.
Complex Types
Arrays
Array types are written in a manner that will seem familiar to C++ or Java programmers. An array of any type t is denoted array<t>. For example, an array of strings is denoted array<string>, and a multidimensional array of Foo records would be array<array<Foo>>.
Maps
Map types are written similarly to array types. An array that contains values of type t is written map<t>. As in the JSON schema format, all maps contain string-type keys.
Unions
Union types are denoted as union { typeA, typeB, typeC, … }. For example, this record contains a string field that is optional (unioned with null):
record RecordWithUnion {
union { null, string } optionalString;
}
Note that the same restrictions apply to Avro IDL unions as apply to unions defined in the JSON format; namely, a record may not contain multiple elements of the same type.
Defining RPC Messages
The syntax to define an RPC message within a Avro IDL protocol is similar to the syntax for a method declaration within a C header file or a Java interface. To define an RPC message add which takes two arguments named foo and bar, returning an int, simply include the following definition within the protocol:
int add(int foo, int bar = 0);
Message arguments, like record fields, may specify default values.
To define a message with no response, you may use the alias void, equivalent to the Avro null type:
void logMessage(string message);
If you have previously defined an error type within the same protocol, you may declare that a message can throw this error using the syntax:
void goKaboom() throws Kaboom;
To define a one-way message, use the keyword oneway after the parameter list, for example:
void fireAndForget(string message) oneway;
Other Language Features
All Java-style comments are supported within a Avro IDL file. Any text following // on a line is ignored, as is any text between /* and */, possibly spanning multiple lines.
Comments that begin with /** are used as the documentation string for the type or field definition that follows the comment.
Escaping Identifiers
Occasionally, one will need to use a reserved language keyword as an identifier. In order to do so, backticks (`) may be used to escape the identifier. For example, to define a message with the literal name error, you may write:
This syntax is allowed anywhere an identifier is expected.
Annotations for Ordering and Namespaces
Java-style annotations may be used to add additional properties to types and fields throughout Avro IDL.
For example, to specify the sort order of a field within a record, one may use the @order annotation before the field name as follows:
record MyRecord {
string @order("ascending") myAscendingSortField;
string @order("descending") myDescendingField;
string @order("ignore") myIgnoredField;
}
A field’s type may also be preceded by annotations, e.g.:
record MyRecord {
@java-class("java.util.ArrayList") array<string> myStrings;
}
This can be used to support java classes that can be serialized/deserialized via their toString/String constructor, e.g.:
record MyRecord {
@java-class("java.math.BigDecimal") string value;
@java-key-class("java.io.File") map<string> fileStates;
array<@java-class("java.math.BigDecimal") string> weights;
}
Similarly, a @namespace annotation may be used to modify the namespace when defining a named schema. For example:
@namespace("org.apache.avro.firstNamespace")
protocol MyProto {
@namespace("org.apache.avro.someOtherNamespace")
record Foo {}
record Bar {}
}
will define a protocol in the firstNamespace namespace. The record Foo will be defined in someOtherNamespace and Bar will be defined in firstNamespace as it inherits its default from its container.
Type and field aliases are specified with the @aliases annotation as follows:
@aliases(["org.old.OldRecord", "org.ancient.AncientRecord"])
record MyRecord {
string @aliases(["oldField", "ancientField"]) myNewField;
}
Some annotations like those listed above are handled specially. All other annotations are added as properties to the protocol, message, schema or field.
Complete Example
The following is a complete example of a Avro IDL file that shows most of the above features:
/**
* An example protocol in Avro IDL
*/
@namespace("org.apache.avro.test")
protocol Simple {
@aliases(["org.foo.KindOf"])
enum Kind {
FOO,
BAR, // the bar enum value
BAZ
}
fixed MD5(16);
record TestRecord {
@order("ignore")
string name;
@order("descending")
Kind kind;
MD5 hash;
union { MD5, null} @aliases(["hash"]) nullableHash;
array<long> arrayOfLongs;
}
error TestError {
string message;
}
string hello(string greeting);
TestRecord echo(TestRecord `record`);
int add(int arg1, int arg2);
bytes echoBytes(bytes data);
void `error`() throws TestError;
void ping() oneway;
}
Additional examples may be found in the Avro source tree under the src/test/idl/input directory.
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
13.7 - SASL profile
Introduction
SASL (RFC 2222) provides a framework for authentication and security of network protocols. Each protocol that uses SASL is meant to define a SASL profile. This document provides a SASL profile for connection-based Avro RPC.
Overview
SASL negotiation proceeds as a series of message interactions over a connection between a client and server using a selected SASL mechanism. The client starts this negotiation by sending its chosen mechanism name with an initial (possibly empty) message. Negotiation proceeds with the exchange of messages until either side indicates success or failure. The content of the messages is mechanism-specific. If the negotiation succeeds, then the session can proceed over the connection, otherwise it must be abandoned.
Some mechanisms continue to process session data after negotiation (e.g., encrypting it), while some specify that further session data is transmitted unmodified.
Negotiation
Commands
Avro SASL negotiation uses four one-byte commands.
- 0: START Used in a client’s initial message.
- 1: CONTINUE Used while negotiation is ongoing.
- 2: FAIL Terminates negotiation unsuccessfully.
- 3: COMPLETE Terminates negotiation successfully.
The format of a START message is:
| 0 | 4-byte mechanism name length | mechanism name | 4-byte payload length | payload data |
The format of a CONTINUE message is:
| 1 | 4-byte payload length | payload data |
The format of a FAIL message is:
| 2 | 4-byte message length | UTF-8 message |
The format of a COMPLETE message is:
| 3 | 4-byte payload length | payload data |
Process
Negotiation is initiated by a client sending a START command containing the client’s chosen mechanism name and any mechanism-specific payload data.
The server and client then interchange some number (possibly zero) of CONTINUE messages. Each message contains payload data that is processed by the security mechanism to generate the next message.
Once either the client or server send a FAIL message then negotiation has failed. UTF-8-encoded text is included in the failure message. Once either a FAIL message has been sent or received, or any other error occurs in the negotiation, further communication on this connection must cease.
Once either the client or server send a COMPLETE message then negotiation has completed successfully. Session data may now be transmitted over the connection until it is closed by either side.
Session Data
If no SASL QOP (quality of protection) is negotiated, then all subsequent writes to/reads over this connection are written/read unmodified. In particular, messages use Avro framing, and are of the form:
| 4-byte frame length | frame data | ... | 4 zero bytes |
If a SASL QOP is negotiated, then it must be used by the connection for all subsequent messages. This is done by wrapping each non-empty frame written using the security mechanism and unwrapping each non-empty frame read. The length written in each non-empty frame is the length of the wrapped data. Complete frames must be passed to the security mechanism for unwrapping. Unwrapped data is then passed to the application as the content of the frame.
If at any point processing fails due to wrapping, unwrapping or framing errors, then all further communication on this connection must cease.
Anonymous Mechanism
The SASL anonymous mechanism (RFC 2245) is quite simple to implement. In particular, an initial anonymous request may be prefixed by the following static sequence:
| 0 | 0009 | ANONYMOUS | 0000 |
If a server uses the anonymous mechanism, it should check that the mechanism name in the start message prefixing the first request received is ‘ANONYMOUS’, then simply prefix its initial response with a COMPLETE message of:
| 3 | 0000 |
If an anonymous server recieves some other mechanism name, then it may respond with a FAIL message as simple as:
| 2 | 0000 |
Note that the anonymous mechanism need add no additional round-trip messages between client and server. The START message can be piggybacked on the initial request and the COMPLETE or FAIL message can be piggybacked on the initial response.
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
13.8 - Wiki
The Wiki page can be found here.
13.9 - FAQ
What is Avro?
Avro is a data serialization system.
Avro provides:
- Rich data structures.
- A compact, fast, binary data format.
- A container file, to store persistent data.
- Remote procedure call (RPC).
- Simple integration with dynamic languages. Code generation is not required to read or write data files nor to use or implement RPC protocols. Code generation as an optional optimization, only worth implementing for statically typed languages.
How can I get started quickly with Avro?
Check out the Quick Start Guide.
How do I statically compile a schema or protocol into generated code?
In Java
- Add the avro jar, the jackson-mapper-asl.jar and jackson-core-asl.jar to your CLASSPATH.
- Run java org.apache.avro.specific.SpecificCompiler .
This appears to be out of date, the SpecificCompiler requires two arguments, presumably an input and and output file, but it isn’t clear that this does.
Or use the Schema or Protocol Ant tasks. Avro’s build.xml provides examples of how these are used.
Lastly, you can also use the “avro-tools” jar which ships with an Avro release. Just use the “compile (schema|protocol)” command.
How are Strings represented in Java?
They use org.apache.avro.util.Utf8, not java.lang.String.
More generally, how do Avro types map to Java types?
The mappings are documented in the package javadoc for generic, specific and reflect API.
From Doug Cutting:
HDFS splits files into blocks, and mapreduce runs a map task for each block. When the task starts, it needs to be able to seek into the file to the start of the block process through the block’s end. If the file were, e.g., a gzip file, this would not be possible, since gzip files must be decompressed from the start. One cannot seek into the middle of a gzip file and start decompressing. So Hadoop’s SequenceFile places a marker periodically (~64k) in the file at record and compression boundaries, where processing can be sensibly started. Then, when a map task starts processing an HDFS block, it finds the first marker after the block’s start and continues through the first marker in the next block of the file. This requires a bit of non-local access (~0.1%). Avro’s data file uses the same method as SequenceFile.
Why isn’t every value in Avro nullable?
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 of every value stored and
corresponding computational costs to create this bit on write and
interpret it on read. These costs are wasted when values may not in
fact be null, as is the case in many datasets. In Avro such costs are
only paid when values may actually be null.
Also, allowing values to be null is a well-known source of errors. In
Avro, a value declared as non-null will always be non-null and programs
need not test for null values when processing it nor will they ever fail
for lack of such tests.
Tony Hoare calls his invention of null references his “Billion Dollar
Mistake”.
http://qconlondon.com/london-2009/presentation/Null+References:+The+Billion+Dollar+Mistake
Also note that in some programming languages not all values are permitted to be null. For example, in Java, values of type boolean, byte, short, char, int, float, long, and double may not be null.
How can I serialize directly to/from a byte array?
As pointed out in the specification, Avro data should always be stored with its schema. The Avro provided classes DataFileWriter, DataFileReader, and DataFileStream all ensure this by serializing the Schema in a container header. In some special cases, such as when implementing a new storage system or writing unit tests, you may need to write and read directly with the bare Avro serialized values. The following examples use code generated for Java from an Avro Schema, specifically the User example from the Quickstart guide.
Serializing to a byte array
This example takes a User object and returns a newly allocated byte array with the Avro serialization of that user.
ByteArrayOutputStream out = new ByteArrayOutputStream();
BinaryEncoder encoder = EncoderFactory.get().binaryEncoder(out, null);
DatumWriter<User> writer = new SpecificDatumWriter<User>(User.getClassSchema());
writer.write(user, encoder);
encoder.flush();
out.close();
byte[] serializedBytes = out.toByteArray();
Deserializing from a byte array
This example takes a byte array containing the Avro serialization of a user and returns a User object.SpecificDatumReader reader = new SpecificDatumReader(User.getClassSchema());
Decoder decoder = DecoderFactory.get().binaryDecoder(bytes, null);
User user = reader.read(null, decoder);
Note: If you are serializing or deserializing in a loop or as a method, you should be reusing objects, readers and/or writers. Check the JavaDoc to see how to reuse the objects.
14 - Apache Avro™ 1.7.5 Documentation
Introduction
Apache Avro™ is a data serialization system.
Avro provides:
- Rich data structures.
- A compact, fast, binary data format.
- A container file, to store persistent data.
- Remote procedure call (RPC).
- Simple integration with dynamic languages. Code generation is not required to read or write data files nor to use or implement RPC protocols. Code generation as an optional optimization, only worth implementing for statically typed languages.
Schemas
Avro relies on schemas. When Avro data is read, the schema used when writing it is always present. This permits each datum to be written with no per-value overheads, making serialization both fast and small. This also facilitates use with dynamic, scripting languages, since data, together with its schema, is fully self-describing.
When Avro data is stored in a file, its schema is stored with it, so that files may be processed later by any program. If the program reading the data expects a different schema this can be easily resolved, since both schemas are present.
When Avro is used in RPC, the client and server exchange schemas in the connection handshake. (This can be optimized so that, for most calls, no schemas are actually transmitted.) Since both client and server both have the other’s full schema, correspondence between same named fields, missing fields, extra fields, etc. can all be easily resolved.
Avro schemas are defined with JSON . This facilitates implementation in languages that already have JSON libraries.
Comparison with other systems
Avro provides functionality similar to systems such as Thrift, Protocol Buffers, etc. Avro differs from these systems in the following fundamental aspects.
- Dynamic typing: Avro does not require that code be generated. Data is always accompanied by a schema that permits full processing of that data without code generation, static datatypes, etc. This facilitates construction of generic data-processing systems and languages.
- Untagged data: Since the schema is present when data is read, considerably less type information need be encoded with data, resulting in smaller serialization size.
- No manually-assigned field IDs: When a schema changes, both the old and new schema are always present when processing data, so differences may be resolved symbolically, using field names.
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
14.1 - Getting Started (Java)
This is a short guide for getting started with Apache Avro™ using Java. This guide only covers using Avro for data serialization; see Patrick Hunt’s Avro RPC Quick Start for a good introduction to using Avro for RPC.
Download
Avro implementations for C, C++, C#, Java, PHP, Python, and Ruby can be downloaded from the Apache Avro™ Releases page. This guide uses Avro 1.7.5, the latest version at the time of writing. For the examples in this guide, download avro-1.7.5.jar and avro-tools-1.7.5.jar. The Avro Java implementation also depends on the Jackson JSON library. From the Jackson download page, download the core-asl and mapper-asl jars. Add avro-1.7.5.jar and the Jackson jars to your project’s classpath (avro-tools will be used for code generation).
Alternatively, if you are using Maven, add the following dependency to your POM:
<dependency>
<groupId>org.apache.avro</groupId>
<artifactId>avro</artifactId>
<version>1.7.5</version>
</dependency>
As well as the Avro Maven plugin (for performing code generation):
<plugin>
<groupId>org.apache.avro</groupId>
<artifactId>avro-maven-plugin</artifactId>
<version>1.7.5</version>
<executions>
<execution>
<phase>generate-sources</phase>
<goals>
<goal>schema</goal>
</goals>
<configuration>
<sourceDirectory>${project.basedir}/src/main/avro/</sourceDirectory>
<outputDirectory>${project.basedir}/src/main/java/</outputDirectory>
</configuration>
</execution>
</executions>
</plugin>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-compiler-plugin</artifactId>
<configuration>
<source>1.6</source>
<target>1.6</target>
</configuration>
</plugin>
You may also build the required Avro jars from source. Building Avro is beyond the scope of this guide; see the Build Documentation page in the wiki for more information.
Defining a schema
Avro schemas are defined using JSON. Schemas are composed of primitive types (null, boolean, int, long, float, double, bytes, and string) and complex types (record, enum, array, map, union, and fixed). You can learn more about Avro schemas and types from the specification, but for now let’s start with a simple schema example, user.avsc:
{"namespace": "example.avro",
"type": "record",
"name": "User",
"fields": [
{"name": "name", "type": "string"},
{"name": "favorite_number", "type": ["int", "null"]},
{"name": "favorite_color", "type": ["string", "null"]}
]
}
This schema defines a record representing a hypothetical user. (Note that a schema file can only contain a single schema definition.) At minimum, a record definition must include its type (“type”: “record”), a name (“name”: “User”), and fields, in this case name, favorite_number, and favorite_color. We also define a namespace (“namespace”: “example.avro”), which together with the name attribute defines the “full name” of the schema (example.avro.User in this case).
Fields are defined via an array of objects, each of which defines a name and type (other attributes are optional, see the record specification for more details). The type attribute of a field is another schema object, which can be either a primitive or complex type. For example, the name field of our User schema is the primitive type string, whereas the favorite_number and favorite_color fields are both unions, represented by JSON arrays. unions are a complex type that can be any of the types listed in the array; e.g., favorite_number can either be an int or null, essentially making it an optional field.
Serializing and deserializing with code generation
Compiling the schema
Code generation allows us to automatically create classes based on our previously-defined schema. Once we have defined the relevant classes, there is no need to use the schema directly in our programs. We use the avro-tools jar to generate code as follows:
java -jar /path/to/avro-tools-1.7.5.jar compile schema <schema file> <destination>
This will generate the appropriate source files in a package based on the schema’s namespace in the provided destination folder. For instance, to generate a User class in package example.avro from the schema defined above, run
java -jar /path/to/avro-tools-1.7.5.jar compile schema user.avsc .
Note that if you using the Avro Maven plugin, there is no need to manually invoke the schema compiler; the plugin automatically performs code generation on any .avsc files present in the configured source directory.
Creating Users
Now that we’ve completed the code generation, let’s create some Users, serialize them to a data file on disk, and then read back the file and deserialize the User objects.
First let’s create some Users and set their fields.
User user1 = new User();
user1.setName("Alyssa");
user1.setFavoriteNumber(256);
// Leave favorite color null
// Alternate constructor
User user2 = new User("Ben", 7, "red");
// Construct via builder
User user3 = User.newBuilder()
.setName("Charlie")
.setFavoriteColor("blue")
.setFavoriteNumber(null)
.build();
As shown in this example, Avro objects can be created either by invoking a constructor directly or by using a builder. Unlike constructors, builders will automatically set any default values specified in the schema. Additionally, builders validate the data as it set, whereas objects constructed directly will not cause an error until the object is serialized. However, using constructors directly generally offers better performance, as builders create a copy of the datastructure before it is written.
Note that we do not set user1’s favorite color. Since that record is of type [“string”, “null”], we can either set it to a string or leave it null; it is essentially optional. Similarly, we set user3’s favorite number to null (using a builder requires setting all fields, even if they are null).
Serializing
Now let’s serialize our Users to disk.
/// Serialize user1, user2 and user3 to disk
DatumWriter<User> userDatumWriter = new SpecificDatumWriter<User>(User.class);
DataFileWriter<User> dataFileWriter = new DataFileWriter<User>(userDatumWriter);
dataFileWriter.create(user1.getSchema(), new File("users.avro"));
dataFileWriter.append(user1);
dataFileWriter.append(user2);
dataFileWriter.append(user3);
dataFileWriter.close();
We create a DatumWriter, which converts Java objects into an in-memory serialized format. The SpecificDatumWriter class is used with generated classes and extracts the schema from the specified generated type.
Next we create a DataFileWriter, which writes the serialized records, as well as the schema, to the file specified in the dataFileWriter.create call. We write our users to the file via calls to the dataFileWriter.append method. When we are done writing, we close the data file.
Deserializing
Finally, let’s deserialize the data file we just created.
// Deserialize Users from disk
DatumReader<User> userDatumReader = new SpecificDatumReader<User>(User.class);
DataFileReader<User> dataFileReader = new DataFileReader<User>(file, userDatumReader);
User user = null;
while (dataFileReader.hasNext()) {
// Reuse user object by passing it to next(). This saves us from
// allocating and garbage collecting many objects for files with
// many items.
user = dataFileReader.next(user);
System.out.println(user);
}
This snippet will output:
{"name": "Alyssa", "favorite_number": 256, "favorite_color": null}
{"name": "Ben", "favorite_number": 7, "favorite_color": "red"}
{"name": "Charlie", "favorite_number": null, "favorite_color": "blue"}
Deserializing is very similar to serializing. We create a SpecificDatumReader, analogous to the SpecificDatumWriter we used in serialization, which converts in-memory serialized items into instances of our generated class, in this case User. We pass the DatumReader and the previously created File to a DataFileReader, analogous to the DataFileWriter, which reads the data file on disk.
Next we use the DataFileReader to iterate through the serialized Users and print the deserialized object to stdout. Note how we perform the iteration: we create a single User object which we store the current deserialized user in, and pass this record object to every call of dataFileReader.next. This is a performance optimization that allows the DataFileReader to reuse the same User object rather than allocating a new User for every iteration, which can be very expensive in terms of object allocation and garbage collection if we deserialize a large data file. While this technique is the standard way to iterate through a data file, it’s also possible to use for (User user : dataFileReader) if performance is not a concern.
Compiling and running the example code
This example code is included as a Maven project in the examples/java-example directory in the Avro docs. From this directory, execute the following commands to build and run the example:
$ mvn compile # includes code generation via Avro Maven plugin
$ mvn -q exec:java -Dexec.mainClass=example.SpecificMain
Serializing and deserializing without code generation
Data in Avro is always stored with its corresponding schema, meaning we can always read a serialized item regardless of whether we know the schema ahead of time. This allows us to perform serialization and deserialization without code generation.
Let’s go over the same example as in the previous section, but without using code generation: we’ll create some users, serialize them to a data file on disk, and then read back the file and deserialize the users objects.
Creating users
First, we use a Parser to read our schema definition and create a Schema object.
Schema schema = new Parser().parse(new File("user.avsc"));
Using this schema, let’s create some users.
GenericRecord user1 = new GenericData.Record(schema);
user1.put("name", "Alyssa");
user1.put("favorite_number", 256);
// Leave favorite color null
GenericRecord user2 = new GenericData.Record(schema);
user2.put("name", "Ben");
user2.put("favorite_number", 7);
user2.put("favorite_color", "red");
Since we’re not using code generation, we use GenericRecords to represent users. GenericRecord uses the schema to verify that we only specify valid fields. If we try to set a non-existent field (e.g., user1.put(“favorite_animal”, “cat”)), we’ll get an AvroRuntimeException when we run the program.
Note that we do not set user1’s favorite color. Since that record is of type [“string”, “null”], we can either set it to a string or leave it null; it is essentially optional.
Serializing
Now that we’ve created our user objects, serializing and deserializing them is almost identical to the example above which uses code generation. The main difference is that we use generic instead of specific readers and writers.
First we’ll serialize our users to a data file on disk.
// Serialize user1 and user2 to disk
DatumWriter<GenericRecord> datumWriter = new GenericDatumWriter<GenericRecord>(schema);
DataFileWriter<GenericRecord> dataFileWriter = new DataFileWriter<GenericRecord>(datumWriter);
dataFileWriter.create(schema, file);
dataFileWriter.append(user1);
dataFileWriter.append(user2);
dataFileWriter.close();
We create a DatumWriter, which converts Java objects into an in-memory serialized format. Since we are not using code generation, we create a GenericDatumWriter. It requires the schema both to determine how to write the GenericRecords and to verify that all non-nullable fields are present.
As in the code generation example, we also create a DataFileWriter, which writes the serialized records, as well as the schema, to the file specified in the dataFileWriter.create call. We write our users to the file via calls to the dataFileWriter.append method. When we are done writing, we close the data file.
Deserializing
Finally, we’ll deserialize the data file we just created.
// Deserialize users from disk
DatumReader<GenericRecord> datumReader = new GenericDatumReader<GenericRecord>(schema);
DataFileReader<GenericRecord> dataFileReader = new DataFileReader<GenericRecord>(file, datumReader);
GenericRecord user = null;
while (dataFileReader.hasNext()) {
// Reuse user object by passing it to next(). This saves us from
// allocating and garbage collecting many objects for files with
// many items.
user = dataFileReader.next(user);
System.out.println(user);
This outputs:
{"name": "Alyssa", "favorite_number": 256, "favorite_color": null}
{"name": "Ben", "favorite_number": 7, "favorite_color": "red"}
Deserializing is very similar to serializing. We create a GenericDatumReader, analogous to the GenericDatumWriter we used in serialization, which converts in-memory serialized items into GenericRecords. We pass the DatumReader and the previously created File to a DataFileReader, analogous to the DataFileWriter, which reads the data file on disk.
Next, we use the DataFileReader to iterate through the serialized users and print the deserialized object to stdout. Note how we perform the iteration: we create a single GenericRecord object which we store the current deserialized user in, and pass this record object to every call of dataFileReader.next. This is a performance optimization that allows the DataFileReader to reuse the same record object rather than allocating a new GenericRecord for every iteration, which can be very expensive in terms of object allocation and garbage collection if we deserialize a large data file. While this technique is the standard way to iterate through a data file, it’s also possible to use for (GenericRecord user : dataFileReader) if performance is not a concern.
Compiling and running the example code
This example code is included as a Maven project in the examples/java-example directory in the Avro docs. From this directory, execute the following commands to build and run the example:
$ mvn compile
$ mvn -q exec:java -Dexec.mainClass=example.GenericMain
14.2 - Getting Started (Python)
This is a short guide for getting started with Apache Avro™ using Python. This guide only covers using Avro for data serialization; see Patrick Hunt’s Avro RPC Quick Start for a good introduction to using Avro for RPC.
Download
Avro implementations for C, C++, C#, Java, PHP, Python, and Ruby can be downloaded from the Apache Avro™ Releases page. This guide uses Avro 1.7.5, the latest version at the time of writing. Download and unzip avro-1.7.5.tar.gz, and install via python setup.py (this will probably require root privileges). Ensure that you can import avro from a Python prompt.
$ tar xvf avro-1.7.5.tar.gz
$ cd avro-1.7.5
$ sudo python setup.py install
$ python
>>> import avro # should not raise ImportError
Alternatively, you may build the Avro Python library from source. From your the root Avro directory, run the commands
$ cd lang/py/
$ ant
$ sudo python setup.py install
$ python
>>> import avro # should not raise ImportError
Defining a schema
Avro schemas are defined using JSON. Schemas are composed of primitive types (null, boolean, int, long, float, double, bytes, and string) and complex types (record, enum, array, map, union, and fixed). You can learn more about Avro schemas and types from the specification, but for now let’s start with a simple schema example, user.avsc:
{"namespace": "example.avro",
"type": "record",
"name": "User",
"fields": [
{"name": "name", "type": "string"},
{"name": "favorite_number", "type": ["int", "null"]},
{"name": "favorite_color", "type": ["string", "null"]}
]
}
This schema defines a record representing a hypothetical user. (Note that a schema file can only contain a single schema definition.) At minimum, a record definition must include its type (“type”: “record”), a name (“name”: “User”), and fields, in this case name, favorite_number, and favorite_color. We also define a namespace (“namespace”: “example.avro”), which together with the name attribute defines the “full name” of the schema (example.avro.User in this case).
Fields are defined via an array of objects, each of which defines a name and type (other attributes are optional, see the record specification for more details). The type attribute of a field is another schema object, which can be either a primitive or complex type. For example, the name field of our User schema is the primitive type string, whereas the favorite_number and favorite_color fields are both unions, represented by JSON arrays. unions are a complex type that can be any of the types listed in the array; e.g., favorite_number can either be an int or null, essentially making it an optional field.
Serializing and deserializing without code generation
Data in Avro is always stored with its corresponding schema, meaning we can always read a serialized item, regardless of whether we know the schema ahead of time. This allows us to perform serialization and deserialization without code generation. Note that the Avro Python library does not support code generation.
Try running the following code snippet, which serializes two users to a data file on disk, and then reads back and deserializes the data file:
import avro.schema
from avro.datafile import DataFileReader, DataFileWriter
from avro.io import DatumReader, DatumWriter
schema = avro.schema.parse(open("user.avsc").read())
writer = DataFileWriter(open("users.avro", "w"), DatumWriter(), schema)
writer.append({"name": "Alyssa", "favorite_number": 256})
writer.append({"name": "Ben", "favorite_number": 7, "favorite_color": "red"})
writer.close()
reader = DataFileReader(open("users.avro", "r"), DatumReader())
for user in reader:
print user
reader.close()
This outputs:
{u'favorite_color': None, u'favorite_number': 256, u'name': u'Alyssa'}
{u'favorite_color': u'red', u'favorite_number': 7, u'name': u'Ben'}
Let’s take a closer look at what’s going on here.
schema = avro.schema.parse(open("user.avsc").read())
avro.schema.parse takes a string containing a JSON schema definition as input and outputs a avro.schema.Schema object (specifically a subclass of Schema, in this case RecordSchema). We’re passing in the contents of our user.avsc schema file here.
writer = DataFileWriter(open("users.avro", "w"), DatumWriter(), schema)
We create a DataFileWriter, which we’ll use to write serialized items to a data file on disk. The DataFileWriter constructor takes three arguments:
- The file we’ll serialize to
- A DatumWriter, which is responsible for actually serializing the items to Avro’s binary format (DatumWriters can be used separately from DataFileWriters, e.g., to perform IPC with Avro TODO: is this true??).
- The schema we’re using. The DataFileWriter needs the schema both to write the schema to the data file, and to verify that the items we write are valid items and write the appropriate fields.
writer.append({"name": "Alyssa", "favorite_number": 256})
writer.append({"name": "Ben", "favorite_number": 7, "favorite_color": "red"})
We use DataFileWriter.append to add items to our data file. Avro records are represented as Python dicts. Since the field favorite_color has type [“int”, “null”], we are not required to specify this field, as shown in the first append. Were we to omit the required name field, an exception would be raised. Any extra entries not corresponding to a field are present in the dict are ignored.
reader = DataFileReader(open("users.avro", "r"), DatumReader())
We open the file again, this time for reading back from disk. We use a DataFileReader and DatumReader analagous to the DataFileWriter and DatumWriter above.
for user in reader:
print user
The DataFileReader is an iterator that returns dicts corresponding to the serialized items.
14.3 - Specification
Introduction
This document defines Apache Avro. It is intended to be the authoritative specification. Implementations of Avro must adhere to this document.
Schema Declaration
A Schema is represented in JSON by one of:
- A JSON string, naming a defined type.
- A JSON object, of the form:
{"type": "typeName" ...attributes...}
where typeName is either a primitive or derived type name, as defined below. Attributes not defined in this document are permitted as metadata, but must not affect the format of serialized data.
- A JSON array, representing a union of embedded types.
Primitive Types
The set of primitive type names is:
- null: no value
- boolean: a binary value
- int: 32-bit signed integer
- long: 64-bit signed integer
- float: single precision (32-bit) IEEE 754 floating-point number
- double: double precision (64-bit) IEEE 754 floating-point number
- bytes: sequence of 8-bit unsigned bytes
- string: unicode character sequence
Primitive types have no specified attributes.
Primitive type names are also defined type names. Thus, for example, the schema “string” is equivalent to:
Complex Types
Avro supports six kinds of complex types: records, enums, arrays, maps, unions and fixed.
Records
Records use the type name “record” and support three attributes:
- name: a JSON string providing the name of the record (required).
- namespace, a JSON string that qualifies the name;
- doc: a JSON string providing documentation to the user of this schema (optional).
- aliases: a JSON array of strings, providing alternate names for this record (optional).
- fields: a JSON array, listing fields (required). Each field is a JSON object with the following attributes:
- name: a JSON string providing the name of the field (required), and
- doc: a JSON string describing this field for users (optional).
- type: A JSON object defining a schema, or a JSON string naming a record definition (required).
- _default: A default value for this field, used when reading instances that lack this field (optional). 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. Default values for bytes and fixed fields are JSON strings, where Unicode code points 0-255 are mapped to unsigned 8-bit byte values 0-255.
field default values
| avro type |
json type |
example |
| null |
null |
null |
| boolean |
boolean |
true |
| int,long |
integer |
1 |
| float,double |
number |
1.1 |
| bytes |
string |
"\u00FF" |
| string |
string |
"foo" |
| record |
object |
{"a": 1} |
| enum |
string |
"FOO" |
| array |
array |
[1] |
| map |
object |
{"a": 1} |
| fixed |
string |
"\u00ff" |
- order: specifies how this field impacts sort ordering of this record (optional). Valid values are “ascending” (the default), “descending”, or “ignore”. For more details on how this is used, see the sort order section below.
- aliases: a JSON array of strings, providing alternate names for this field (optional).
For example, a linked-list of 64-bit values may be defined with:
{
"type": "record",
"name": "LongList",
"aliases": ["LinkedLongs"], // old name for this
"fields" : [
{"name": "value", "type": "long"}, // each element has a long
{"name": "next", "type": ["LongList", "null"]} // optional next element
]
}
Enums
Enums use the type name “enum” and support the following attributes:
- name: a JSON string providing the name of the enum (required).
- namespace, a JSON string that qualifies the name ;
- aliases: a JSON array of strings, providing alternate names for this enum (optional).
- doc: a JSON string providing documentation to the user of this schema (optional).
- symbols: a JSON array, listing symbols, as JSON strings (required). All symbols in an enum must be unique; duplicates are prohibited.
For example, playing card suits might be defined with:
{
"type": "enum",
"name": "Suit",
"symbols" : ["SPADES", "HEARTS", "DIAMONDS", "CLUBS"]
}
Arrays
Arrays use the type name “array” and support a single attribute:
- items: the schema of the array’s items.
For example, an array of strings is declared with:
{
"type": "array",
"items" : "string"
}
Maps
Maps use the type name “map” and support one attribute:
- values: the schema of the map’s values.
Map keys are assumed to be strings.
For example, a map from string to long is declared with:
{
"type": "map",
"values" : "long"
}
Unions
Unions, as mentioned above, are represented using JSON arrays. For example, ["string", "null"] declares a schema which may be either a string or null.
Unions may not contain more than one schema with the same type, except for the named types record, fixed and enum. For example, unions containing two array types or two map types are not permitted, but two types with different names are permitted. (Names permit efficient resolution when reading and writing unions.)
Unions may not immediately contain other unions.
Fixed
Fixed uses the type name “fixed” and supports two attributes:
- name: a string naming this fixed (required).
- namespace, a string that qualifies the name;
- aliases: a JSON array of strings, providing alternate names for this enum (optional).
- size: an integer, specifying the number of bytes per value (required).
For example, 16-byte quantity may be declared with:
{"type": "fixed", "size": 16, "name": "md5"}
Names
Record, enums and fixed are named types. Each has a fullname that is composed of two parts; a name and a namespace. Equality of names is defined on the fullname.
The name portion of a fullname, record field names, and enum symbols must:
- start with [A-Za-z_]
- subsequently contain only [A-Za-z0-9_]
A namespace is a dot-separated sequence of such names. Equality of names (including field names and enum symbols) as well as fullnames is case-sensitive.
In record, enum and fixed definitions, the fullname is determined in one of the following ways:
A name and namespace are both specified. For example, one might use “name”: “X”, “namespace”: “org.foo” to indicate the fullname org.foo.X.
-
A fullname is specified. If the name specified contains a dot, then it is assumed to be a fullname, and any namespace also specified is ignored. For example, use “name”: “org.foo.X” to indicate the fullname org.foo.X.
-
A name only is specified, i.e., a name that contains no dots. In this case the namespace is taken from the most tightly enclosing schema or protocol. For example, if “name”: “X” is specified, and this occurs within a field of the record definition of org.foo.Y, then the fullname is org.foo.X.
References to previously defined names are as in the latter two cases above: if they contain a dot they are a fullname, if they do not contain a dot, the namespace is the namespace of the enclosing definition.
Primitive type names have no namespace and their names may not be defined in any namespace.
A schema or protocol may not contain multiple definitions of a fullname. Further, a name must be defined before it is used (“before” in the depth-first, left-to-right traversal of the JSON parse tree, where the types attribute of a protocol is always deemed to come “before” the messages attribute.)
Aliases
Named types and fields may have aliases. An implementation may optionally use aliases to map a writer’s schema to the reader’s. This facilitates both schema evolution as well as processing disparate datasets.
Aliases function by re-writing the writer’s schema using aliases from the reader’s schema. For example, if the writer’s schema was named “Foo” and the reader’s schema is named “Bar” and has an alias of “Foo”, then the implementation would act as though “Foo” were named “Bar” when reading. Similarly, if data was written as a record with a field named “x” and is read as a record with a field named “y” with alias “x”, then the implementation would act as though “x” were named “y” when reading.
A type alias may be specified either as a fully namespace-qualified, or relative to the namespace of the name it is an alias for. For example, if a type named “a.b” has aliases of “c” and “x.y”, then the fully qualified names of its aliases are “a.c” and “x.y”.
Data Serialization
Avro data is always serialized with its schema. Files that store Avro data should always also include the schema for that data in the same file. Avro-based remote procedure call (RPC) systems must also guarantee that remote recipients of data have a copy of the schema used to write that data.
Because the schema used to write data is always available when the data is read, Avro data itself is not tagged with type information. The schema is required to parse data.
In general, both serialization and deserialization proceed as a depth-first, left-to-right traversal of the schema, serializing primitive types as they are encountered.
Encodings
Avro specifies two serialization encodings: binary and JSON. Most applications will use the binary encoding, as it is smaller and faster. But, for debugging and web-based applications, the JSON encoding may sometimes be appropriate.
Binary Encoding
Primitive Types
Primitive types are encoded in binary as follows:
- null is written as zero bytes.
- a boolean is written as a single byte whose value is either 0 (false) or 1 (true).
- int and long values are written using variable-length zig-zag coding. Some examples:
| value |
hex |
| 0 |
00 |
| -1 |
01 |
| 1 |
02 |
| -2 |
03 |
| 2 |
04 |
| … |
|
| -64 |
7f |
| 64 |
80 01 |
| … |
|
- a float is written as 4 bytes. The float is converted into a 32-bit integer using a method equivalent to Java’s floatToIntBits and then encoded in little-endian format.
- a double is written as 8 bytes. The double is converted into a 64-bit integer using a method equivalent to Java’s doubleToLongBits and then encoded in little-endian format.
- bytes are encoded as a long followed by that many bytes of data.
- a string is encoded as a long followed by that many bytes of UTF-8 encoded character data.
For example, the three-character string “foo” would be encoded as the long value 3 (encoded as hex 06) followed by the UTF-8 encoding of ‘f’, ‘o’, and ‘o’ (the hex bytes 66 6f 6f):
06 66 6f 6f
Complex Types
Complex types are encoded in binary as follows:
Records
A record is encoded by encoding the values of its fields in the order that they are declared. In other words, a record is encoded as just the concatenation of the encodings of its fields. Field values are encoded per their schema.
For example, the record schema
{
"type": "record",
"name": "test",
"fields" : [
{"name": "a", "type": "long"},
{"name": "b", "type": "string"}
]
}
An instance of this record whose a field has value 27 (encoded as hex 36) and whose b field has value “foo” (encoded as hex bytes 06 66 6f 6f), would be encoded simply as the concatenation of these, namely the hex byte sequence:
36 06 66 6f 6f
Enums
An enum is encoded by a int, representing the zero-based position of the symbol in the schema.
For example, consider the enum:
{"type": "enum", "name": "Foo", "symbols": ["A", "B", "C", "D"] }
This would be encoded by an int between zero and three, with zero indicating “A”, and 3 indicating “D”.
Arrays
Arrays are encoded as a series of blocks. Each block consists of a long count value, followed by that many array items. A block with count zero indicates the end of the array. Each item is encoded per the array’s item schema.
If a block’s count is negative, its absolute value is used, and the count is followed immediately by a long block size indicating the number of bytes in the block. This block size permits fast skipping through data, e.g., when projecting a record to a subset of its fields.
For example, the array schema
{"type": "array", "items": "long"}
an array containing the items 3 and 27 could be encoded as the long value 2 (encoded as hex 04) followed by long values 3 and 27 (encoded as hex 06 36) terminated by zero:
04 06 36 00
The blocked representation permits one to read and write arrays larger than can be buffered in memory, since one can start writing items without knowing the full length of the array.
Maps
Maps are encoded as a series of blocks. Each block consists of a long count value, followed by that many key/value pairs. A block with count zero indicates the end of the map. Each item is encoded per the map’s value schema.
If a block’s count is negative, its absolute value is used, and the count is followed immediately by a long block size indicating the number of bytes in the block. This block size permits fast skipping through data, e.g., when projecting a record to a subset of its fields.
The blocked representation permits one to read and write maps larger than can be buffered in memory, since one can start writing items without knowing the full length of the map.
Unions
A union is encoded by first writing a long value indicating the zero-based position within the union of the schema of its value. The value is then encoded per the indicated schema within the union.
For example, the union schema ["string","null"] would encode:
-
null as the integer 1 (the index of “null” in the union, encoded as hex 02):
02
-
the string “a” as zero (the index of “string” in the union), followed by the serialized string:
02 02 61
Fixed
Fixed instances are encoded using the number of bytes declared in the schema.
JSON Encoding
Except for unions, the JSON encoding is the same as is used to encode field default values.
The value of a union is encoded in JSON as follows:
- if its type is null, then it is encoded as a JSON null;
- otherwise it is encoded as a JSON object with one name/value pair whose name is the type’s name and whose value is the recursively encoded value. For Avro’s named types (record, fixed or enum) the user-specified name is used, for other types the type name is used.
For example, the union schema ["null","string","Foo"], where Foo is a record name, would encode:
- null as null;
- the string “a” as
{"string": "a"} and
- a Foo instance as
{"Foo": {...}}, where {...} indicates the JSON encoding of a Foo instance.
Note that a schema is still required to correctly process JSON-encoded data. For example, the JSON encoding does not distinguish between int and long, float and double, records and maps, enums and strings, etc.
Sort Order
Avro defines a standard sort order for data. This permits data written by one system to be efficiently sorted by another system. This can be an important optimization, as sort order comparisons are sometimes the most frequent per-object operation. Note also that Avro binary-encoded data can be efficiently ordered without deserializing it to objects.
Data items may only be compared if they have identical schemas. Pairwise comparisons are implemented recursively with a depth-first, left-to-right traversal of the schema. The first mismatch encountered determines the order of the items.
Two items with the same schema are compared according to the following rules.
- null data is always equal.
- boolean data is ordered with false before true.
- int, long, float and double data is ordered by ascending numeric value.
- bytes and fixed data are compared lexicographically by unsigned 8-bit values.
- string data is compared lexicographically by Unicode code point. Note that since UTF-8 is used as the binary encoding for strings, sorting of bytes and string binary data is identical.
- array data is compared lexicographically by element.
- enum data is ordered by the symbol’s position in the enum schema. For example, an enum whose symbols are
["z", "a"] would sort “z” values before “a” values.
- union data is first ordered by the branch within the union, and, within that, by the type of the branch. For example, an
["int", "string"] union would order all int values before all string values, with the ints and strings themselves ordered as defined above.
- record data is ordered lexicographically by field. If a field specifies that its order is:
- “ascending”, then the order of its values is unaltered.
- “descending”, then the order of its values is reversed.
- “ignore”, then its values are ignored when sorting.
- map data may not be compared. It is an error to attempt to compare data containing maps unless those maps are in an
"order":"ignore" record field.
Object Container Files
Avro includes a simple object container file format. A file has a schema, and all objects stored in the file must be written according to that schema, using binary encoding. Objects are stored in blocks that may be compressed. Syncronization markers are used between blocks to permit efficient splitting of files for MapReduce processing.
Files may include arbitrary user-specified metadata.
A file consists of:
- A file header, followed by
- one or more file data blocks.
A file header consists of:
- Four bytes, ASCII ‘O’, ‘b’, ‘j’, followed by 1.
- file metadata, including the schema.
- The 16-byte, randomly-generated sync marker for this file.
File metadata is written as if defined by the following map schema:
{"type": "map", "values": "bytes"}
All metadata properties that start with “avro.” are reserved. The following file metadata properties are currently used:
- avro.schema contains the schema of objects stored in the file, as JSON data (required).
- avro.codec the name of the compression codec used to compress blocks, as a string. Implementations are required to support the following codecs: “null” and “deflate”. If codec is absent, it is assumed to be “null”. The codecs are described with more detail below.
A file header is thus described by the following schema:
{"type": "record", "name": "org.apache.avro.file.Header",
"fields" : [
{"name": "magic", "type": {"type": "fixed", "name": "Magic", "size": 4}},
{"name": "meta", "type": {"type": "map", "values": "bytes"}},
{"name": "sync", "type": {"type": "fixed", "name": "Sync", "size": 16}}
]
}
A file data block consists of:
- A long indicating the count of objects in this block.
- A long indicating the size in bytes of the serialized objects in the current block, after any codec is applied
- The serialized objects. If a codec is specified, this is compressed by that codec.
- The file’s 16-byte sync marker.
Thus, each block’s binary data can be efficiently extracted or skipped without deserializing the contents. The combination of block size, object counts, and sync markers enable detection of corrupt blocks and help ensure data integrity.
Required Codecs
null
The “null” codec simply passes through data uncompressed.
deflate
The “deflate” codec writes the data block using the deflate algorithm as specified in RFC 1951, and typically implemented using the zlib library. Note that this format (unlike the “zlib format” in RFC 1950) does not have a checksum.
Optional Codecs
snappy
The “snappy” codec uses Google’s Snappy compression library. Each compressed block is followed by the 4-byte, big-endian CRC32 checksum of the uncompressed data in the block.
Protocol Declaration
Avro protocols describe RPC interfaces. Like schemas, they are defined with JSON text.
A protocol is a JSON object with the following attributes:
- protocol, a string, the name of the protocol (required);
- namespace, an optional string that qualifies the name;
- doc, an optional string describing this protocol;
- types, an optional list of definitions of named types (records, enums, fixed and errors). An error definition is just like a record definition except it uses “error” instead of “record”. Note that forward references to named types are not permitted.
- messages, an optional JSON object whose keys are message names and whose values are objects whose attributes are described below. No two messages may have the same name.
The name and namespace qualification rules defined for schema objects apply to protocols as well.
Messages
A message has attributes:
- a doc, an optional description of the message,
- a request, a list of named, typed parameter schemas (this has the same form as the fields of a record declaration);
- a response schema;
- an optional union of declared error schemas. The effective union has “string” prepended to the declared union, to permit transmission of undeclared “system” errors. For example, if the declared error union is
["AccessError"], then the effective union is ["string", "AccessError"]. When no errors are declared, the effective error union is ["string"]. Errors are serialized using the effective union; however, a protocol’s JSON declaration contains only the declared union.
- an optional one-way boolean parameter.
A request parameter list is processed equivalently to an anonymous record. Since record field lists may vary between reader and writer, request parameters may also differ between the caller and responder, and such differences are resolved in the same manner as record field differences.
The one-way parameter may only be true when the response type is "null" and no errors are listed.
Sample Protocol
For example, one may define a simple HelloWorld protocol with:
{
"namespace": "com.acme",
"protocol": "HelloWorld",
"doc": "Protocol Greetings",
"types": [
{"name": "Greeting", "type": "record", "fields": [
{"name": "message", "type": "string"}]},
{"name": "Curse", "type": "error", "fields": [
{"name": "message", "type": "string"}]}
],
"messages": {
"hello": {
"doc": "Say hello.",
"request": [{"name": "greeting", "type": "Greeting" }],
"response": "Greeting",
"errors": ["Curse"]
}
}
}
Message Transport
Messages may be transmitted via different transport mechanisms.
To the transport, a message is an opaque byte sequence.
A transport is a system that supports:
- transmission of request messages
- receipt of corresponding response messages
Servers may send a response message back to the client corresponding to a request message. The mechanism of correspondance is transport-specific. For example, in HTTP it is implicit, since HTTP directly supports requests and responses. But a transport that multiplexes many client threads over a single socket would need to tag messages with unique identifiers.
Transports may be either stateless or stateful. In a stateless transport, messaging assumes no established connection state, while stateful transports establish connections that may be used for multiple messages. This distinction is discussed further in the handshake section below.
HTTP as Transport
When HTTP is used as a transport, each Avro message exchange is an HTTP request/response pair. All messages of an Avro protocol should share a single URL at an HTTP server. Other protocols may also use that URL. Both normal and error Avro response messages should use the 200 (OK) response code. The chunked encoding may be used for requests and responses, but, regardless the Avro request and response are the entire content of an HTTP request and response. The HTTP Content-Type of requests and responses should be specified as “avro/binary”. Requests should be made using the POST method.
HTTP is used by Avro as a stateless transport.
Message Framing
Avro messages are framed as a list of buffers.
Framing is a layer between messages and the transport. It exists to optimize certain operations.
The format of framed message data is:
- a series of buffers, where each buffer consists of:
- a four-byte, big-endian buffer length, followed by
- that many bytes of buffer data.
- A message is always terminated by a zero-lenghted buffer.
Framing is transparent to request and response message formats (described below). Any message may be presented as a single or multiple buffers.
Framing can permit readers to more efficiently get different buffers from different sources and for writers to more efficiently store different buffers to different destinations. In particular, it can reduce the number of times large binary objects are copied. For example, if an RPC parameter consists of a megabyte of file data, that data can be copied directly to a socket from a file descriptor, and, on the other end, it could be written directly to a file descriptor, never entering user space.
A simple, recommended, framing policy is for writers to create a new segment whenever a single binary object is written that is larger than a normal output buffer. Small objects are then appended in buffers, while larger objects are written as their own buffers. When a reader then tries to read a large object the runtime can hand it an entire buffer directly, without having to copy it.
Handshake
The purpose of the handshake is to ensure that the client and the server have each other’s protocol definition, so that the client can correctly deserialize responses, and the server can correctly deserialize requests. Both clients and servers should maintain a cache of recently seen protocols, so that, in most cases, a handshake will be completed without extra round-trip network exchanges or the transmission of full protocol text.
RPC requests and responses may not be processed until a handshake has been completed. With a stateless transport, all requests and responses are prefixed by handshakes. With a stateful transport, handshakes are only attached to requests and responses until a successful handshake response has been returned over a connection. After this, request and response payloads are sent without handshakes for the lifetime of that connection.
The handshake process uses the following record schemas:
{
"type": "record",
"name": "HandshakeRequest", "namespace":"org.apache.avro.ipc",
"fields": [
{"name": "clientHash",
"type": {"type": "fixed", "name": "MD5", "size": 16}},
{"name": "clientProtocol", "type": ["null", "string"]},
{"name": "serverHash", "type": "MD5"},
{"name": "meta", "type": ["null", {"type": "map", "values": "bytes"}]}
]
}
{
"type": "record",
"name": "HandshakeResponse", "namespace": "org.apache.avro.ipc",
"fields": [
{"name": "match",
"type": {"type": "enum", "name": "HandshakeMatch",
"symbols": ["BOTH", "CLIENT", "NONE"]}},
{"name": "serverProtocol",
"type": ["null", "string"]},
{"name": "serverHash",
"type": ["null", {"type": "fixed", "name": "MD5", "size": 16}]},
{"name": "meta",
"type": ["null", {"type": "map", "values": "bytes"}]}
]
}
- A client first prefixes each request with a
HandshakeRequest containing just the hash of its protocol and of the server’s protocol (clientHash!=null, clientProtocol=null, serverHash!=null), where the hashes are 128-bit MD5 hashes of the JSON protocol text. If a client has never connected to a given server, it sends its hash as a guess of the server’s hash, otherwise it sends the hash that it previously obtained from this server.
The server responds with a HandshakeResponse containing one of:
-
match=BOTH, serverProtocol=null, serverHash=null if the client sent the valid hash of the server’s protocol and the server knows what protocol corresponds to the client’s hash. In this case, the request is complete and the response data immediately follows the HandshakeResponse.
-
match=CLIENT, serverProtocol!=null, serverHash!=null if the server has previously seen the client’s protocol, but the client sent an incorrect hash of the server’s protocol. The request is complete and the response data immediately follows the HandshakeResponse. The client must use the returned protocol to process the response and should also cache that protocol and its hash for future interactions with this server.
-
match=NONE if the server has not previously seen the client’s protocol. The serverHash and serverProtocol may also be non-null if the server’s protocol hash was incorrect.
In this case the client must then re-submit its request with its protocol text (clientHash!=null, clientProtocol!=null, serverHash!=null) and the server should respond with a successful match (match=BOTH, serverProtocol=null, serverHash=null) as above.
The meta field is reserved for future handshake enhancements.
A call consists of a request message paired with its resulting response or error message. Requests and responses contain extensible metadata, and both kinds of messages are framed as described above.
The format of a call request is:
- request metadata, a map with values of type bytes
- the message name, an Avro string, followed by
- the message parameters. Parameters are serialized according to the message’s request declaration.
When the empty string is used as a message name a server should ignore the parameters and return an empty response. A client may use this to ping a server or to perform a handshake without sending a protocol message.
When a message is declared one-way and a stateful connection has been established by a successful handshake response, no response data is sent. Otherwise the format of the call response is:
- response metadata, a map with values of type bytes
- a one-byte error flag boolean, followed by either:
- if the error flag is false, the message response, serialized per the message’s response schema.
- if the error flag is true, the error, serialized per the message’s effective error union schema.
Schema Resolution
A reader of Avro data, whether from an RPC or a file, can always parse that data because its schema is provided. But that schema may not be exactly the schema that was expected. For example, if the data was written with a different version of the software than it is read, then records may have had fields added or removed. This section specifies how such schema differences should be resolved.
We refer to the schema used to write the data as the writer’s schema, and the schema that the application expects the reader’s schema. Differences between these should be resolved as follows:
- It is an error if the two schemas do not match.
To match, one of the following must hold:
-
both schemas are arrays whose item types match
-
both schemas are maps whose value types match
-
both schemas are enums whose names match
-
both schemas are fixed whose sizes and names match
-
both schemas are records with the same name
-
either schema is a union
-
both schemas have same primitive type
-
the writer’s schema may be promoted to the reader’s as follows:
- int is promotable to long, float, or double
- long is promotable to float or double
- float is promotable to double
- string is promotable to bytes
- bytes is promotable to string
-
if both are records:
- the ordering of fields may be different: fields are matched by name.
- schemas for fields with the same name in both records are resolved recursively.
- if the writer’s record contains a field with a name not present in the reader’s record, the writer’s value for that field is ignored.
- if the reader’s record schema has a field that contains a default value, and writer’s schema does not have a field with the same name, then the reader should use the default value from its field.
- if the reader’s record schema has a field with no default value, and writer’s schema does not have a field with the same name, an error is signalled.
-
if both are enums:
if the writer’s symbol is not present in the reader’s enum and the reader has a default value, then that value is used, otherwise an error is signalled.
This resolution algorithm is applied recursively to the reader’s and writer’s array item schemas.
This resolution algorithm is applied recursively to the reader’s and writer’s value schemas.
The first schema in the reader’s union that matches the selected writer’s union schema is recursively resolved against it. if none match, an error is signalled.
- if reader’s is a union, but writer’s is not
The first schema in the reader’s union that matches the writer’s schema is recursively resolved against it. If none match, an error is signalled.
- if writer’s is a union, but reader’s is not
If the reader’s schema matches the selected writer’s schema, it is recursively resolved against it. If they do not match, an error is signalled.
A schema’s doc fields are ignored for the purposes of schema resolution. Hence, the doc portion of a schema may be dropped at serialization.
One of the defining characteristics of Avro is that a reader is assumed to have the “same” schema used by the writer of the data the reader is reading. This assumption leads to a data format that’s compact and also amenable to many forms of schema evolution. However, the specification so far has not defined what it means for the reader to have the “same” schema as the writer. Does the schema need to be textually identical? Well, clearly adding or removing some whitespace to a JSON expression does not change its meaning. At the same time, reordering the fields of records clearly does change the meaning. So what does it mean for a reader to have “the same” schema as a writer?
Parsing Canonical Form is a transformation of a writer’s schema that let’s us define what it means for two schemas to be “the same” for the purpose of reading data written against the schema. It is called Parsing Canonical Form because the transformations strip away parts of the schema, like “doc” attributes, that are irrelevant to readers trying to parse incoming data. It is called Canonical Form because the transformations normalize the JSON text (such as the order of attributes) in a way that eliminates unimportant differences between schemas. If the Parsing Canonical Forms of two different schemas are textually equal, then those schemas are “the same” as far as any reader is concerned, i.e., there is no serialized data that would allow a reader to distinguish data generated by a writer using one of the original schemas from data generated by a writing using the other original schema. (We sketch a proof of this property in a companion document.)
The next subsection specifies the transformations that define Parsing Canonical Form. But with a well-defined canonical form, it can be convenient to go one step further, transforming these canonical forms into simple integers (“fingerprints”) that can be used to uniquely identify schemas. The subsection after next recommends some standard practices for generating such fingerprints.
Assuming an input schema (in JSON form) that’s already UTF-8 text for a valid Avro schema (including all quotes as required by JSON), the following transformations will produce its Parsing Canonical Form:
- [PRIMITIVES] Convert primitive schemas to their simple form (e.g., int instead of
{"type":"int"}).
- [FULLNAMES] Replace short names with fullnames, using applicable namespaces to do so. Then eliminate namespace attributes, which are now redundant.
- [STRIP] Keep only attributes that are relevant to parsing data, which are: type, name, fields, symbols, items, values, size. Strip all others (e.g., doc and aliases).
- [ORDER] Order the appearance of fields of JSON objects as follows: name, type, fields, symbols, items, values, size. For example, if an object has type, name, and size fields, then the name field should appear first, followed by the type and then the size fields.
- [STRINGS] For all JSON string literals in the schema text, replace any escaped characters (e.g., \uXXXX escapes) with their UTF-8 equivalents.
- [INTEGERS] Eliminate quotes around and any leading zeros in front of JSON integer literals (which appear in the size attributes of fixed schemas).
- [WHITESPACE] Eliminate all whitespace in JSON outside of string literals.
Schema Fingerprints
“[A] fingerprinting algorithm is a procedure that maps an arbitrarily large data item (such as a computer file) to a much shorter bit string, its fingerprint, that uniquely identifies the original data for all practical purposes” (quoted from Wikipedia). In the Avro context, fingerprints of Parsing Canonical Form can be useful in a number of applications; for example, to cache encoder and decoder objects, to tag data items with a short substitute for the writer’s full schema, and to quickly negotiate common-case schemas between readers and writers.
In designing fingerprinting algorithms, there is a fundamental trade-off between the length of the fingerprint and the probability of collisions. To help application designers find appropriate points within this trade-off space, while encouraging interoperability and ease of implementation, we recommend using one of the following three algorithms when fingerprinting Avro schemas:
- When applications can tolerate longer fingerprints, we recommend using the SHA-256 digest algorithm to generate 256-bit fingerprints of Parsing Canonical Forms. Most languages today have SHA-256 implementations in their libraries.
- At the opposite extreme, the smallest fingerprint we recommend is a 64-bit Rabin fingerprint. Below, we provide pseudo-code for this algorithm that can be easily translated into any programming language. 64-bit fingerprints should guarantee uniqueness for schema caches of up to a million entries (for such a cache, the chance of a collision is 3E-8). We don’t recommend shorter fingerprints, as the chances of collisions is too great (for example, with 32-bit fingerprints, a cache with as few as 100,000 schemas has a 50% chance of having a collision).
- Between these two extremes, we recommend using the MD5 message digest to generate 128-bit fingerprints. These make sense only where very large numbers of schemas are being manipulated (tens of millions); otherwise, 64-bit fingerprints should be sufficient. As with SHA-256, MD5 implementations are found in most libraries today.
These fingerprints are not meant to provide any security guarantees, even the longer SHA-256-based ones. Most Avro applications should be surrounded by security measures that prevent attackers from writing random data and otherwise interfering with the consumers of schemas. We recommend that these surrounding mechanisms be used to prevent collision and pre-image attacks (i.e., “forgery”) on schema fingerprints, rather than relying on the security properties of the fingerprints themselves.
Rabin fingerprints are cyclic redundancy checks computed using irreducible polynomials. In the style of the Appendix of RFC 1952 (pg 10), which defines the CRC-32 algorithm, here’s our definition of the 64-bit AVRO fingerprinting algorithm:
long fingerprint64(byte[] buf) {
if (FP_TABLE == null) initFPTable();
long fp = EMPTY;
for (int i = 0; i < buf.length; i++)
fp = (fp >>> 8) ^ FP_TABLE[(int)(fp ^ buf[i]) & 0xff];
return fp;
}
static long EMPTY = 0xc15d213aa4d7a795L;
static long[] FP_TABLE = null;
void initFPTable() {
FP_TABLE = new long[256];
for (int i = 0; i < 256; i++) {
long fp = i;
for (int j = 0; j < 8; j++)
fp = (fp >>> 1) ^ (EMPTY & -(fp & 1L));
FP_TABLE[i] = fp;
}
}
Readers interested in the mathematics behind this algorithm may want to read this book chapter. (Unlike RFC-1952 and the book chapter, we prepend a single one bit to messages. We do this because CRCs ignore leading zero bits, which can be problematic. Our code prepends a one-bit by initializing fingerprints using EMPTY, rather than initializing using zero as in RFC-1952 and the book chapter.)
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
14.4 - Trevni
Version 0.1
DRAFT
This document is the authoritative specification of a file format. Its intent is to permit compatible, independent implementations that read and/or write files in this format.
Introduction
Data sets are often described as a table composed of rows and columns. Each record in the dataset is considered a row, with each field of the record occupying a different column. Writing records to a file one-by-one as they are created results in a row-major format, like Hadoop’s SequenceFile or Avro data files.
In many cases higher query performance may be achieved if the data is instead organized in a column-major format, where multiple values of a given column are stored adjacently. This document defines such a column-major file format for datasets.
To permit scalable, distributed query evaluation, datasets are partitioned into row groups, containing distinct collections of rows. Each row group is organized in column-major order, while row groups form a row-major partitioning of the entire dataset.
Rationale
Goals
The format is meant satisfy the following goals:
Maximize the size of row groups. Disc drives are used most efficiently when sequentially accessing data. Consider a drive that takes 10ms to seek and transfers at 100MB/second. If a 10-column dataset whose values are all the same size is split into 10MB row groups, then accessing a single column will require a sequence of seek+1MB reads, for a cost of 20ms/MB processed. If the same dataset is split into 100MB row groups then this drops to 11ms/MB processed. This effect is exaggerated for datasets with larger numbers of columns and with columns whose values are smaller than average. So we’d prefer row groups that are 100MB or greater.
Permit random access within a row group. Some queries will first examine one column, and, only when certain relatively rare criteria are met, examine other columns. Rather than iterating through selected columns of the row-group in parallel, one might iterate through one column and randomly access another. This is called support for WHERE clauses, after the SQL operator of that name.
Minimize the number of files per dataset. HDFS is a primary intended deployment platform for these files. The HDFS Namenode requires memory for each file in the filesystem, thus for a format to be HDFS-friendly it should strive to require the minimum number of distinct files.
Support co-location of columns within row-groups. Row groups are the unit of parallel operation on a column dataset. For efficient file i/o, the entirety of a row-group should ideally reside on the host that is evaluating the query in order to avoid network latencies and bottlenecks.
Data integrity. The format should permit applications to detect data corruption. Many file systems may prevent corruption, but files may be moved between filesystems and be subject to corruption at points in that process. It is best if the data in a file can be validated independently.
Extensibility. The format should permit applications to store additional annotations about a datasets in the files, such as type information, origin, etc. Some environments may have metadata stores for such information, but not all do, and files might be moved among systems with different metadata systems. The ability to keep such information within the file simplifies the coordination of such information.
Minimal overhead. The column format should not make datasets appreciably larger. Storage is a primary cost and a choice to use this format should not require additional storage.
Primary format. The column format should be usable as a primary format for datasets, not as an auxiliary, accelerated format. Applications that process a dataset in row-major order should be able to easily consume column files and applications that produce datasets in row-major order should be able to easily generate column files.
Design
To meet these goals we propose the following design.
Each row group is a separate file. All values of a column in a file are written contiguously. This maximizes the row group size, optimizing performance when querying few and small columns.
Each file occupies a single HDFS block. A larger than normal block size may be specified, e.g., ~1GB instead of the typical ~100MB. This guarantees co-location and eliminates network use when query processing can be co-located with the file. This also moderates the memory impact on the HDFS Namenode since no small files are written.
Each column in a file is written as a sequence of ~64kB compressed blocks. The sequence is prefixed by a table describing all of the blocks in the column to permit random access within the column.
Application-specific metadata may be added at the file, column, and block levels.
Checksums are included with each block, providing data integrity.
Discussion
The use of a single block per file achieves the same effect as the custom block placement policy described in the CIF paper, but while still permitting HDFS rebalancing and not increasing the number of files in the namespace.
This section formally describes the proposed column file format.
Data Model
We assume a simple data model, where a record is a set of named fields, and the value of each field is a sequence of untyped bytes. A type system may be layered on top of this, as specified in the Type Mapping section below.
Primitive Values
We define the following primitive value types:
- Signed 64-bit long values are written using a variable-length zig-zag coding, where the high-order bit in each byte determines whether subsequent bytes are present. For example:
decimal value hex bytes
| decimal value |
hex bytes |
|
| 0 |
00 |
|
| -1 |
01 |
|
| 1 |
02 |
|
| … |
|
|
| -64 |
7f |
|
| 64 |
80 01 |
|
| … |
|
|
- bytes are encoded as a long followed by that many bytes of data.
- a string is encoded as a long followed by that many bytes of UTF-8 encoded character data.
For example, the three-character string “foo” would be encoded as the long value 3 (encoded as hex 06) followed by the UTF-8 encoding of ‘f’, ‘o’, and ‘o’ (the hex bytes 66 6f 6f): 06 66 6f 6f
Type Names
The following type names are used to describe column values:
- null, requires zero bytes. Sometimes used in array columns.
- boolean, one bit, packed into bytes, little-endian;
- int, like long, but restricted to 32-bit signed values
- long 64-bit signed values, represented as above
- fixed32 32-bit values stored as four bytes, little-endian.
- fixed64 64-bit values stored as eight bytes, little-endian.
- float 32-bit IEEE floating point value, little-endian
- double 64-bit IEEE floating point value, little-endian
- string as above
- bytes as above, may be used to encapsulate more complex objects
Type names are represented as strings (UTF-8 encoded, length-prefixed).
Metadata consists of:
- A long indicating the number of metadata key/value pairs.
- For each pair, a string key and bytes value.
All metadata properties that start with “trevni.” are reserved.
The following file metadata properties are defined:
- trevni.codec the name of the default compression codec used to compress blocks, as a string. Implementations are required to support the “null” codec. Optional. If absent, it is assumed to be “null”. Codecs are described in more detail below.
- trevni.checksum the name of the checksum algorithm used in this file, as a string. Implementations are required to support the “crc-32” checksum. Optional. If absent, it is assumed to be “null”. Checksums are described in more detail below.
The following column metadata properties are defined:
- trevni.codec the name of the compression codec used to compress the blocks of this column, as a string. Implementations are required to support the “null” codec. Optional. If absent, it is assumed to be “null”. Codecs are described in more detail below.
- trevni.name the name of the column, as a string. Required.
- trevni.type the type of data in the column. One of the type names above. Required.
- trevni.values if present, indicates that the initial value of each block in this column will be stored in the block’s descriptor. Not permitted for array columns or columns that specify a parent.
- trevni.array if present, indicates that each row in this column contains a sequence of values of the named type rather than just a single value. An integer length precedes each sequence of values indicating the count of values in the sequence. If the length is negative then it indicates a sequence of zero or one lengths, where -1 indicates two zeros, -2 two ones, -3 three zeros, -4 three ones, etc.
- trevni.parent if present, the name of an array column whose lengths are also used by this column. Thus values of this column are sequences but no lengths are stored in this column.
For example, consider the following row, as JSON, where all values are primitive types, but one has multiple values.
{"id"=566, "date"=23423234234
"from"="foo@bar.com",
"to"=["bar@baz.com", "bang@foo.com"],
"content"="Hi!"}
The columns for this might be specified as:
name=id type=int
name=date type=long
name=from type=string
name=to type=string array=true
name=content type=string
If a row contains an array of records, e.g. “received” in the following:
{"id"=566, "date"=23423234234
"from"="foo@bar.com",
"to"=["bar@baz.com", "bang@foo.com"],
"content"="Hi!"
"received"=[{"date"=234234234234, "host"="192.168.0.0.1"},
{"date"=234234545645, "host"="192.168.0.0.2"}]
}
Then one can define a parent column followed by a column for each field in the record, adding the following columns:
name=received type=null array=true
name=date type=long parent=received
name=host type=string parent=received
If an array value itself contains an array, e.g. the “sigs” below:
{"id"=566, "date"=23423234234
"from"="foo@bar.com",
"to"=["bar@baz.com", "bang@foo.com"],
"content"="Hi!"
"received"=[{"date"=234234234234, "host"="192.168.0.0.1",
"sigs"=[{"algo"="weak", "value"="0af345de"}]},
{"date"=234234545645, "host"="192.168.0.0.2",
"sigs"=[]}]
}
Then a parent column may be defined that itself has a parent column.
name=sigs type=null array=true parent=received
name=algo type=string parent=sigs
name=value type=string parent=sigs
No block metadata properties are currently defined.
A file consists of:
- A file header, followed by
- one or more columns.
A file header consists of:
- Four bytes, ASCII ‘T’, ‘r’, ‘v’, followed by 0x02.
- a fixed64 indicating the number of rows in the file
- a fixed32 indicating the number of columns in the file
- file metadata.
- for each column, its column metadata
- for each column, its starting position in the file as a fixed64.
A column consists of:
- A fixed32 indicating the number of blocks in this column.
- For each block, a block descriptor
- One or more blocks.
A block descriptor consists of:
- A fixed32 indicating the number of rows in the block
- A fixed32 indicating the size in bytes of the block before the codec is applied (excluding checksum).
- A fixed32 indicating the size in bytes of the block after the codec is applied (excluding checksum).
- If this column’s metadata declares it to include values, the first value in the column, serialized according to this column’s type.
A block consists of:
- The serialized column values. If a column is an array column then value sequences are preceded by their length, as an int. If a codec is specified, the values and lengths are compressed by that codec.
- The checksum, as determined by the file metadata.
Codecs
null
The “null” codec simply passes data through uncompressed.
deflate
The “deflate” codec writes the data block using the deflate algorithm as specified in RFC 1951.
snappy
The “snappy” codec uses Google’s Snappy compression library.
Checksum algorithms
null
The “null” checksum contains zero bytes.
crc-32
Each “crc-32” checksum contains the four bytes of an ISO 3309 CRC-32 checksum of the uncompressed block data as a fixed32.
Type Mappings
We define a standard mapping for how types defined in various serialization systems are represented in a column file. Records from these systems are shredded into columns. When records are nested, a depth-first recursive walk can assign a separate column for each primitive value.
Avro
Protocol Buffers
Thrift
Implementation Notes
Some possible techniques for writing column files include:
- Use a standard ~100MB block, buffer in memory up to the block size, then flush the file directly to HDFS. A single reduce task might create multiple output files. The namenode requires memory proportional to the number of names and blocks*replication. This would increase the number of names but not blocks, so this should still be much better than a file per column.
- Spill each column to a separate local, temporary file then, when the file is closed, append these files, writing a single file to HDFS whose block size is set to be that of the entire file. This would be a bit slower than and may have trouble when the local disk is full, but it would better use HDFS namespace and further reduce seeks when processing columns whose values are small.
- Use a separate mapreduce job to convert row-major files to column-major. The map output would output a by (row#, column#, value) tuple, partitioned by row# but sorted by column# then row#. The reducer could directly write the column file. But the column file format would need to be changed to write counts, descriptors, etc. at the end of files rather than at the front.
(1) is the simplest to implement and most implementations should start with it.
References
CIF Column-Oriented Storage Techniques for MapReduce, Floratou, Patel, Shekita, & Tata, VLDB 2011.
DREMEL Dremel: Interactive Analysis of Web-Scale Datasets, Melnik, Gubarev, Long, Romer, Shivakumar, & Tolton, VLDB 2010.
14.5 - MapReduce guide
Avro provides a convenient way to represent complex data structures within a Hadoop MapReduce job. Avro data can be used as both input to and output from a MapReduce job, as well as the intermediate format. The example in this guide uses Avro data for all three, but it’s possible to mix and match; for instance, MapReduce can be used to aggregate a particular field in an Avro record.
This guide assumes basic familiarity with both Hadoop MapReduce and Avro. See the Hadoop documentation and the Avro getting started guide for introductions to these projects. This guide uses the old MapReduce API (org.apache.hadoop.mapred).
Setup
The code from this guide is included in the Avro docs under examples/mr-example. The example is set up as a Maven project that includes the necessary Avro and MapReduce dependencies and the Avro Maven plugin for code generation, so no external jars are needed to run the example. In particular, the POM includes the following dependencies:
<dependency>
<groupId>org.apache.avro</groupId>
<artifactId>avro</artifactId>
<version>1.7.5</version>
</dependency>
<dependency>
<groupId>org.apache.avro</groupId>
<artifactId>avro-mapred</artifactId>
<version>1.7.5</version>
</dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-core</artifactId>
<version>1.1.0</version>
</dependency>
And the following plugin:
<plugin>
<groupId>org.apache.avro</groupId>
<artifactId>avro-maven-plugin</artifactId>
<version>1.7.5</version>
<executions>
<execution>
<phase>generate-sources</phase>
<goals>
<goal>schema</goal>
</goals>
<configuration>
<sourceDirectory>${project.basedir}/../</sourceDirectory>
<outputDirectory>${project.basedir}/src/main/java/</outputDirectory>
</configuration>
</execution>
</executions>
</plugin>
Alternatively, Avro jars can be downloaded directly from the Apache Avro™ Releases page. The relevant Avro jars for this guide are avro-1.7.5.jar and avro-mapred-1.7.5.jar, as well as avro-tools-1.7.5.jar for code generation and viewing Avro data files as JSON. In addition, you will need to install Hadoop in order to use MapReduce.
Example: ColorCount
Below is a simple example of a MapReduce that uses Avro. This example can be found in the Avro docs under examples/mr-example/src/main/java/example/ColorCount.java. We’ll go over the specifics of what’s going on in subsequent sections.
package example;
import java.io.IOException;
import org.apache.avro.*;
import org.apache.avro.Schema.Type;
import org.apache.avro.mapred.*;
import org.apache.hadoop.conf.*;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.mapred.*;
import org.apache.hadoop.util.*;
import example.avro.User;
public class ColorCount extends Configured implements Tool {
public static class ColorCountMapper extends AvroMapper<User, Pair<CharSequence, Integer>> {
@Override
public void map(User user, AvroCollector<Pair<CharSequence, Integer>> collector, Reporter reporter)
throws IOException {
CharSequence color = user.getFavoriteColor();
// We need this check because the User.favorite_color field has type ["string", "null"]
if (color == null) {
color = "none";
}
collector.collect(new Pair<CharSequence, Integer>(color, 1));
}
}
public static class ColorCountReducer extends AvroReducer<CharSequence, Integer,
Pair<CharSequence, Integer>> {
@Override
public void reduce(CharSequence key, Iterable<Integer> values,
AvroCollector<Pair<CharSequence, Integer>> collector,
Reporter reporter)
throws IOException {
int sum = 0;
for (Integer value : values) {
sum += value;
}
collector.collect(new Pair<CharSequence, Integer>(key, sum));
}
}
public int run(String[] args) throws Exception {
if (args.length != 2) {
System.err.println("Usage: ColorCount <input path> <output path>");
return -1;
}
JobConf conf = new JobConf(getConf(), ColorCount.class);
conf.setJobName("colorcount");
FileInputFormat.setInputPaths(conf, new Path(args[0]));
FileOutputFormat.setOutputPath(conf, new Path(args[1]));
AvroJob.setMapperClass(conf, ColorCountMapper.class);
AvroJob.setReducerClass(conf, ColorCountReducer.class);
// Note that AvroJob.setInputSchema and AvroJob.setOutputSchema set
// relevant config options such as input/output format, map output
// classes, and output key class.
AvroJob.setInputSchema(conf, User.SCHEMA$);
AvroJob.setOutputSchema(conf, Pair.getPairSchema(Schema.create(Type.STRING),
Schema.create(Type.INT)));
JobClient.runJob(conf);
return 0;
}
public static void main(String[] args) throws Exception {
int res = ToolRunner.run(new Configuration(), new ColorCount(), args);
System.exit(res);
}
}
ColorCount reads in data files containing User records, defined in examples/user.avsc, and counts the number of instances of each favorite color. (This example draws inspiration from the canonical WordCount MapReduce application.) The User schema is defined as follows:
{"namespace": "example.avro",
"type": "record",
"name": "User",
"fields": [
{"name": "name", "type": "string"},
{"name": "favorite_number", "type": ["int", "null"]},
{"name": "favorite_color", "type": ["string", "null"]}
]
}
This schema is compiled into the User class used by ColorCount via the Avro Maven plugin (see examples/mr-example/pom.xml for how this is set up).
ColorCountMapper essentially takes a User as input and extracts the User’s favorite color, emitting the key-value pair <favoriteColor, 1>. ColorCountReducer then adds up how many occurrences of a particular favorite color were emitted, and outputs the result as a Pair record. These Pairs are serialized to an Avro data file.
Running ColorCount
The ColorCount application is provided as a Maven project in the Avro docs under examples/mr-example. To build the project, including the code generation of the User schema, run:
Next, run GenerateData to create an Avro data file, input/users.avro, containing 20 Users with favorite colors chosen randomly from a list:
mvn exec:java -q -Dexec.mainClass=example.GenerateData
Besides creating the data file, GenerateData prints the JSON representations of the Users generated to stdout, for example:
{"name": "user", "favorite_number": null, "favorite_color": "red"}
{"name": "user", "favorite_number": null, "favorite_color": "green"}
{"name": "user", "favorite_number": null, "favorite_color": "purple"}
{"name": "user", "favorite_number": null, "favorite_color": null}
...
Now we’re ready to run ColorCount. We specify our freshly-generated input folder as the input path and output as our output folder (note that MapReduce will not start a job if the output folder already exists):
mvn exec:java -q -Dexec.mainClass=example.ColorCount -Dexec.args="input output"
Once ColorCount completes, checking the contents of the new output directory should yield the following:
$ ls output/
part-00000.avro _SUCCESS
You can check the contents of the generated Avro file using the avro-tools jar:
$ java -jar /path/to/avro-tools-1.7.5.jar tojson output/part-00000.avro
{"value": 3, "key": "blue"}
{"value": 7, "key": "green"}
{"value": 1, "key": "none"}
{"value": 2, "key": "orange"}
{"value": 3, "key": "purple"}
{"value": 2, "key": "red"}
{"value": 2, "key": "yellow"}
Now let’s go over the ColorCount example in detail.
AvroMapper
The easiest way to use Avro data files as input to a MapReduce job is to subclass AvroMapper. An AvroMapper defines a map function that takes an Avro datum as input and outputs a key/value pair represented as a Pair record. In the ColorCount example, ColorCountMapper is an AvroMapper that takes a User as input and outputs a Pair<CharSequence, Integer>>, where the CharSequence key is the user’s favorite color and the Integer value is 1.
public static class ColorCountMapper extends AvroMapper<User, Pair<CharSequence, Integer>> {
@Override
public void map(User user, AvroCollector<Pair<CharSequence, Integer>> collector, Reporter reporter)
throws IOException {
CharSequence color = user.getFavoriteColor();
// We need this check because the User.favorite_color field has type ["string", "null"]
if (color == null) {
color = "none";
}
collector.collect(new Pair<CharSequence, Integer>(color, 1));
}
}
In order to use our AvroMapper, we must call AvroJob.setMapperClass and AvroJob.setInputSchema.
AvroJob.setMapperClass(conf, ColorCountMapper.class);
AvroJob.setInputSchema(conf, User.SCHEMA$);
Note that AvroMapper does not implement the Mapper interface. Under the hood, the specified Avro data files are deserialized into AvroWrappers containing the actual data, which are processed by a Mapper that calls the configured AvroMapper’s map function. AvroJob.setInputSchema sets up the relevant configuration parameters needed to make this happen, thus you should not need to call JobConf.setMapperClass, JobConf.setInputFormat, JobConf.setMapOutputKeyClass, JobConf.setMapOutputValueClass, or JobConf.setOutputKeyComparatorClass.
AvroReducer
Analogously to AvroMapper, an AvroReducer defines a reducer function that takes the key/value types output by an AvroMapper (or any mapper that outputs Pairs) and outputs a key/value pair represented a Pair record. In the ColorCount example, ColorCountReducer is an AvroReducer that takes the CharSequence key representing a favorite color and the Iterable<Integer> representing the counts for that color (they should all be 1 in this example) and adds up the counts.
public static class ColorCountReducer extends AvroReducer<CharSequence, Integer,
Pair<CharSequence, Integer>> {
@Override
public void reduce(CharSequence key, Iterable<Integer> values,
AvroCollector<Pair<CharSequence, Integer>> collector,
Reporter reporter)
throws IOException {
int sum = 0;
for (Integer value : values) {
sum += value;
}
collector.collect(new Pair<CharSequence, Integer>(key, sum));
}
}
In order to use our AvroReducer, we must call AvroJob.setReducerClass and AvroJob.setOutputSchema.
AvroJob.setReducerClass(conf, ColorCountReducer.class);
AvroJob.setOutputSchema(conf, Pair.getPairSchema(Schema.create(Type.STRING),
Schema.create(Type.INT)));
Note that AvroReducer does not implement the Reducer interface. The intermediate Pairs output by the mapper are split into AvroKeys and AvroValues, which are processed by a Reducer that calls the configured AvroReducer’s reduce function. AvroJob.setOutputSchema sets up the relevant configuration parameters needed to make this happen, thus you should not need to call JobConf.setReducerClass, JobConf.setOutputFormat, JobConf.setOutputKeyClass, JobConf.setMapOutputKeyClass, JobConf.setMapOutputValueClass, or JobConf.setOutputKeyComparatorClass.
Learning more
It’s possible to mix AvroMappers and AvroReducers with non-Avro Mappers and Reducers. See the org.apache.avro.mapred documentation for more details. There is also a org.apache.avro.mapreduce package for use with the new MapReduce API (org.apache.hadoop.mapreduce). It’s also possible to implement your own Mappers and Reducers directly using the public classes provided in these libraries. See the AvroWordCount application, found under examples/mr-example/src/main/java/example/AvroWordCount.java in the Avro documentation, for an example of implementing a Reducer that outputs Avro data.
14.6 - IDL Language
Introduction
This document defines Avro IDL, a higher-level language for authoring Avro schemata. Before reading this document, you should have familiarity with the concepts of schemata and protocols, as well as the various primitive and complex types available in Avro.
Overview
Purpose
The aim of the Avro IDL language is to enable developers to author schemata in a way that feels more similar to common programming languages like Java, C++, or Python. Additionally, the Avro IDL language may feel more familiar for those users who have previously used the interface description languages (IDLs) in other frameworks like Thrift, Protocol Buffers, or CORBA.
Usage
Each Avro IDL file defines a single Avro Protocol, and thus generates as its output a JSON-format Avro Protocol file with extension .avpr.
To convert a .avdl file into a .avpr file, it may be processed by the idl tool. For example:
$ java -jar avroj-tools.jar idl src/test/idl/input/namespaces.avdl /tmp/namespaces.avpr
$ head /tmp/namespaces.avpr
{
"protocol" : "TestNamespace",
"namespace" : "avro.test.protocol",
The idl tool can also process input to and from stdin and stdout. See idl --help for full usage information.
A Maven plugin is also provided to compile .avdl files. To use it, add something like the following to your pom.xml:
<build>
<plugins>
<plugin>
<groupId>org.apache.avro</groupId>
<artifactId>avro-maven-plugin</artifactId>
<executions>
<execution>
<goals>
<goal>idl-protocol</goal>
</goals>
</execution>
</executions>
</plugin>
</plugins>
</build>
Defining a Protocol in Avro IDL
An Avro IDL file consists of exactly one protocol definition. The minimal protocol is defined by the following code:
This is equivalent to (and generates) the following JSON protocol definition:
{
"protocol" : "MyProtocol",
"types" : [ ],
"messages" : {
}
}
The namespace of the protocol may be changed using the @namespace annotation:
@namespace("mynamespace")
protocol MyProtocol {
}
This notation is used throughout Avro IDL as a way of specifying properties for the annotated element, as will be described later in this document.
Protocols in Avro IDL can contain the following items:
- Imports of external protocol and schema files.
- Definitions of named schemata, including records, errors, enums, and fixeds.
- Definitions of RPC messages
Imports
Files may be imported in one of three formats:
-
An IDL file may be imported with a statement like:
import idl "foo.avdl";
-
A JSON protocol file may be imported with a statement like:
import protocol "foo.avpr";
-
A JSON schema file may be imported with a statement like:
import schema "foo.avsc";
Messages and types in the imported file are added to this file’s protocol.
Imported file names are resolved relative to the current IDL file.
Defining an Enumeration
Enums are defined in Avro IDL using a syntax similar to C or Java:
enum Suit {
SPADES, DIAMONDS, CLUBS, HEARTS
}
Note that, unlike the JSON format, anonymous enums cannot be defined.
Defining a Fixed Length Field
Fixed fields are defined using the following syntax:
fixed MD5(16);
This example defines a fixed-length type called MD5 which contains 16 bytes.
Defining Records and Errors
Records are defined in Avro IDL using a syntax similar to a struct definition in C:
record Employee {
string name;
boolean active = true;
long salary;
}
The above example defines a record with the name “Employee” with three fields.
To define an error, simply use the keyword error instead of record. For example:
error Kaboom {
string explanation;
int result_code = -1;
}
Each field in a record or error consists of a type and a name, optional property annotations and an optional default value.
A type reference in Avro IDL must be one of:
- A primitive type
- A logical type
- A named schema defined prior to this usage in the same Protocol
- A complex type (array, map, or union)
Primitive Types
The primitive types supported by Avro IDL are the same as those supported by Avro’s JSON format. This list includes int, long, string, boolean, float, double, null, and bytes.
References to Named Schemata
If a named schema has already been defined in the same Avro IDL file, it may be referenced by name as if it were a primitive type:
record Card {
Suit suit; // refers to the enum Card defined above
int number;
}
Default Values
Default values for fields may be optionally specified by using an equals sign after the field name followed by a JSON expression indicating the default value. This JSON is interpreted as described in the spec.
Complex Types
Arrays
Array types are written in a manner that will seem familiar to C++ or Java programmers. An array of any type t is denoted array<t>. For example, an array of strings is denoted array<string>, and a multidimensional array of Foo records would be array<array<Foo>>.
Maps
Map types are written similarly to array types. An array that contains values of type t is written map<t>. As in the JSON schema format, all maps contain string-type keys.
Unions
Union types are denoted as union { typeA, typeB, typeC, … }. For example, this record contains a string field that is optional (unioned with null):
record RecordWithUnion {
union { null, string } optionalString;
}
Note that the same restrictions apply to Avro IDL unions as apply to unions defined in the JSON format; namely, a record may not contain multiple elements of the same type.
Defining RPC Messages
The syntax to define an RPC message within a Avro IDL protocol is similar to the syntax for a method declaration within a C header file or a Java interface. To define an RPC message add which takes two arguments named foo and bar, returning an int, simply include the following definition within the protocol:
int add(int foo, int bar = 0);
Message arguments, like record fields, may specify default values.
To define a message with no response, you may use the alias void, equivalent to the Avro null type:
void logMessage(string message);
If you have previously defined an error type within the same protocol, you may declare that a message can throw this error using the syntax:
void goKaboom() throws Kaboom;
To define a one-way message, use the keyword oneway after the parameter list, for example:
void fireAndForget(string message) oneway;
Other Language Features
All Java-style comments are supported within a Avro IDL file. Any text following // on a line is ignored, as is any text between /* and */, possibly spanning multiple lines.
Comments that begin with /** are used as the documentation string for the type or field definition that follows the comment.
Escaping Identifiers
Occasionally, one will need to use a reserved language keyword as an identifier. In order to do so, backticks (`) may be used to escape the identifier. For example, to define a message with the literal name error, you may write:
This syntax is allowed anywhere an identifier is expected.
Annotations for Ordering and Namespaces
Java-style annotations may be used to add additional properties to types and fields throughout Avro IDL.
For example, to specify the sort order of a field within a record, one may use the @order annotation before the field name as follows:
record MyRecord {
string @order("ascending") myAscendingSortField;
string @order("descending") myDescendingField;
string @order("ignore") myIgnoredField;
}
A field’s type may also be preceded by annotations, e.g.:
record MyRecord {
@java-class("java.util.ArrayList") array<string> myStrings;
}
This can be used to support java classes that can be serialized/deserialized via their toString/String constructor, e.g.:
record MyRecord {
@java-class("java.math.BigDecimal") string value;
@java-key-class("java.io.File") map<string> fileStates;
array<@java-class("java.math.BigDecimal") string> weights;
}
Similarly, a @namespace annotation may be used to modify the namespace when defining a named schema. For example:
@namespace("org.apache.avro.firstNamespace")
protocol MyProto {
@namespace("org.apache.avro.someOtherNamespace")
record Foo {}
record Bar {}
}
will define a protocol in the firstNamespace namespace. The record Foo will be defined in someOtherNamespace and Bar will be defined in firstNamespace as it inherits its default from its container.
Type and field aliases are specified with the @aliases annotation as follows:
@aliases(["org.old.OldRecord", "org.ancient.AncientRecord"])
record MyRecord {
string @aliases(["oldField", "ancientField"]) myNewField;
}
Some annotations like those listed above are handled specially. All other annotations are added as properties to the protocol, message, schema or field.
Complete Example
The following is a complete example of a Avro IDL file that shows most of the above features:
/**
* An example protocol in Avro IDL
*/
@namespace("org.apache.avro.test")
protocol Simple {
@aliases(["org.foo.KindOf"])
enum Kind {
FOO,
BAR, // the bar enum value
BAZ
}
fixed MD5(16);
record TestRecord {
@order("ignore")
string name;
@order("descending")
Kind kind;
MD5 hash;
union { MD5, null} @aliases(["hash"]) nullableHash;
array<long> arrayOfLongs;
}
error TestError {
string message;
}
string hello(string greeting);
TestRecord echo(TestRecord `record`);
int add(int arg1, int arg2);
bytes echoBytes(bytes data);
void `error`() throws TestError;
void ping() oneway;
}
Additional examples may be found in the Avro source tree under the src/test/idl/input directory.
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
14.7 - SASL profile
Introduction
SASL (RFC 2222) provides a framework for authentication and security of network protocols. Each protocol that uses SASL is meant to define a SASL profile. This document provides a SASL profile for connection-based Avro RPC.
Overview
SASL negotiation proceeds as a series of message interactions over a connection between a client and server using a selected SASL mechanism. The client starts this negotiation by sending its chosen mechanism name with an initial (possibly empty) message. Negotiation proceeds with the exchange of messages until either side indicates success or failure. The content of the messages is mechanism-specific. If the negotiation succeeds, then the session can proceed over the connection, otherwise it must be abandoned.
Some mechanisms continue to process session data after negotiation (e.g., encrypting it), while some specify that further session data is transmitted unmodified.
Negotiation
Commands
Avro SASL negotiation uses four one-byte commands.
- 0: START Used in a client’s initial message.
- 1: CONTINUE Used while negotiation is ongoing.
- 2: FAIL Terminates negotiation unsuccessfully.
- 3: COMPLETE Terminates negotiation successfully.
The format of a START message is:
| 0 | 4-byte mechanism name length | mechanism name | 4-byte payload length | payload data |
The format of a CONTINUE message is:
| 1 | 4-byte payload length | payload data |
The format of a FAIL message is:
| 2 | 4-byte message length | UTF-8 message |
The format of a COMPLETE message is:
| 3 | 4-byte payload length | payload data |
Process
Negotiation is initiated by a client sending a START command containing the client’s chosen mechanism name and any mechanism-specific payload data.
The server and client then interchange some number (possibly zero) of CONTINUE messages. Each message contains payload data that is processed by the security mechanism to generate the next message.
Once either the client or server send a FAIL message then negotiation has failed. UTF-8-encoded text is included in the failure message. Once either a FAIL message has been sent or received, or any other error occurs in the negotiation, further communication on this connection must cease.
Once either the client or server send a COMPLETE message then negotiation has completed successfully. Session data may now be transmitted over the connection until it is closed by either side.
Session Data
If no SASL QOP (quality of protection) is negotiated, then all subsequent writes to/reads over this connection are written/read unmodified. In particular, messages use Avro framing, and are of the form:
| 4-byte frame length | frame data | ... | 4 zero bytes |
If a SASL QOP is negotiated, then it must be used by the connection for all subsequent messages. This is done by wrapping each non-empty frame written using the security mechanism and unwrapping each non-empty frame read. The length written in each non-empty frame is the length of the wrapped data. Complete frames must be passed to the security mechanism for unwrapping. Unwrapped data is then passed to the application as the content of the frame.
If at any point processing fails due to wrapping, unwrapping or framing errors, then all further communication on this connection must cease.
Anonymous Mechanism
The SASL anonymous mechanism (RFC 2245) is quite simple to implement. In particular, an initial anonymous request may be prefixed by the following static sequence:
| 0 | 0009 | ANONYMOUS | 0000 |
If a server uses the anonymous mechanism, it should check that the mechanism name in the start message prefixing the first request received is ‘ANONYMOUS’, then simply prefix its initial response with a COMPLETE message of:
| 3 | 0000 |
If an anonymous server recieves some other mechanism name, then it may respond with a FAIL message as simple as:
| 2 | 0000 |
Note that the anonymous mechanism need add no additional round-trip messages between client and server. The START message can be piggybacked on the initial request and the COMPLETE or FAIL message can be piggybacked on the initial response.
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
14.8 - Wiki
The Wiki page can be found here.
14.9 - FAQ
What is Avro?
Avro is a data serialization system.
Avro provides:
- Rich data structures.
- A compact, fast, binary data format.
- A container file, to store persistent data.
- Remote procedure call (RPC).
- Simple integration with dynamic languages. Code generation is not required to read or write data files nor to use or implement RPC protocols. Code generation as an optional optimization, only worth implementing for statically typed languages.
How can I get started quickly with Avro?
Check out the Quick Start Guide.
How do I statically compile a schema or protocol into generated code?
In Java
- Add the avro jar, the jackson-mapper-asl.jar and jackson-core-asl.jar to your CLASSPATH.
- Run java org.apache.avro.specific.SpecificCompiler .
This appears to be out of date, the SpecificCompiler requires two arguments, presumably an input and and output file, but it isn’t clear that this does.
Or use the Schema or Protocol Ant tasks. Avro’s build.xml provides examples of how these are used.
Lastly, you can also use the “avro-tools” jar which ships with an Avro release. Just use the “compile (schema|protocol)” command.
How are Strings represented in Java?
They use org.apache.avro.util.Utf8, not java.lang.String.
More generally, how do Avro types map to Java types?
The mappings are documented in the package javadoc for generic, specific and reflect API.
From Doug Cutting:
HDFS splits files into blocks, and mapreduce runs a map task for each block. When the task starts, it needs to be able to seek into the file to the start of the block process through the block’s end. If the file were, e.g., a gzip file, this would not be possible, since gzip files must be decompressed from the start. One cannot seek into the middle of a gzip file and start decompressing. So Hadoop’s SequenceFile places a marker periodically (~64k) in the file at record and compression boundaries, where processing can be sensibly started. Then, when a map task starts processing an HDFS block, it finds the first marker after the block’s start and continues through the first marker in the next block of the file. This requires a bit of non-local access (~0.1%). Avro’s data file uses the same method as SequenceFile.
Why isn’t every value in Avro nullable?
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 of every value stored and
corresponding computational costs to create this bit on write and
interpret it on read. These costs are wasted when values may not in
fact be null, as is the case in many datasets. In Avro such costs are
only paid when values may actually be null.
Also, allowing values to be null is a well-known source of errors. In
Avro, a value declared as non-null will always be non-null and programs
need not test for null values when processing it nor will they ever fail
for lack of such tests.
Tony Hoare calls his invention of null references his “Billion Dollar
Mistake”.
http://qconlondon.com/london-2009/presentation/Null+References:+The+Billion+Dollar+Mistake
Also note that in some programming languages not all values are permitted to be null. For example, in Java, values of type boolean, byte, short, char, int, float, long, and double may not be null.
How can I serialize directly to/from a byte array?
As pointed out in the specification, Avro data should always be stored with its schema. The Avro provided classes DataFileWriter, DataFileReader, and DataFileStream all ensure this by serializing the Schema in a container header. In some special cases, such as when implementing a new storage system or writing unit tests, you may need to write and read directly with the bare Avro serialized values. The following examples use code generated for Java from an Avro Schema, specifically the User example from the Quickstart guide.
Serializing to a byte array
This example takes a User object and returns a newly allocated byte array with the Avro serialization of that user.
ByteArrayOutputStream out = new ByteArrayOutputStream();
BinaryEncoder encoder = EncoderFactory.get().binaryEncoder(out, null);
DatumWriter<User> writer = new SpecificDatumWriter<User>(User.getClassSchema());
writer.write(user, encoder);
encoder.flush();
out.close();
byte[] serializedBytes = out.toByteArray();
Deserializing from a byte array
This example takes a byte array containing the Avro serialization of a user and returns a User object.SpecificDatumReader reader = new SpecificDatumReader(User.getClassSchema());
Decoder decoder = DecoderFactory.get().binaryDecoder(bytes, null);
User user = reader.read(null, decoder);
Note: If you are serializing or deserializing in a loop or as a method, you should be reusing objects, readers and/or writers. Check the JavaDoc to see how to reuse the objects.
15 - Apache Avro™ 1.7.4 Documentation
Introduction
Apache Avro™ is a data serialization system.
Avro provides:
- Rich data structures.
- A compact, fast, binary data format.
- A container file, to store persistent data.
- Remote procedure call (RPC).
- Simple integration with dynamic languages. Code generation is not required to read or write data files nor to use or implement RPC protocols. Code generation as an optional optimization, only worth implementing for statically typed languages.
Schemas
Avro relies on schemas. When Avro data is read, the schema used when writing it is always present. This permits each datum to be written with no per-value overheads, making serialization both fast and small. This also facilitates use with dynamic, scripting languages, since data, together with its schema, is fully self-describing.
When Avro data is stored in a file, its schema is stored with it, so that files may be processed later by any program. If the program reading the data expects a different schema this can be easily resolved, since both schemas are present.
When Avro is used in RPC, the client and server exchange schemas in the connection handshake. (This can be optimized so that, for most calls, no schemas are actually transmitted.) Since both client and server both have the other’s full schema, correspondence between same named fields, missing fields, extra fields, etc. can all be easily resolved.
Avro schemas are defined with JSON . This facilitates implementation in languages that already have JSON libraries.
Comparison with other systems
Avro provides functionality similar to systems such as Thrift, Protocol Buffers, etc. Avro differs from these systems in the following fundamental aspects.
- Dynamic typing: Avro does not require that code be generated. Data is always accompanied by a schema that permits full processing of that data without code generation, static datatypes, etc. This facilitates construction of generic data-processing systems and languages.
- Untagged data: Since the schema is present when data is read, considerably less type information need be encoded with data, resulting in smaller serialization size.
- No manually-assigned field IDs: When a schema changes, both the old and new schema are always present when processing data, so differences may be resolved symbolically, using field names.
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
15.1 - Getting Started (Java)
This is a short guide for getting started with Apache Avro™ using Java. This guide only covers using Avro for data serialization; see Patrick Hunt’s Avro RPC Quick Start for a good introduction to using Avro for RPC.
Download
Avro implementations for C, C++, C#, Java, PHP, Python, and Ruby can be downloaded from the Apache Avro™ Releases page. This guide uses Avro 1.7.4, the latest version at the time of writing. For the examples in this guide, download avro-1.7.4.jar and avro-tools-1.7.4.jar. The Avro Java implementation also depends on the Jackson JSON library. From the Jackson download page, download the core-asl and mapper-asl jars. Add avro-1.7.4.jar and the Jackson jars to your project’s classpath (avro-tools will be used for code generation).
Alternatively, if you are using Maven, add the following dependency to your POM:
<dependency>
<groupId>org.apache.avro</groupId>
<artifactId>avro</artifactId>
<version>1.7.4</version>
</dependency>
As well as the Avro Maven plugin (for performing code generation):
<plugin>
<groupId>org.apache.avro</groupId>
<artifactId>avro-maven-plugin</artifactId>
<version>1.7.4</version>
<executions>
<execution>
<phase>generate-sources</phase>
<goals>
<goal>schema</goal>
</goals>
<configuration>
<sourceDirectory>${project.basedir}/src/main/avro/</sourceDirectory>
<outputDirectory>${project.basedir}/src/main/java/</outputDirectory>
</configuration>
</execution>
</executions>
</plugin>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-compiler-plugin</artifactId>
<configuration>
<source>1.6</source>
<target>1.6</target>
</configuration>
</plugin>
You may also build the required Avro jars from source. Building Avro is beyond the scope of this guide; see the Build Documentation page in the wiki for more information.
Defining a schema
Avro schemas are defined using JSON. Schemas are composed of primitive types (null, boolean, int, long, float, double, bytes, and string) and complex types (record, enum, array, map, union, and fixed). You can learn more about Avro schemas and types from the specification, but for now let’s start with a simple schema example, user.avsc:
{"namespace": "example.avro",
"type": "record",
"name": "User",
"fields": [
{"name": "name", "type": "string"},
{"name": "favorite_number", "type": ["int", "null"]},
{"name": "favorite_color", "type": ["string", "null"]}
]
}
This schema defines a record representing a hypothetical user. (Note that a schema file can only contain a single schema definition.) At minimum, a record definition must include its type (“type”: “record”), a name (“name”: “User”), and fields, in this case name, favorite_number, and favorite_color. We also define a namespace (“namespace”: “example.avro”), which together with the name attribute defines the “full name” of the schema (example.avro.User in this case).
Fields are defined via an array of objects, each of which defines a name and type (other attributes are optional, see the record specification for more details). The type attribute of a field is another schema object, which can be either a primitive or complex type. For example, the name field of our User schema is the primitive type string, whereas the favorite_number and favorite_color fields are both unions, represented by JSON arrays. unions are a complex type that can be any of the types listed in the array; e.g., favorite_number can either be an int or null, essentially making it an optional field.
Serializing and deserializing with code generation
Compiling the schema
Code generation allows us to automatically create classes based on our previously-defined schema. Once we have defined the relevant classes, there is no need to use the schema directly in our programs. We use the avro-tools jar to generate code as follows:
java -jar /path/to/avro-tools-1.7.4.jar compile schema <schema file> <destination>
This will generate the appropriate source files in a package based on the schema’s namespace in the provided destination folder. For instance, to generate a User class in package example.avro from the schema defined above, run
java -jar /path/to/avro-tools-1.7.4.jar compile schema user.avsc .
Note that if you using the Avro Maven plugin, there is no need to manually invoke the schema compiler; the plugin automatically performs code generation on any .avsc files present in the configured source directory.
Creating Users
Now that we’ve completed the code generation, let’s create some Users, serialize them to a data file on disk, and then read back the file and deserialize the User objects.
First let’s create some Users and set their fields.
User user1 = new User();
user1.setName("Alyssa");
user1.setFavoriteNumber(256);
// Leave favorite color null
// Alternate constructor
User user2 = new User("Ben", 7, "red");
// Construct via builder
User user3 = User.newBuilder()
.setName("Charlie")
.setFavoriteColor("blue")
.setFavoriteNumber(null)
.build();
As shown in this example, Avro objects can be created either by invoking a constructor directly or by using a builder. Unlike constructors, builders will automatically set any default values specified in the schema. Additionally, builders validate the data as it set, whereas objects constructed directly will not cause an error until the object is serialized. However, using constructors directly generally offers better performance, as builders create a copy of the datastructure before it is written.
Note that we do not set user1’s favorite color. Since that record is of type [“string”, “null”], we can either set it to a string or leave it null; it is essentially optional. Similarly, we set user3’s favorite number to null (using a builder requires setting all fields, even if they are null).
Serializing
Now let’s serialize our Users to disk.
// Serialize user1 and user2 to disk
File file = new File("users.avro");
DatumWriter<User> userDatumWriter = new SpecificDatumWriter<User>(User.class);
DataFileWriter<User> dataFileWriter = new DataFileWriter<User>(userDatumWriter);
dataFileWriter.create(user1.getSchema(), new File("users.avro"));
dataFileWriter.append(user1);
dataFileWriter.append(user2);
dataFileWriter.append(user3);
dataFileWriter.close();
We create a DatumWriter, which converts Java objects into an in-memory serialized format. The SpecificDatumWriter class is used with generated classes and extracts the schema from the specified generated type.
Next we create a DataFileWriter, which writes the serialized records, as well as the schema, to the file specified in the dataFileWriter.create call. We write our users to the file via calls to the dataFileWriter.append method. When we are done writing, we close the data file.
Deserializing
Finally, let’s deserialize the data file we just created.
// Deserialize Users from disk
DatumReader<User> userDatumReader = new SpecificDatumReader<User>(User.class);
DataFileReader<User> dataFileReader = new DataFileReader<User>(file, userDatumReader);
User user = null;
while (dataFileReader.hasNext()) {
// Reuse user object by passing it to next(). This saves us from
// allocating and garbage collecting many objects for files with
// many items.
user = dataFileReader.next(user);
System.out.println(user);
}
This snippet will output:
{"name": "Alyssa", "favorite_number": 256, "favorite_color": null}
{"name": "Ben", "favorite_number": 7, "favorite_color": "red"}
{"name": "Charlie", "favorite_number": null, "favorite_color": "blue"}
Deserializing is very similar to serializing. We create a SpecificDatumReader, analogous to the SpecificDatumWriter we used in serialization, which converts in-memory serialized items into instances of our generated class, in this case User. We pass the DatumReader and the previously created File to a DataFileReader, analogous to the DataFileWriter, which reads the data file on disk.
Next we use the DataFileReader to iterate through the serialized Users and print the deserialized object to stdout. Note how we perform the iteration: we create a single User object which we store the current deserialized user in, and pass this record object to every call of dataFileReader.next. This is a performance optimization that allows the DataFileReader to reuse the same User object rather than allocating a new User for every iteration, which can be very expensive in terms of object allocation and garbage collection if we deserialize a large data file. While this technique is the standard way to iterate through a data file, it’s also possible to use for (User user : dataFileReader) if performance is not a concern.
Compiling and running the example code
This example code is included as a Maven project in the examples/java-example directory in the Avro docs. From this directory, execute the following commands to build and run the example:
$ mvn compile # includes code generation via Avro Maven plugin
$ mvn -q exec:java -Dexec.mainClass=example.SpecificMain
Serializing and deserializing without code generation
Data in Avro is always stored with its corresponding schema, meaning we can always read a serialized item regardless of whether we know the schema ahead of time. This allows us to perform serialization and deserialization without code generation.
Let’s go over the same example as in the previous section, but without using code generation: we’ll create some users, serialize them to a data file on disk, and then read back the file and deserialize the users objects.
Creating users
First, we use a Parser to read our schema definition and create a Schema object.
Schema schema = new Parser().parse(new File("user.avsc"));
Using this schema, let’s create some users.
GenericRecord user1 = new GenericData.Record(schema);
user1.put("name", "Alyssa");
user1.put("favorite_number", 256);
// Leave favorite color null
GenericRecord user2 = new GenericData.Record(schema);
user2.put("name", "Ben");
user2.put("favorite_number", 7);
user2.put("favorite_color", "red");
Since we’re not using code generation, we use GenericRecords to represent users. GenericRecord uses the schema to verify that we only specify valid fields. If we try to set a non-existent field (e.g., user1.put(“favorite_animal”, “cat”)), we’ll get an AvroRuntimeException when we run the program.
Note that we do not set user1’s favorite color. Since that record is of type [“string”, “null”], we can either set it to a string or leave it null; it is essentially optional.
Serializing
Now that we’ve created our user objects, serializing and deserializing them is almost identical to the example above which uses code generation. The main difference is that we use generic instead of specific readers and writers.
First we’ll serialize our users to a data file on disk.
// Serialize user1 and user2 to disk
File file = new File("users.avro");
DatumWriter<GenericRecord> datumWriter = new GenericDatumWriter<GenericRecord>(schema);
DataFileWriter<GenericRecord> dataFileWriter = new DataFileWriter<GenericRecord>(datumWriter);
dataFileWriter.create(schema, file);
dataFileWriter.append(user1);
dataFileWriter.append(user2);
dataFileWriter.close();
We create a DatumWriter, which converts Java objects into an in-memory serialized format. Since we are not using code generation, we create a GenericDatumWriter. It requires the schema both to determine how to write the GenericRecords and to verify that all non-nullable fields are present.
As in the code generation example, we also create a DataFileWriter, which writes the serialized records, as well as the schema, to the file specified in the dataFileWriter.create call. We write our users to the file via calls to the dataFileWriter.append method. When we are done writing, we close the data file.
Deserializing
Finally, we’ll deserialize the data file we just created.
// Deserialize users from disk
DatumReader<GenericRecord> datumReader = new GenericDatumReader<GenericRecord>(schema);
DataFileReader<GenericRecord> dataFileReader = new DataFileReader<GenericRecord>(file, datumReader);
GenericRecord user = null;
while (dataFileReader.hasNext()) {
// Reuse user object by passing it to next(). This saves us from
// allocating and garbage collecting many objects for files with
// many items.
user = dataFileReader.next(user);
System.out.println(user);
This outputs:
{"name": "Alyssa", "favorite_number": 256, "favorite_color": null}
{"name": "Ben", "favorite_number": 7, "favorite_color": "red"}
Deserializing is very similar to serializing. We create a GenericDatumReader, analogous to the GenericDatumWriter we used in serialization, which converts in-memory serialized items into GenericRecords. We pass the DatumReader and the previously created File to a DataFileReader, analogous to the DataFileWriter, which reads the data file on disk.
Next, we use the DataFileReader to iterate through the serialized users and print the deserialized object to stdout. Note how we perform the iteration: we create a single GenericRecord object which we store the current deserialized user in, and pass this record object to every call of dataFileReader.next. This is a performance optimization that allows the DataFileReader to reuse the same record object rather than allocating a new GenericRecord for every iteration, which can be very expensive in terms of object allocation and garbage collection if we deserialize a large data file. While this technique is the standard way to iterate through a data file, it’s also possible to use for (GenericRecord user : dataFileReader) if performance is not a concern.
Compiling and running the example code
This example code is included as a Maven project in the examples/java-example directory in the Avro docs. From this directory, execute the following commands to build and run the example:
$ mvn compile
$ mvn -q exec:java -Dexec.mainClass=example.GenericMain
15.2 - Getting Started (Python)
This is a short guide for getting started with Apache Avro™ using Python. This guide only covers using Avro for data serialization; see Patrick Hunt’s Avro RPC Quick Start for a good introduction to using Avro for RPC.
Download
Avro implementations for C, C++, C#, Java, PHP, Python, and Ruby can be downloaded from the Apache Avro™ Releases page. This guide uses Avro 1.7.4, the latest version at the time of writing. Download and unzip avro-1.7.4.tar.gz, and install via python setup.py (this will probably require root privileges). Ensure that you can import avro from a Python prompt.
$ tar xvf avro-1.7.4.tar.gz
$ cd avro-1.7.4
$ sudo python setup.py install
$ python
>>> import avro # should not raise ImportError
Alternatively, you may build the Avro Python library from source. From your the root Avro directory, run the commands
$ cd lang/py/
$ ant
$ sudo python setup.py install
$ python
>>> import avro # should not raise ImportError
Defining a schema
Avro schemas are defined using JSON. Schemas are composed of primitive types (null, boolean, int, long, float, double, bytes, and string) and complex types (record, enum, array, map, union, and fixed). You can learn more about Avro schemas and types from the specification, but for now let’s start with a simple schema example, user.avsc:
{"namespace": "example.avro",
"type": "record",
"name": "User",
"fields": [
{"name": "name", "type": "string"},
{"name": "favorite_number", "type": ["int", "null"]},
{"name": "favorite_color", "type": ["string", "null"]}
]
}
This schema defines a record representing a hypothetical user. (Note that a schema file can only contain a single schema definition.) At minimum, a record definition must include its type (“type”: “record”), a name (“name”: “User”), and fields, in this case name, favorite_number, and favorite_color. We also define a namespace (“namespace”: “example.avro”), which together with the name attribute defines the “full name” of the schema (example.avro.User in this case).
Fields are defined via an array of objects, each of which defines a name and type (other attributes are optional, see the record specification for more details). The type attribute of a field is another schema object, which can be either a primitive or complex type. For example, the name field of our User schema is the primitive type string, whereas the favorite_number and favorite_color fields are both unions, represented by JSON arrays. unions are a complex type that can be any of the types listed in the array; e.g., favorite_number can either be an int or null, essentially making it an optional field.
Serializing and deserializing without code generation
Data in Avro is always stored with its corresponding schema, meaning we can always read a serialized item, regardless of whether we know the schema ahead of time. This allows us to perform serialization and deserialization without code generation. Note that the Avro Python library does not support code generation.
Try running the following code snippet, which serializes two users to a data file on disk, and then reads back and deserializes the data file:
import avro.schema
from avro.datafile import DataFileReader, DataFileWriter
from avro.io import DatumReader, DatumWriter
schema = avro.schema.parse(open("user.avsc").read())
writer = DataFileWriter(open("users.avro", "w"), DatumWriter(), schema)
writer.append({"name": "Alyssa", "favorite_number": 256})
writer.append({"name": "Ben", "favorite_number": 7, "favorite_color": "red"})
writer.close()
reader = DataFileReader(open("users.avro", "r"), DatumReader())
for user in reader:
print user
reader.close()
This outputs:
{u'favorite_color': None, u'favorite_number': 256, u'name': u'Alyssa'}
{u'favorite_color': u'red', u'favorite_number': 7, u'name': u'Ben'}
Let’s take a closer look at what’s going on here.
schema = avro.schema.parse(open("user.avsc").read())
avro.schema.parse takes a string containing a JSON schema definition as input and outputs a avro.schema.Schema object (specifically a subclass of Schema, in this case RecordSchema). We’re passing in the contents of our user.avsc schema file here.
writer = DataFileWriter(open("users.avro", "w"), DatumWriter(), schema)
We create a DataFileWriter, which we’ll use to write serialized items to a data file on disk. The DataFileWriter constructor takes three arguments:
- The file we’ll serialize to
- A DatumWriter, which is responsible for actually serializing the items to Avro’s binary format (DatumWriters can be used separately from DataFileWriters, e.g., to perform IPC with Avro TODO: is this true??).
- The schema we’re using. The DataFileWriter needs the schema both to write the schema to the data file, and to verify that the items we write are valid items and write the appropriate fields.
writer.append({"name": "Alyssa", "favorite_number": 256})
writer.append({"name": "Ben", "favorite_number": 7, "favorite_color": "red"})
We use DataFileWriter.append to add items to our data file. Avro records are represented as Python dicts. Since the field favorite_color has type [“int”, “null”], we are not required to specify this field, as shown in the first append. Were we to omit the required name field, an exception would be raised. Any extra entries not corresponding to a field are present in the dict are ignored.
reader = DataFileReader(open("users.avro", "r"), DatumReader())
We open the file again, this time for reading back from disk. We use a DataFileReader and DatumReader analagous to the DataFileWriter and DatumWriter above.
for user in reader:
print user
The DataFileReader is an iterator that returns dicts corresponding to the serialized items.
15.3 - Specification
Introduction
This document defines Apache Avro. It is intended to be the authoritative specification. Implementations of Avro must adhere to this document.
Schema Declaration
A Schema is represented in JSON by one of:
- A JSON string, naming a defined type.
- A JSON object, of the form:
{"type": "typeName" ...attributes...}
where typeName is either a primitive or derived type name, as defined below. Attributes not defined in this document are permitted as metadata, but must not affect the format of serialized data.
- A JSON array, representing a union of embedded types.
Primitive Types
The set of primitive type names is:
- null: no value
- boolean: a binary value
- int: 32-bit signed integer
- long: 64-bit signed integer
- float: single precision (32-bit) IEEE 754 floating-point number
- double: double precision (64-bit) IEEE 754 floating-point number
- bytes: sequence of 8-bit unsigned bytes
- string: unicode character sequence
Primitive types have no specified attributes.
Primitive type names are also defined type names. Thus, for example, the schema “string” is equivalent to:
Complex Types
Avro supports six kinds of complex types: records, enums, arrays, maps, unions and fixed.
Records
Records use the type name “record” and support three attributes:
- name: a JSON string providing the name of the record (required).
- namespace, a JSON string that qualifies the name;
- doc: a JSON string providing documentation to the user of this schema (optional).
- aliases: a JSON array of strings, providing alternate names for this record (optional).
- fields: a JSON array, listing fields (required). Each field is a JSON object with the following attributes:
- name: a JSON string providing the name of the field (required), and
- doc: a JSON string describing this field for users (optional).
- type: A JSON object defining a schema, or a JSON string naming a record definition (required).
- _default: A default value for this field, used when reading instances that lack this field (optional). 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. Default values for bytes and fixed fields are JSON strings, where Unicode code points 0-255 are mapped to unsigned 8-bit byte values 0-255.
field default values
| avro type |
json type |
example |
| null |
null |
null |
| boolean |
boolean |
true |
| int,long |
integer |
1 |
| float,double |
number |
1.1 |
| bytes |
string |
"\u00FF" |
| string |
string |
"foo" |
| record |
object |
{"a": 1} |
| enum |
string |
"FOO" |
| array |
array |
[1] |
| map |
object |
{"a": 1} |
| fixed |
string |
"\u00ff" |
- order: specifies how this field impacts sort ordering of this record (optional). Valid values are “ascending” (the default), “descending”, or “ignore”. For more details on how this is used, see the sort order section below.
- aliases: a JSON array of strings, providing alternate names for this field (optional).
For example, a linked-list of 64-bit values may be defined with:
{
"type": "record",
"name": "LongList",
"aliases": ["LinkedLongs"], // old name for this
"fields" : [
{"name": "value", "type": "long"}, // each element has a long
{"name": "next", "type": ["LongList", "null"]} // optional next element
]
}
Enums
Enums use the type name “enum” and support the following attributes:
- name: a JSON string providing the name of the enum (required).
- namespace, a JSON string that qualifies the name ;
- aliases: a JSON array of strings, providing alternate names for this enum (optional).
- doc: a JSON string providing documentation to the user of this schema (optional).
- symbols: a JSON array, listing symbols, as JSON strings (required). All symbols in an enum must be unique; duplicates are prohibited.
For example, playing card suits might be defined with:
{
"type": "enum",
"name": "Suit",
"symbols" : ["SPADES", "HEARTS", "DIAMONDS", "CLUBS"]
}
Arrays
Arrays use the type name “array” and support a single attribute:
- items: the schema of the array’s items.
For example, an array of strings is declared with:
{
"type": "array",
"items" : "string"
}
Maps
Maps use the type name “map” and support one attribute:
- values: the schema of the map’s values.
Map keys are assumed to be strings.
For example, a map from string to long is declared with:
{
"type": "map",
"values" : "long"
}
Unions
Unions, as mentioned above, are represented using JSON arrays. For example, ["string", "null"] declares a schema which may be either a string or null.
Unions may not contain more than one schema with the same type, except for the named types record, fixed and enum. For example, unions containing two array types or two map types are not permitted, but two types with different names are permitted. (Names permit efficient resolution when reading and writing unions.)
Unions may not immediately contain other unions.
Fixed
Fixed uses the type name “fixed” and supports two attributes:
- name: a string naming this fixed (required).
- namespace, a string that qualifies the name;
- aliases: a JSON array of strings, providing alternate names for this enum (optional).
- size: an integer, specifying the number of bytes per value (required).
For example, 16-byte quantity may be declared with:
{"type": "fixed", "size": 16, "name": "md5"}
Names
Record, enums and fixed are named types. Each has a fullname that is composed of two parts; a name and a namespace. Equality of names is defined on the fullname.
The name portion of a fullname, record field names, and enum symbols must:
- start with [A-Za-z_]
- subsequently contain only [A-Za-z0-9_]
A namespace is a dot-separated sequence of such names. Equality of names (including field names and enum symbols) as well as fullnames is case-sensitive.
In record, enum and fixed definitions, the fullname is determined in one of the following ways:
A name and namespace are both specified. For example, one might use “name”: “X”, “namespace”: “org.foo” to indicate the fullname org.foo.X.
-
A fullname is specified. If the name specified contains a dot, then it is assumed to be a fullname, and any namespace also specified is ignored. For example, use “name”: “org.foo.X” to indicate the fullname org.foo.X.
-
A name only is specified, i.e., a name that contains no dots. In this case the namespace is taken from the most tightly enclosing schema or protocol. For example, if “name”: “X” is specified, and this occurs within a field of the record definition of org.foo.Y, then the fullname is org.foo.X.
References to previously defined names are as in the latter two cases above: if they contain a dot they are a fullname, if they do not contain a dot, the namespace is the namespace of the enclosing definition.
Primitive type names have no namespace and their names may not be defined in any namespace.
A schema or protocol may not contain multiple definitions of a fullname. Further, a name must be defined before it is used (“before” in the depth-first, left-to-right traversal of the JSON parse tree, where the types attribute of a protocol is always deemed to come “before” the messages attribute.)
Aliases
Named types and fields may have aliases. An implementation may optionally use aliases to map a writer’s schema to the reader’s. This facilitates both schema evolution as well as processing disparate datasets.
Aliases function by re-writing the writer’s schema using aliases from the reader’s schema. For example, if the writer’s schema was named “Foo” and the reader’s schema is named “Bar” and has an alias of “Foo”, then the implementation would act as though “Foo” were named “Bar” when reading. Similarly, if data was written as a record with a field named “x” and is read as a record with a field named “y” with alias “x”, then the implementation would act as though “x” were named “y” when reading.
A type alias may be specified either as a fully namespace-qualified, or relative to the namespace of the name it is an alias for. For example, if a type named “a.b” has aliases of “c” and “x.y”, then the fully qualified names of its aliases are “a.c” and “x.y”.
Data Serialization
Avro data is always serialized with its schema. Files that store Avro data should always also include the schema for that data in the same file. Avro-based remote procedure call (RPC) systems must also guarantee that remote recipients of data have a copy of the schema used to write that data.
Because the schema used to write data is always available when the data is read, Avro data itself is not tagged with type information. The schema is required to parse data.
In general, both serialization and deserialization proceed as a depth-first, left-to-right traversal of the schema, serializing primitive types as they are encountered.
Encodings
Avro specifies two serialization encodings: binary and JSON. Most applications will use the binary encoding, as it is smaller and faster. But, for debugging and web-based applications, the JSON encoding may sometimes be appropriate.
Binary Encoding
Primitive Types
Primitive types are encoded in binary as follows:
- null is written as zero bytes.
- a boolean is written as a single byte whose value is either 0 (false) or 1 (true).
- int and long values are written using variable-length zig-zag coding. Some examples:
| value |
hex |
| 0 |
00 |
| -1 |
01 |
| 1 |
02 |
| -2 |
03 |
| 2 |
04 |
| … |
|
| -64 |
7f |
| 64 |
80 01 |
| … |
|
- a float is written as 4 bytes. The float is converted into a 32-bit integer using a method equivalent to Java’s floatToIntBits and then encoded in little-endian format.
- a double is written as 8 bytes. The double is converted into a 64-bit integer using a method equivalent to Java’s doubleToLongBits and then encoded in little-endian format.
- bytes are encoded as a long followed by that many bytes of data.
- a string is encoded as a long followed by that many bytes of UTF-8 encoded character data.
For example, the three-character string “foo” would be encoded as the long value 3 (encoded as hex 06) followed by the UTF-8 encoding of ‘f’, ‘o’, and ‘o’ (the hex bytes 66 6f 6f):
06 66 6f 6f
Complex Types
Complex types are encoded in binary as follows:
Records
A record is encoded by encoding the values of its fields in the order that they are declared. In other words, a record is encoded as just the concatenation of the encodings of its fields. Field values are encoded per their schema.
For example, the record schema
{
"type": "record",
"name": "test",
"fields" : [
{"name": "a", "type": "long"},
{"name": "b", "type": "string"}
]
}
An instance of this record whose a field has value 27 (encoded as hex 36) and whose b field has value “foo” (encoded as hex bytes 06 66 6f 6f), would be encoded simply as the concatenation of these, namely the hex byte sequence:
36 06 66 6f 6f
Enums
An enum is encoded by a int, representing the zero-based position of the symbol in the schema.
For example, consider the enum:
{"type": "enum", "name": "Foo", "symbols": ["A", "B", "C", "D"] }
This would be encoded by an int between zero and three, with zero indicating “A”, and 3 indicating “D”.
Arrays
Arrays are encoded as a series of blocks. Each block consists of a long count value, followed by that many array items. A block with count zero indicates the end of the array. Each item is encoded per the array’s item schema.
If a block’s count is negative, its absolute value is used, and the count is followed immediately by a long block size indicating the number of bytes in the block. This block size permits fast skipping through data, e.g., when projecting a record to a subset of its fields.
For example, the array schema
{"type": "array", "items": "long"}
an array containing the items 3 and 27 could be encoded as the long value 2 (encoded as hex 04) followed by long values 3 and 27 (encoded as hex 06 36) terminated by zero:
04 06 36 00
The blocked representation permits one to read and write arrays larger than can be buffered in memory, since one can start writing items without knowing the full length of the array.
Maps
Maps are encoded as a series of blocks. Each block consists of a long count value, followed by that many key/value pairs. A block with count zero indicates the end of the map. Each item is encoded per the map’s value schema.
If a block’s count is negative, its absolute value is used, and the count is followed immediately by a long block size indicating the number of bytes in the block. This block size permits fast skipping through data, e.g., when projecting a record to a subset of its fields.
The blocked representation permits one to read and write maps larger than can be buffered in memory, since one can start writing items without knowing the full length of the map.
Unions
A union is encoded by first writing a long value indicating the zero-based position within the union of the schema of its value. The value is then encoded per the indicated schema within the union.
For example, the union schema ["string","null"] would encode:
-
null as the integer 1 (the index of “null” in the union, encoded as hex 02):
02
-
the string “a” as zero (the index of “string” in the union), followed by the serialized string:
02 02 61
Fixed
Fixed instances are encoded using the number of bytes declared in the schema.
JSON Encoding
Except for unions, the JSON encoding is the same as is used to encode field default values.
The value of a union is encoded in JSON as follows:
- if its type is null, then it is encoded as a JSON null;
- otherwise it is encoded as a JSON object with one name/value pair whose name is the type’s name and whose value is the recursively encoded value. For Avro’s named types (record, fixed or enum) the user-specified name is used, for other types the type name is used.
For example, the union schema ["null","string","Foo"], where Foo is a record name, would encode:
- null as null;
- the string “a” as
{"string": "a"} and
- a Foo instance as
{"Foo": {...}}, where {...} indicates the JSON encoding of a Foo instance.
Note that a schema is still required to correctly process JSON-encoded data. For example, the JSON encoding does not distinguish between int and long, float and double, records and maps, enums and strings, etc.
Sort Order
Avro defines a standard sort order for data. This permits data written by one system to be efficiently sorted by another system. This can be an important optimization, as sort order comparisons are sometimes the most frequent per-object operation. Note also that Avro binary-encoded data can be efficiently ordered without deserializing it to objects.
Data items may only be compared if they have identical schemas. Pairwise comparisons are implemented recursively with a depth-first, left-to-right traversal of the schema. The first mismatch encountered determines the order of the items.
Two items with the same schema are compared according to the following rules.
- null data is always equal.
- boolean data is ordered with false before true.
- int, long, float and double data is ordered by ascending numeric value.
- bytes and fixed data are compared lexicographically by unsigned 8-bit values.
- string data is compared lexicographically by Unicode code point. Note that since UTF-8 is used as the binary encoding for strings, sorting of bytes and string binary data is identical.
- array data is compared lexicographically by element.
- enum data is ordered by the symbol’s position in the enum schema. For example, an enum whose symbols are
["z", "a"] would sort “z” values before “a” values.
- union data is first ordered by the branch within the union, and, within that, by the type of the branch. For example, an
["int", "string"] union would order all int values before all string values, with the ints and strings themselves ordered as defined above.
- record data is ordered lexicographically by field. If a field specifies that its order is:
- “ascending”, then the order of its values is unaltered.
- “descending”, then the order of its values is reversed.
- “ignore”, then its values are ignored when sorting.
- map data may not be compared. It is an error to attempt to compare data containing maps unless those maps are in an
"order":"ignore" record field.
Object Container Files
Avro includes a simple object container file format. A file has a schema, and all objects stored in the file must be written according to that schema, using binary encoding. Objects are stored in blocks that may be compressed. Syncronization markers are used between blocks to permit efficient splitting of files for MapReduce processing.
Files may include arbitrary user-specified metadata.
A file consists of:
- A file header, followed by
- one or more file data blocks.
A file header consists of:
- Four bytes, ASCII ‘O’, ‘b’, ‘j’, followed by 1.
- file metadata, including the schema.
- The 16-byte, randomly-generated sync marker for this file.
File metadata is written as if defined by the following map schema:
{"type": "map", "values": "bytes"}
All metadata properties that start with “avro.” are reserved. The following file metadata properties are currently used:
- avro.schema contains the schema of objects stored in the file, as JSON data (required).
- avro.codec the name of the compression codec used to compress blocks, as a string. Implementations are required to support the following codecs: “null” and “deflate”. If codec is absent, it is assumed to be “null”. The codecs are described with more detail below.
A file header is thus described by the following schema:
{"type": "record", "name": "org.apache.avro.file.Header",
"fields" : [
{"name": "magic", "type": {"type": "fixed", "name": "Magic", "size": 4}},
{"name": "meta", "type": {"type": "map", "values": "bytes"}},
{"name": "sync", "type": {"type": "fixed", "name": "Sync", "size": 16}}
]
}
A file data block consists of:
- A long indicating the count of objects in this block.
- A long indicating the size in bytes of the serialized objects in the current block, after any codec is applied
- The serialized objects. If a codec is specified, this is compressed by that codec.
- The file’s 16-byte sync marker.
Thus, each block’s binary data can be efficiently extracted or skipped without deserializing the contents. The combination of block size, object counts, and sync markers enable detection of corrupt blocks and help ensure data integrity.
Required Codecs
null
The “null” codec simply passes through data uncompressed.
deflate
The “deflate” codec writes the data block using the deflate algorithm as specified in RFC 1951, and typically implemented using the zlib library. Note that this format (unlike the “zlib format” in RFC 1950) does not have a checksum.
Optional Codecs
snappy
The “snappy” codec uses Google’s Snappy compression library. Each compressed block is followed by the 4-byte, big-endian CRC32 checksum of the uncompressed data in the block.
Protocol Declaration
Avro protocols describe RPC interfaces. Like schemas, they are defined with JSON text.
A protocol is a JSON object with the following attributes:
- protocol, a string, the name of the protocol (required);
- namespace, an optional string that qualifies the name;
- doc, an optional string describing this protocol;
- types, an optional list of definitions of named types (records, enums, fixed and errors). An error definition is just like a record definition except it uses “error” instead of “record”. Note that forward references to named types are not permitted.
- messages, an optional JSON object whose keys are message names and whose values are objects whose attributes are described below. No two messages may have the same name.
The name and namespace qualification rules defined for schema objects apply to protocols as well.
Messages
A message has attributes:
- a doc, an optional description of the message,
- a request, a list of named, typed parameter schemas (this has the same form as the fields of a record declaration);
- a response schema;
- an optional union of declared error schemas. The effective union has “string” prepended to the declared union, to permit transmission of undeclared “system” errors. For example, if the declared error union is
["AccessError"], then the effective union is ["string", "AccessError"]. When no errors are declared, the effective error union is ["string"]. Errors are serialized using the effective union; however, a protocol’s JSON declaration contains only the declared union.
- an optional one-way boolean parameter.
A request parameter list is processed equivalently to an anonymous record. Since record field lists may vary between reader and writer, request parameters may also differ between the caller and responder, and such differences are resolved in the same manner as record field differences.
The one-way parameter may only be true when the response type is "null" and no errors are listed.
Sample Protocol
For example, one may define a simple HelloWorld protocol with:
{
"namespace": "com.acme",
"protocol": "HelloWorld",
"doc": "Protocol Greetings",
"types": [
{"name": "Greeting", "type": "record", "fields": [
{"name": "message", "type": "string"}]},
{"name": "Curse", "type": "error", "fields": [
{"name": "message", "type": "string"}]}
],
"messages": {
"hello": {
"doc": "Say hello.",
"request": [{"name": "greeting", "type": "Greeting" }],
"response": "Greeting",
"errors": ["Curse"]
}
}
}
Message Transport
Messages may be transmitted via different transport mechanisms.
To the transport, a message is an opaque byte sequence.
A transport is a system that supports:
- transmission of request messages
- receipt of corresponding response messages
Servers may send a response message back to the client corresponding to a request message. The mechanism of correspondance is transport-specific. For example, in HTTP it is implicit, since HTTP directly supports requests and responses. But a transport that multiplexes many client threads over a single socket would need to tag messages with unique identifiers.
Transports may be either stateless or stateful. In a stateless transport, messaging assumes no established connection state, while stateful transports establish connections that may be used for multiple messages. This distinction is discussed further in the handshake section below.
HTTP as Transport
When HTTP is used as a transport, each Avro message exchange is an HTTP request/response pair. All messages of an Avro protocol should share a single URL at an HTTP server. Other protocols may also use that URL. Both normal and error Avro response messages should use the 200 (OK) response code. The chunked encoding may be used for requests and responses, but, regardless the Avro request and response are the entire content of an HTTP request and response. The HTTP Content-Type of requests and responses should be specified as “avro/binary”. Requests should be made using the POST method.
HTTP is used by Avro as a stateless transport.
Message Framing
Avro messages are framed as a list of buffers.
Framing is a layer between messages and the transport. It exists to optimize certain operations.
The format of framed message data is:
- a series of buffers, where each buffer consists of:
- a four-byte, big-endian buffer length, followed by
- that many bytes of buffer data.
- A message is always terminated by a zero-lenghted buffer.
Framing is transparent to request and response message formats (described below). Any message may be presented as a single or multiple buffers.
Framing can permit readers to more efficiently get different buffers from different sources and for writers to more efficiently store different buffers to different destinations. In particular, it can reduce the number of times large binary objects are copied. For example, if an RPC parameter consists of a megabyte of file data, that data can be copied directly to a socket from a file descriptor, and, on the other end, it could be written directly to a file descriptor, never entering user space.
A simple, recommended, framing policy is for writers to create a new segment whenever a single binary object is written that is larger than a normal output buffer. Small objects are then appended in buffers, while larger objects are written as their own buffers. When a reader then tries to read a large object the runtime can hand it an entire buffer directly, without having to copy it.
Handshake
The purpose of the handshake is to ensure that the client and the server have each other’s protocol definition, so that the client can correctly deserialize responses, and the server can correctly deserialize requests. Both clients and servers should maintain a cache of recently seen protocols, so that, in most cases, a handshake will be completed without extra round-trip network exchanges or the transmission of full protocol text.
RPC requests and responses may not be processed until a handshake has been completed. With a stateless transport, all requests and responses are prefixed by handshakes. With a stateful transport, handshakes are only attached to requests and responses until a successful handshake response has been returned over a connection. After this, request and response payloads are sent without handshakes for the lifetime of that connection.
The handshake process uses the following record schemas:
{
"type": "record",
"name": "HandshakeRequest", "namespace":"org.apache.avro.ipc",
"fields": [
{"name": "clientHash",
"type": {"type": "fixed", "name": "MD5", "size": 16}},
{"name": "clientProtocol", "type": ["null", "string"]},
{"name": "serverHash", "type": "MD5"},
{"name": "meta", "type": ["null", {"type": "map", "values": "bytes"}]}
]
}
{
"type": "record",
"name": "HandshakeResponse", "namespace": "org.apache.avro.ipc",
"fields": [
{"name": "match",
"type": {"type": "enum", "name": "HandshakeMatch",
"symbols": ["BOTH", "CLIENT", "NONE"]}},
{"name": "serverProtocol",
"type": ["null", "string"]},
{"name": "serverHash",
"type": ["null", {"type": "fixed", "name": "MD5", "size": 16}]},
{"name": "meta",
"type": ["null", {"type": "map", "values": "bytes"}]}
]
}
- A client first prefixes each request with a
HandshakeRequest containing just the hash of its protocol and of the server’s protocol (clientHash!=null, clientProtocol=null, serverHash!=null), where the hashes are 128-bit MD5 hashes of the JSON protocol text. If a client has never connected to a given server, it sends its hash as a guess of the server’s hash, otherwise it sends the hash that it previously obtained from this server.
The server responds with a HandshakeResponse containing one of:
-
match=BOTH, serverProtocol=null, serverHash=null if the client sent the valid hash of the server’s protocol and the server knows what protocol corresponds to the client’s hash. In this case, the request is complete and the response data immediately follows the HandshakeResponse.
-
match=CLIENT, serverProtocol!=null, serverHash!=null if the server has previously seen the client’s protocol, but the client sent an incorrect hash of the server’s protocol. The request is complete and the response data immediately follows the HandshakeResponse. The client must use the returned protocol to process the response and should also cache that protocol and its hash for future interactions with this server.
-
match=NONE if the server has not previously seen the client’s protocol. The serverHash and serverProtocol may also be non-null if the server’s protocol hash was incorrect.
In this case the client must then re-submit its request with its protocol text (clientHash!=null, clientProtocol!=null, serverHash!=null) and the server should respond with a successful match (match=BOTH, serverProtocol=null, serverHash=null) as above.
The meta field is reserved for future handshake enhancements.
A call consists of a request message paired with its resulting response or error message. Requests and responses contain extensible metadata, and both kinds of messages are framed as described above.
The format of a call request is:
- request metadata, a map with values of type bytes
- the message name, an Avro string, followed by
- the message parameters. Parameters are serialized according to the message’s request declaration.
When the empty string is used as a message name a server should ignore the parameters and return an empty response. A client may use this to ping a server or to perform a handshake without sending a protocol message.
When a message is declared one-way and a stateful connection has been established by a successful handshake response, no response data is sent. Otherwise the format of the call response is:
- response metadata, a map with values of type bytes
- a one-byte error flag boolean, followed by either:
- if the error flag is false, the message response, serialized per the message’s response schema.
- if the error flag is true, the error, serialized per the message’s effective error union schema.
Schema Resolution
A reader of Avro data, whether from an RPC or a file, can always parse that data because its schema is provided. But that schema may not be exactly the schema that was expected. For example, if the data was written with a different version of the software than it is read, then records may have had fields added or removed. This section specifies how such schema differences should be resolved.
We refer to the schema used to write the data as the writer’s schema, and the schema that the application expects the reader’s schema. Differences between these should be resolved as follows:
- It is an error if the two schemas do not match.
To match, one of the following must hold:
-
both schemas are arrays whose item types match
-
both schemas are maps whose value types match
-
both schemas are enums whose names match
-
both schemas are fixed whose sizes and names match
-
both schemas are records with the same name
-
either schema is a union
-
both schemas have same primitive type
-
the writer’s schema may be promoted to the reader’s as follows:
- int is promotable to long, float, or double
- long is promotable to float or double
- float is promotable to double
- string is promotable to bytes
- bytes is promotable to string
-
if both are records:
- the ordering of fields may be different: fields are matched by name.
- schemas for fields with the same name in both records are resolved recursively.
- if the writer’s record contains a field with a name not present in the reader’s record, the writer’s value for that field is ignored.
- if the reader’s record schema has a field that contains a default value, and writer’s schema does not have a field with the same name, then the reader should use the default value from its field.
- if the reader’s record schema has a field with no default value, and writer’s schema does not have a field with the same name, an error is signalled.
-
if both are enums:
if the writer’s symbol is not present in the reader’s enum and the reader has a default value, then that value is used, otherwise an error is signalled.
This resolution algorithm is applied recursively to the reader’s and writer’s array item schemas.
This resolution algorithm is applied recursively to the reader’s and writer’s value schemas.
The first schema in the reader’s union that matches the selected writer’s union schema is recursively resolved against it. if none match, an error is signalled.
- if reader’s is a union, but writer’s is not
The first schema in the reader’s union that matches the writer’s schema is recursively resolved against it. If none match, an error is signalled.
- if writer’s is a union, but reader’s is not
If the reader’s schema matches the selected writer’s schema, it is recursively resolved against it. If they do not match, an error is signalled.
A schema’s doc fields are ignored for the purposes of schema resolution. Hence, the doc portion of a schema may be dropped at serialization.
One of the defining characteristics of Avro is that a reader is assumed to have the “same” schema used by the writer of the data the reader is reading. This assumption leads to a data format that’s compact and also amenable to many forms of schema evolution. However, the specification so far has not defined what it means for the reader to have the “same” schema as the writer. Does the schema need to be textually identical? Well, clearly adding or removing some whitespace to a JSON expression does not change its meaning. At the same time, reordering the fields of records clearly does change the meaning. So what does it mean for a reader to have “the same” schema as a writer?
Parsing Canonical Form is a transformation of a writer’s schema that let’s us define what it means for two schemas to be “the same” for the purpose of reading data written against the schema. It is called Parsing Canonical Form because the transformations strip away parts of the schema, like “doc” attributes, that are irrelevant to readers trying to parse incoming data. It is called Canonical Form because the transformations normalize the JSON text (such as the order of attributes) in a way that eliminates unimportant differences between schemas. If the Parsing Canonical Forms of two different schemas are textually equal, then those schemas are “the same” as far as any reader is concerned, i.e., there is no serialized data that would allow a reader to distinguish data generated by a writer using one of the original schemas from data generated by a writing using the other original schema. (We sketch a proof of this property in a companion document.)
The next subsection specifies the transformations that define Parsing Canonical Form. But with a well-defined canonical form, it can be convenient to go one step further, transforming these canonical forms into simple integers (“fingerprints”) that can be used to uniquely identify schemas. The subsection after next recommends some standard practices for generating such fingerprints.
Assuming an input schema (in JSON form) that’s already UTF-8 text for a valid Avro schema (including all quotes as required by JSON), the following transformations will produce its Parsing Canonical Form:
- [PRIMITIVES] Convert primitive schemas to their simple form (e.g., int instead of
{"type":"int"}).
- [FULLNAMES] Replace short names with fullnames, using applicable namespaces to do so. Then eliminate namespace attributes, which are now redundant.
- [STRIP] Keep only attributes that are relevant to parsing data, which are: type, name, fields, symbols, items, values, size. Strip all others (e.g., doc and aliases).
- [ORDER] Order the appearance of fields of JSON objects as follows: name, type, fields, symbols, items, values, size. For example, if an object has type, name, and size fields, then the name field should appear first, followed by the type and then the size fields.
- [STRINGS] For all JSON string literals in the schema text, replace any escaped characters (e.g., \uXXXX escapes) with their UTF-8 equivalents.
- [INTEGERS] Eliminate quotes around and any leading zeros in front of JSON integer literals (which appear in the size attributes of fixed schemas).
- [WHITESPACE] Eliminate all whitespace in JSON outside of string literals.
Schema Fingerprints
“[A] fingerprinting algorithm is a procedure that maps an arbitrarily large data item (such as a computer file) to a much shorter bit string, its fingerprint, that uniquely identifies the original data for all practical purposes” (quoted from Wikipedia). In the Avro context, fingerprints of Parsing Canonical Form can be useful in a number of applications; for example, to cache encoder and decoder objects, to tag data items with a short substitute for the writer’s full schema, and to quickly negotiate common-case schemas between readers and writers.
In designing fingerprinting algorithms, there is a fundamental trade-off between the length of the fingerprint and the probability of collisions. To help application designers find appropriate points within this trade-off space, while encouraging interoperability and ease of implementation, we recommend using one of the following three algorithms when fingerprinting Avro schemas:
- When applications can tolerate longer fingerprints, we recommend using the SHA-256 digest algorithm to generate 256-bit fingerprints of Parsing Canonical Forms. Most languages today have SHA-256 implementations in their libraries.
- At the opposite extreme, the smallest fingerprint we recommend is a 64-bit Rabin fingerprint. Below, we provide pseudo-code for this algorithm that can be easily translated into any programming language. 64-bit fingerprints should guarantee uniqueness for schema caches of up to a million entries (for such a cache, the chance of a collision is 3E-8). We don’t recommend shorter fingerprints, as the chances of collisions is too great (for example, with 32-bit fingerprints, a cache with as few as 100,000 schemas has a 50% chance of having a collision).
- Between these two extremes, we recommend using the MD5 message digest to generate 128-bit fingerprints. These make sense only where very large numbers of schemas are being manipulated (tens of millions); otherwise, 64-bit fingerprints should be sufficient. As with SHA-256, MD5 implementations are found in most libraries today.
These fingerprints are not meant to provide any security guarantees, even the longer SHA-256-based ones. Most Avro applications should be surrounded by security measures that prevent attackers from writing random data and otherwise interfering with the consumers of schemas. We recommend that these surrounding mechanisms be used to prevent collision and pre-image attacks (i.e., “forgery”) on schema fingerprints, rather than relying on the security properties of the fingerprints themselves.
Rabin fingerprints are cyclic redundancy checks computed using irreducible polynomials. In the style of the Appendix of RFC 1952 (pg 10), which defines the CRC-32 algorithm, here’s our definition of the 64-bit AVRO fingerprinting algorithm:
long fingerprint64(byte[] buf) {
if (FP_TABLE == null) initFPTable();
long fp = EMPTY;
for (int i = 0; i < buf.length; i++)
fp = (fp >>> 8) ^ FP_TABLE[(int)(fp ^ buf[i]) & 0xff];
return fp;
}
static long EMPTY = 0xc15d213aa4d7a795L;
static long[] FP_TABLE = null;
void initFPTable() {
FP_TABLE = new long[256];
for (int i = 0; i < 256; i++) {
long fp = i;
for (int j = 0; j < 8; j++)
fp = (fp >>> 1) ^ (EMPTY & -(fp & 1L));
FP_TABLE[i] = fp;
}
}
Readers interested in the mathematics behind this algorithm may want to read this book chapter. (Unlike RFC-1952 and the book chapter, we prepend a single one bit to messages. We do this because CRCs ignore leading zero bits, which can be problematic. Our code prepends a one-bit by initializing fingerprints using EMPTY, rather than initializing using zero as in RFC-1952 and the book chapter.)
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
15.4 - Trevni
Version 0.1
DRAFT
This document is the authoritative specification of a file format. Its intent is to permit compatible, independent implementations that read and/or write files in this format.
Introduction
Data sets are often described as a table composed of rows and columns. Each record in the dataset is considered a row, with each field of the record occupying a different column. Writing records to a file one-by-one as they are created results in a row-major format, like Hadoop’s SequenceFile or Avro data files.
In many cases higher query performance may be achieved if the data is instead organized in a column-major format, where multiple values of a given column are stored adjacently. This document defines such a column-major file format for datasets.
To permit scalable, distributed query evaluation, datasets are partitioned into row groups, containing distinct collections of rows. Each row group is organized in column-major order, while row groups form a row-major partitioning of the entire dataset.
Rationale
Goals
The format is meant satisfy the following goals:
Maximize the size of row groups. Disc drives are used most efficiently when sequentially accessing data. Consider a drive that takes 10ms to seek and transfers at 100MB/second. If a 10-column dataset whose values are all the same size is split into 10MB row groups, then accessing a single column will require a sequence of seek+1MB reads, for a cost of 20ms/MB processed. If the same dataset is split into 100MB row groups then this drops to 11ms/MB processed. This effect is exaggerated for datasets with larger numbers of columns and with columns whose values are smaller than average. So we’d prefer row groups that are 100MB or greater.
Permit random access within a row group. Some queries will first examine one column, and, only when certain relatively rare criteria are met, examine other columns. Rather than iterating through selected columns of the row-group in parallel, one might iterate through one column and randomly access another. This is called support for WHERE clauses, after the SQL operator of that name.
Minimize the number of files per dataset. HDFS is a primary intended deployment platform for these files. The HDFS Namenode requires memory for each file in the filesystem, thus for a format to be HDFS-friendly it should strive to require the minimum number of distinct files.
Support co-location of columns within row-groups. Row groups are the unit of parallel operation on a column dataset. For efficient file i/o, the entirety of a row-group should ideally reside on the host that is evaluating the query in order to avoid network latencies and bottlenecks.
Data integrity. The format should permit applications to detect data corruption. Many file systems may prevent corruption, but files may be moved between filesystems and be subject to corruption at points in that process. It is best if the data in a file can be validated independently.
Extensibility. The format should permit applications to store additional annotations about a datasets in the files, such as type information, origin, etc. Some environments may have metadata stores for such information, but not all do, and files might be moved among systems with different metadata systems. The ability to keep such information within the file simplifies the coordination of such information.
Minimal overhead. The column format should not make datasets appreciably larger. Storage is a primary cost and a choice to use this format should not require additional storage.
Primary format. The column format should be usable as a primary format for datasets, not as an auxiliary, accelerated format. Applications that process a dataset in row-major order should be able to easily consume column files and applications that produce datasets in row-major order should be able to easily generate column files.
Design
To meet these goals we propose the following design.
Each row group is a separate file. All values of a column in a file are written contiguously. This maximizes the row group size, optimizing performance when querying few and small columns.
Each file occupies a single HDFS block. A larger than normal block size may be specified, e.g., ~1GB instead of the typical ~100MB. This guarantees co-location and eliminates network use when query processing can be co-located with the file. This also moderates the memory impact on the HDFS Namenode since no small files are written.
Each column in a file is written as a sequence of ~64kB compressed blocks. The sequence is prefixed by a table describing all of the blocks in the column to permit random access within the column.
Application-specific metadata may be added at the file, column, and block levels.
Checksums are included with each block, providing data integrity.
Discussion
The use of a single block per file achieves the same effect as the custom block placement policy described in the CIF paper, but while still permitting HDFS rebalancing and not increasing the number of files in the namespace.
This section formally describes the proposed column file format.
Data Model
We assume a simple data model, where a record is a set of named fields, and the value of each field is a sequence of untyped bytes. A type system may be layered on top of this, as specified in the Type Mapping section below.
Primitive Values
We define the following primitive value types:
- Signed 64-bit long values are written using a variable-length zig-zag coding, where the high-order bit in each byte determines whether subsequent bytes are present. For example:
decimal value hex bytes
| decimal value |
hex bytes |
|
| 0 |
00 |
|
| -1 |
01 |
|
| 1 |
02 |
|
| … |
|
|
| -64 |
7f |
|
| 64 |
80 01 |
|
| … |
|
|
- bytes are encoded as a long followed by that many bytes of data.
- a string is encoded as a long followed by that many bytes of UTF-8 encoded character data.
For example, the three-character string “foo” would be encoded as the long value 3 (encoded as hex 06) followed by the UTF-8 encoding of ‘f’, ‘o’, and ‘o’ (the hex bytes 66 6f 6f): 06 66 6f 6f
Type Names
The following type names are used to describe column values:
- null, requires zero bytes. Sometimes used in array columns.
- boolean, one bit, packed into bytes, little-endian;
- int, like long, but restricted to 32-bit signed values
- long 64-bit signed values, represented as above
- fixed32 32-bit values stored as four bytes, little-endian.
- fixed64 64-bit values stored as eight bytes, little-endian.
- float 32-bit IEEE floating point value, little-endian
- double 64-bit IEEE floating point value, little-endian
- string as above
- bytes as above, may be used to encapsulate more complex objects
Type names are represented as strings (UTF-8 encoded, length-prefixed).
Metadata consists of:
- A long indicating the number of metadata key/value pairs.
- For each pair, a string key and bytes value.
All metadata properties that start with “trevni.” are reserved.
The following file metadata properties are defined:
- trevni.codec the name of the default compression codec used to compress blocks, as a string. Implementations are required to support the “null” codec. Optional. If absent, it is assumed to be “null”. Codecs are described in more detail below.
- trevni.checksum the name of the checksum algorithm used in this file, as a string. Implementations are required to support the “crc-32” checksum. Optional. If absent, it is assumed to be “null”. Checksums are described in more detail below.
The following column metadata properties are defined:
- trevni.codec the name of the compression codec used to compress the blocks of this column, as a string. Implementations are required to support the “null” codec. Optional. If absent, it is assumed to be “null”. Codecs are described in more detail below.
- trevni.name the name of the column, as a string. Required.
- trevni.type the type of data in the column. One of the type names above. Required.
- trevni.values if present, indicates that the initial value of each block in this column will be stored in the block’s descriptor. Not permitted for array columns or columns that specify a parent.
- trevni.array if present, indicates that each row in this column contains a sequence of values of the named type rather than just a single value. An integer length precedes each sequence of values indicating the count of values in the sequence.
- trevni.parent if present, the name of an array column whose lengths are also used by this column. Thus values of this column are sequences but no lengths are stored in this column.
For example, consider the following row, as JSON, where all values are primitive types, but one has multiple values.
{"id"=566, "date"=23423234234
"from"="foo@bar.com",
"to"=["bar@baz.com", "bang@foo.com"],
"content"="Hi!"}
The columns for this might be specified as:
name=id type=int
name=date type=long
name=from type=string
name=to type=string array=true
name=content type=string
If a row contains an array of records, e.g. “received” in the following:
{"id"=566, "date"=23423234234
"from"="foo@bar.com",
"to"=["bar@baz.com", "bang@foo.com"],
"content"="Hi!"
"received"=[{"date"=234234234234, "host"="192.168.0.0.1"},
{"date"=234234545645, "host"="192.168.0.0.2"}]
}
Then one can define a parent column followed by a column for each field in the record, adding the following columns:
name=received type=null array=true
name=date type=long parent=received
name=host type=string parent=received
If an array value itself contains an array, e.g. the “sigs” below:
{"id"=566, "date"=23423234234
"from"="foo@bar.com",
"to"=["bar@baz.com", "bang@foo.com"],
"content"="Hi!"
"received"=[{"date"=234234234234, "host"="192.168.0.0.1",
"sigs"=[{"algo"="weak", "value"="0af345de"}]},
{"date"=234234545645, "host"="192.168.0.0.2",
"sigs"=[]}]
}
Then a parent column may be defined that itself has a parent column.
name=sigs type=null array=true parent=received
name=algo type=string parent=sigs
name=value type=string parent=sigs
No block metadata properties are currently defined.
A file consists of:
- A file header, followed by
- one or more columns.
A file header consists of:
- Four bytes, ASCII ‘T’, ‘r’, ‘v’, followed by 0x02.
- a fixed64 indicating the number of rows in the file
- a fixed32 indicating the number of columns in the file
- file metadata.
- for each column, its column metadata
- for each column, its starting position in the file as a fixed64.
A column consists of:
- A fixed32 indicating the number of blocks in this column.
- For each block, a block descriptor
- One or more blocks.
A block descriptor consists of:
- A fixed32 indicating the number of rows in the block
- A fixed32 indicating the size in bytes of the block before the codec is applied (excluding checksum).
- A fixed32 indicating the size in bytes of the block after the codec is applied (excluding checksum).
- If this column’s metadata declares it to include values, the first value in the column, serialized according to this column’s type.
A block consists of:
- The serialized column values. If a column is an array column then value sequences are preceded by their length, as an int. If a codec is specified, the values and lengths are compressed by that codec.
- The checksum, as determined by the file metadata.
Codecs
null
The “null” codec simply passes data through uncompressed.
deflate
The “deflate” codec writes the data block using the deflate algorithm as specified in RFC 1951.
snappy
The “snappy” codec uses Google’s Snappy compression library.
Checksum algorithms
null
The “null” checksum contains zero bytes.
crc-32
Each “crc-32” checksum contains the four bytes of an ISO 3309 CRC-32 checksum of the uncompressed block data as a fixed32.
Type Mappings
We define a standard mapping for how types defined in various serialization systems are represented in a column file. Records from these systems are shredded into columns. When records are nested, a depth-first recursive walk can assign a separate column for each primitive value.
Avro
Protocol Buffers
Thrift
Implementation Notes
Some possible techniques for writing column files include:
- Use a standard ~100MB block, buffer in memory up to the block size, then flush the file directly to HDFS. A single reduce task might create multiple output files. The namenode requires memory proportional to the number of names and blocks*replication. This would increase the number of names but not blocks, so this should still be much better than a file per column.
- Spill each column to a separate local, temporary file then, when the file is closed, append these files, writing a single file to HDFS whose block size is set to be that of the entire file. This would be a bit slower than and may have trouble when the local disk is full, but it would better use HDFS namespace and further reduce seeks when processing columns whose values are small.
- Use a separate mapreduce job to convert row-major files to column-major. The map output would output a by (row#, column#, value) tuple, partitioned by row# but sorted by column# then row#. The reducer could directly write the column file. But the column file format would need to be changed to write counts, descriptors, etc. at the end of files rather than at the front.
(1) is the simplest to implement and most implementations should start with it.
References
CIF Column-Oriented Storage Techniques for MapReduce, Floratou, Patel, Shekita, & Tata, VLDB 2011.
DREMEL Dremel: Interactive Analysis of Web-Scale Datasets, Melnik, Gubarev, Long, Romer, Shivakumar, & Tolton, VLDB 2010.
15.5 - MapReduce guide
Avro provides a convenient way to represent complex data structures within a Hadoop MapReduce job. Avro data can be used as both input to and output from a MapReduce job, as well as the intermediate format. The example in this guide uses Avro data for all three, but it’s possible to mix and match; for instance, MapReduce can be used to aggregate a particular field in an Avro record.
This guide assumes basic familiarity with both Hadoop MapReduce and Avro. See the Hadoop documentation and the Avro getting started guide for introductions to these projects. This guide uses the old MapReduce API (org.apache.hadoop.mapred).
Setup
The code from this guide is included in the Avro docs under examples/mr-example. The example is set up as a Maven project that includes the necessary Avro and MapReduce dependencies and the Avro Maven plugin for code generation, so no external jars are needed to run the example. In particular, the POM includes the following dependencies:
<dependency>
<groupId>org.apache.avro</groupId>
<artifactId>avro</artifactId>
<version>1.7.4</version>
</dependency>
<dependency>
<groupId>org.apache.avro</groupId>
<artifactId>avro-mapred</artifactId>
<version>1.7.4</version>
</dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-core</artifactId>
<version>1.1.0</version>
</dependency>
And the following plugin:
<plugin>
<groupId>org.apache.avro</groupId>
<artifactId>avro-maven-plugin</artifactId>
<version>1.7.4</version>
<executions>
<execution>
<phase>generate-sources</phase>
<goals>
<goal>schema</goal>
</goals>
<configuration>
<sourceDirectory>${project.basedir}/../</sourceDirectory>
<outputDirectory>${project.basedir}/src/main/java/</outputDirectory>
</configuration>
</execution>
</executions>
</plugin>
Alternatively, Avro jars can be downloaded directly from the Apache Avro™ Releases page. The relevant Avro jars for this guide are avro-1.7.4.jar and avro-mapred-1.7.4.jar, as well as avro-tools-1.7.4.jar for code generation and viewing Avro data files as JSON. In addition, you will need to install Hadoop in order to use MapReduce.
Example: ColorCount
Below is a simple example of a MapReduce that uses Avro. This example can be found in the Avro docs under examples/mr-example/src/main/java/example/ColorCount.java. We’ll go over the specifics of what’s going on in subsequent sections.
package example;
import java.io.IOException;
import org.apache.avro.*;
import org.apache.avro.Schema.Type;
import org.apache.avro.mapred.*;
import org.apache.hadoop.conf.*;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.mapred.*;
import org.apache.hadoop.util.*;
import example.avro.User;
public class ColorCount extends Configured implements Tool {
public static class ColorCountMapper extends AvroMapper<User, Pair<CharSequence, Integer>> {
@Override
public void map(User user, AvroCollector<Pair<CharSequence, Integer>> collector, Reporter reporter)
throws IOException {
CharSequence color = user.getFavoriteColor();
// We need this check because the User.favorite_color field has type ["string", "null"]
if (color == null) {
color = "none";
}
collector.collect(new Pair<CharSequence, Integer>(color, 1));
}
}
public static class ColorCountReducer extends AvroReducer<CharSequence, Integer,
Pair<CharSequence, Integer>> {
@Override
public void reduce(CharSequence key, Iterable<Integer> values,
AvroCollector<Pair<CharSequence, Integer>> collector,
Reporter reporter)
throws IOException {
int sum = 0;
for (Integer value : values) {
sum += value;
}
collector.collect(new Pair<CharSequence, Integer>(key, sum));
}
}
public int run(String[] args) throws Exception {
if (args.length != 2) {
System.err.println("Usage: ColorCount <input path> <output path>");
return -1;
}
JobConf conf = new JobConf(getConf(), ColorCount.class);
conf.setJobName("colorcount");
FileInputFormat.setInputPaths(conf, new Path(args[0]));
FileOutputFormat.setOutputPath(conf, new Path(args[1]));
AvroJob.setMapperClass(conf, ColorCountMapper.class);
AvroJob.setReducerClass(conf, ColorCountReducer.class);
// Note that AvroJob.setInputSchema and AvroJob.setOutputSchema set
// relevant config options such as input/output format, map output
// classes, and output key class.
AvroJob.setInputSchema(conf, User.SCHEMA$);
AvroJob.setOutputSchema(conf, Pair.getPairSchema(Schema.create(Type.STRING),
Schema.create(Type.INT)));
JobClient.runJob(conf);
return 0;
}
public static void main(String[] args) throws Exception {
int res = ToolRunner.run(new Configuration(), new ColorCount(), args);
System.exit(res);
}
}
ColorCount reads in data files containing User records, defined in examples/user.avsc, and counts the number of instances of each favorite color. (This example draws inspiration from the canonical WordCount MapReduce application.) The User schema is defined as follows:
{"namespace": "example.avro",
"type": "record",
"name": "User",
"fields": [
{"name": "name", "type": "string"},
{"name": "favorite_number", "type": ["int", "null"]},
{"name": "favorite_color", "type": ["string", "null"]}
]
}
This schema is compiled into the User class used by ColorCount via the Avro Maven plugin (see examples/mr-example/pom.xml for how this is set up).
ColorCountMapper essentially takes a User as input and extracts the User’s favorite color, emitting the key-value pair <favoriteColor, 1>. ColorCountReducer then adds up how many occurrences of a particular favorite color were emitted, and outputs the result as a Pair record. These Pairs are serialized to an Avro data file.
Running ColorCount
The ColorCount application is provided as a Maven project in the Avro docs under examples/mr-example. To build the project, including the code generation of the User schema, run:
Next, run GenerateData to create an Avro data file, input/users.avro, containing 20 Users with favorite colors chosen randomly from a list:
mvn exec:java -q -Dexec.mainClass=example.GenerateData
Besides creating the data file, GenerateData prints the JSON representations of the Users generated to stdout, for example:
{"name": "user", "favorite_number": null, "favorite_color": "red"}
{"name": "user", "favorite_number": null, "favorite_color": "green"}
{"name": "user", "favorite_number": null, "favorite_color": "purple"}
{"name": "user", "favorite_number": null, "favorite_color": null}
...
Now we’re ready to run ColorCount. We specify our freshly-generated input folder as the input path and output as our output folder (note that MapReduce will not start a job if the output folder already exists):
mvn exec:java -q -Dexec.mainClass=example.ColorCount -Dexec.args="input output"
Once ColorCount completes, checking the contents of the new output directory should yield the following:
$ ls output/
part-00000.avro _SUCCESS
You can check the contents of the generated Avro file using the avro-tools jar:
$ java -jar /path/to/avro-tools-1.7.4.jar tojson output/part-00000.avro
{"value": 3, "key": "blue"}
{"value": 7, "key": "green"}
{"value": 1, "key": "none"}
{"value": 2, "key": "orange"}
{"value": 3, "key": "purple"}
{"value": 2, "key": "red"}
{"value": 2, "key": "yellow"}
Now let’s go over the ColorCount example in detail.
AvroMapper
The easiest way to use Avro data files as input to a MapReduce job is to subclass AvroMapper. An AvroMapper defines a map function that takes an Avro datum as input and outputs a key/value pair represented as a Pair record. In the ColorCount example, ColorCountMapper is an AvroMapper that takes a User as input and outputs a Pair<CharSequence, Integer>>, where the CharSequence key is the user’s favorite color and the Integer value is 1.
public static class ColorCountMapper extends AvroMapper<User, Pair<CharSequence, Integer>> {
@Override
public void map(User user, AvroCollector<Pair<CharSequence, Integer>> collector, Reporter reporter)
throws IOException {
CharSequence color = user.getFavoriteColor();
// We need this check because the User.favorite_color field has type ["string", "null"]
if (color == null) {
color = "none";
}
collector.collect(new Pair<CharSequence, Integer>(color, 1));
}
}
In order to use our AvroMapper, we must call AvroJob.setMapperClass and AvroJob.setInputSchema.
AvroJob.setMapperClass(conf, ColorCountMapper.class);
AvroJob.setInputSchema(conf, User.SCHEMA$);
Note that AvroMapper does not implement the Mapper interface. Under the hood, the specified Avro data files are deserialized into AvroWrappers containing the actual data, which are processed by a Mapper that calls the configured AvroMapper’s map function. AvroJob.setInputSchema sets up the relevant configuration parameters needed to make this happen, thus you should not need to call JobConf.setMapperClass, JobConf.setInputFormat, JobConf.setMapOutputKeyClass, JobConf.setMapOutputValueClass, or JobConf.setOutputKeyComparatorClass.
AvroReducer
Analogously to AvroMapper, an AvroReducer defines a reducer function that takes the key/value types output by an AvroMapper (or any mapper that outputs Pairs) and outputs a key/value pair represented a Pair record. In the ColorCount example, ColorCountReducer is an AvroReducer that takes the CharSequence key representing a favorite color and the Iterable<Integer> representing the counts for that color (they should all be 1 in this example) and adds up the counts.
public static class ColorCountReducer extends AvroReducer<CharSequence, Integer,
Pair<CharSequence, Integer>> {
@Override
public void reduce(CharSequence key, Iterable<Integer> values,
AvroCollector<Pair<CharSequence, Integer>> collector,
Reporter reporter)
throws IOException {
int sum = 0;
for (Integer value : values) {
sum += value;
}
collector.collect(new Pair<CharSequence, Integer>(key, sum));
}
}
In order to use our AvroReducer, we must call AvroJob.setReducerClass and AvroJob.setOutputSchema.
AvroJob.setReducerClass(conf, ColorCountReducer.class);
AvroJob.setOutputSchema(conf, Pair.getPairSchema(Schema.create(Type.STRING),
Schema.create(Type.INT)));
Note that AvroReducer does not implement the Reducer interface. The intermediate Pairs output by the mapper are split into AvroKeys and AvroValues, which are processed by a Reducer that calls the configured AvroReducer’s reduce function. AvroJob.setOutputSchema sets up the relevant configuration parameters needed to make this happen, thus you should not need to call JobConf.setReducerClass, JobConf.setOutputFormat, JobConf.setOutputKeyClass, JobConf.setMapOutputKeyClass, JobConf.setMapOutputValueClass, or JobConf.setOutputKeyComparatorClass.
Learning more
It’s possible to mix AvroMappers and AvroReducers with non-Avro Mappers and Reducers. See the org.apache.avro.mapred documentation for more details. There is also a org.apache.avro.mapreduce package for use with the new MapReduce API (org.apache.hadoop.mapreduce). It’s also possible to implement your own Mappers and Reducers directly using the public classes provided in these libraries. See the AvroWordCount application, found under examples/mr-example/src/main/java/example/AvroWordCount.java in the Avro documentation, for an example of implementing a Reducer that outputs Avro data.
15.6 - IDL Language
Introduction
This document defines Avro IDL, a higher-level language for authoring Avro schemata. Before reading this document, you should have familiarity with the concepts of schemata and protocols, as well as the various primitive and complex types available in Avro.
Overview
Purpose
The aim of the Avro IDL language is to enable developers to author schemata in a way that feels more similar to common programming languages like Java, C++, or Python. Additionally, the Avro IDL language may feel more familiar for those users who have previously used the interface description languages (IDLs) in other frameworks like Thrift, Protocol Buffers, or CORBA.
Usage
Each Avro IDL file defines a single Avro Protocol, and thus generates as its output a JSON-format Avro Protocol file with extension .avpr.
To convert a .avdl file into a .avpr file, it may be processed by the idl tool. For example:
$ java -jar avroj-tools.jar idl src/test/idl/input/namespaces.avdl /tmp/namespaces.avpr
$ head /tmp/namespaces.avpr
{
"protocol" : "TestNamespace",
"namespace" : "avro.test.protocol",
The idl tool can also process input to and from stdin and stdout. See idl --help for full usage information.
A Maven plugin is also provided to compile .avdl files. To use it, add something like the following to your pom.xml:
<build>
<plugins>
<plugin>
<groupId>org.apache.avro</groupId>
<artifactId>avro-maven-plugin</artifactId>
<executions>
<execution>
<goals>
<goal>idl-protocol</goal>
</goals>
</execution>
</executions>
</plugin>
</plugins>
</build>
Defining a Protocol in Avro IDL
An Avro IDL file consists of exactly one protocol definition. The minimal protocol is defined by the following code:
This is equivalent to (and generates) the following JSON protocol definition:
{
"protocol" : "MyProtocol",
"types" : [ ],
"messages" : {
}
}
The namespace of the protocol may be changed using the @namespace annotation:
@namespace("mynamespace")
protocol MyProtocol {
}
This notation is used throughout Avro IDL as a way of specifying properties for the annotated element, as will be described later in this document.
Protocols in Avro IDL can contain the following items:
- Imports of external protocol and schema files.
- Definitions of named schemata, including records, errors, enums, and fixeds.
- Definitions of RPC messages
Imports
Files may be imported in one of three formats:
-
An IDL file may be imported with a statement like:
import idl "foo.avdl";
-
A JSON protocol file may be imported with a statement like:
import protocol "foo.avpr";
-
A JSON schema file may be imported with a statement like:
import schema "foo.avsc";
Messages and types in the imported file are added to this file’s protocol.
Imported file names are resolved relative to the current IDL file.
Defining an Enumeration
Enums are defined in Avro IDL using a syntax similar to C or Java:
enum Suit {
SPADES, DIAMONDS, CLUBS, HEARTS
}
Note that, unlike the JSON format, anonymous enums cannot be defined.
Defining a Fixed Length Field
Fixed fields are defined using the following syntax:
fixed MD5(16);
This example defines a fixed-length type called MD5 which contains 16 bytes.
Defining Records and Errors
Records are defined in Avro IDL using a syntax similar to a struct definition in C:
record Employee {
string name;
boolean active = true;
long salary;
}
The above example defines a record with the name “Employee” with three fields.
To define an error, simply use the keyword error instead of record. For example:
error Kaboom {
string explanation;
int result_code = -1;
}
Each field in a record or error consists of a type and a name, optional property annotations and an optional default value.
A type reference in Avro IDL must be one of:
- A primitive type
- A logical type
- A named schema defined prior to this usage in the same Protocol
- A complex type (array, map, or union)
Primitive Types
The primitive types supported by Avro IDL are the same as those supported by Avro’s JSON format. This list includes int, long, string, boolean, float, double, null, and bytes.
References to Named Schemata
If a named schema has already been defined in the same Avro IDL file, it may be referenced by name as if it were a primitive type:
record Card {
Suit suit; // refers to the enum Card defined above
int number;
}
Default Values
Default values for fields may be optionally specified by using an equals sign after the field name followed by a JSON expression indicating the default value. This JSON is interpreted as described in the spec.
Complex Types
Arrays
Array types are written in a manner that will seem familiar to C++ or Java programmers. An array of any type t is denoted array<t>. For example, an array of strings is denoted array<string>, and a multidimensional array of Foo records would be array<array<Foo>>.
Maps
Map types are written similarly to array types. An array that contains values of type t is written map<t>. As in the JSON schema format, all maps contain string-type keys.
Unions
Union types are denoted as union { typeA, typeB, typeC, … }. For example, this record contains a string field that is optional (unioned with null):
record RecordWithUnion {
union { null, string } optionalString;
}
Note that the same restrictions apply to Avro IDL unions as apply to unions defined in the JSON format; namely, a record may not contain multiple elements of the same type.
Defining RPC Messages
The syntax to define an RPC message within a Avro IDL protocol is similar to the syntax for a method declaration within a C header file or a Java interface. To define an RPC message add which takes two arguments named foo and bar, returning an int, simply include the following definition within the protocol:
int add(int foo, int bar = 0);
Message arguments, like record fields, may specify default values.
To define a message with no response, you may use the alias void, equivalent to the Avro null type:
void logMessage(string message);
If you have previously defined an error type within the same protocol, you may declare that a message can throw this error using the syntax:
void goKaboom() throws Kaboom;
To define a one-way message, use the keyword oneway after the parameter list, for example:
void fireAndForget(string message) oneway;
Other Language Features
All Java-style comments are supported within a Avro IDL file. Any text following // on a line is ignored, as is any text between /* and */, possibly spanning multiple lines.
Comments that begin with /** are used as the documentation string for the type or field definition that follows the comment.
Escaping Identifiers
Occasionally, one will need to use a reserved language keyword as an identifier. In order to do so, backticks (`) may be used to escape the identifier. For example, to define a message with the literal name error, you may write:
This syntax is allowed anywhere an identifier is expected.
Annotations for Ordering and Namespaces
Java-style annotations may be used to add additional properties to types and fields throughout Avro IDL.
For example, to specify the sort order of a field within a record, one may use the @order annotation before the field name as follows:
record MyRecord {
string @order("ascending") myAscendingSortField;
string @order("descending") myDescendingField;
string @order("ignore") myIgnoredField;
}
A field’s type may also be preceded by annotations, e.g.:
record MyRecord {
@java-class("java.util.ArrayList") array<string> myStrings;
}
Similarly, a @namespace annotation may be used to modify the namespace when defining a named schema. For example:
@namespace("org.apache.avro.firstNamespace")
protocol MyProto {
@namespace("org.apache.avro.someOtherNamespace")
record Foo {}
record Bar {}
}
will define a protocol in the firstNamespace namespace. The record Foo will be defined in someOtherNamespace and Bar will be defined in firstNamespace as it inherits its default from its container.
Type and field aliases are specified with the @aliases annotation as follows:
@aliases(["org.old.OldRecord", "org.ancient.AncientRecord"])
record MyRecord {
string @aliases(["oldField", "ancientField"]) myNewField;
}
Some annotations like those listed above are handled specially. All other annotations are added as properties to the protocol, message, schema or field.
Complete Example
The following is a complete example of a Avro IDL file that shows most of the above features:
/**
* An example protocol in Avro IDL
*/
@namespace("org.apache.avro.test")
protocol Simple {
@aliases(["org.foo.KindOf"])
enum Kind {
FOO,
BAR, // the bar enum value
BAZ
}
fixed MD5(16);
record TestRecord {
@order("ignore")
string name;
@order("descending")
Kind kind;
MD5 hash;
union { MD5, null} @aliases(["hash"]) nullableHash;
array<long> arrayOfLongs;
}
error TestError {
string message;
}
string hello(string greeting);
TestRecord echo(TestRecord `record`);
int add(int arg1, int arg2);
bytes echoBytes(bytes data);
void `error`() throws TestError;
void ping() oneway;
}
Additional examples may be found in the Avro source tree under the src/test/idl/input directory.
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
15.7 - SASL profile
Introduction
SASL (RFC 2222) provides a framework for authentication and security of network protocols. Each protocol that uses SASL is meant to define a SASL profile. This document provides a SASL profile for connection-based Avro RPC.
Overview
SASL negotiation proceeds as a series of message interactions over a connection between a client and server using a selected SASL mechanism. The client starts this negotiation by sending its chosen mechanism name with an initial (possibly empty) message. Negotiation proceeds with the exchange of messages until either side indicates success or failure. The content of the messages is mechanism-specific. If the negotiation succeeds, then the session can proceed over the connection, otherwise it must be abandoned.
Some mechanisms continue to process session data after negotiation (e.g., encrypting it), while some specify that further session data is transmitted unmodified.
Negotiation
Commands
Avro SASL negotiation uses four one-byte commands.
- 0: START Used in a client’s initial message.
- 1: CONTINUE Used while negotiation is ongoing.
- 2: FAIL Terminates negotiation unsuccessfully.
- 3: COMPLETE Terminates negotiation successfully.
The format of a START message is:
| 0 | 4-byte mechanism name length | mechanism name | 4-byte payload length | payload data |
The format of a CONTINUE message is:
| 1 | 4-byte payload length | payload data |
The format of a FAIL message is:
| 2 | 4-byte message length | UTF-8 message |
The format of a COMPLETE message is:
| 3 | 4-byte payload length | payload data |
Process
Negotiation is initiated by a client sending a START command containing the client’s chosen mechanism name and any mechanism-specific payload data.
The server and client then interchange some number (possibly zero) of CONTINUE messages. Each message contains payload data that is processed by the security mechanism to generate the next message.
Once either the client or server send a FAIL message then negotiation has failed. UTF-8-encoded text is included in the failure message. Once either a FAIL message has been sent or received, or any other error occurs in the negotiation, further communication on this connection must cease.
Once either the client or server send a COMPLETE message then negotiation has completed successfully. Session data may now be transmitted over the connection until it is closed by either side.
Session Data
If no SASL QOP (quality of protection) is negotiated, then all subsequent writes to/reads over this connection are written/read unmodified. In particular, messages use Avro framing, and are of the form:
| 4-byte frame length | frame data | ... | 4 zero bytes |
If a SASL QOP is negotiated, then it must be used by the connection for all subsequent messages. This is done by wrapping each non-empty frame written using the security mechanism and unwrapping each non-empty frame read. The length written in each non-empty frame is the length of the wrapped data. Complete frames must be passed to the security mechanism for unwrapping. Unwrapped data is then passed to the application as the content of the frame.
If at any point processing fails due to wrapping, unwrapping or framing errors, then all further communication on this connection must cease.
Anonymous Mechanism
The SASL anonymous mechanism (RFC 2245) is quite simple to implement. In particular, an initial anonymous request may be prefixed by the following static sequence:
| 0 | 0009 | ANONYMOUS | 0000 |
If a server uses the anonymous mechanism, it should check that the mechanism name in the start message prefixing the first request received is ‘ANONYMOUS’, then simply prefix its initial response with a COMPLETE message of:
| 3 | 0000 |
If an anonymous server recieves some other mechanism name, then it may respond with a FAIL message as simple as:
| 2 | 0000 |
Note that the anonymous mechanism need add no additional round-trip messages between client and server. The START message can be piggybacked on the initial request and the COMPLETE or FAIL message can be piggybacked on the initial response.
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
15.8 - Wiki
The Wiki page can be found here.
15.9 - FAQ
What is Avro?
Avro is a data serialization system.
Avro provides:
- Rich data structures.
- A compact, fast, binary data format.
- A container file, to store persistent data.
- Remote procedure call (RPC).
- Simple integration with dynamic languages. Code generation is not required to read or write data files nor to use or implement RPC protocols. Code generation as an optional optimization, only worth implementing for statically typed languages.
How can I get started quickly with Avro?
Check out the Quick Start Guide.
How do I statically compile a schema or protocol into generated code?
In Java
- Add the avro jar, the jackson-mapper-asl.jar and jackson-core-asl.jar to your CLASSPATH.
- Run java org.apache.avro.specific.SpecificCompiler .
This appears to be out of date, the SpecificCompiler requires two arguments, presumably an input and and output file, but it isn’t clear that this does.
Or use the Schema or Protocol Ant tasks. Avro’s build.xml provides examples of how these are used.
Lastly, you can also use the “avro-tools” jar which ships with an Avro release. Just use the “compile (schema|protocol)” command.
How are Strings represented in Java?
They use org.apache.avro.util.Utf8, not java.lang.String.
More generally, how do Avro types map to Java types?
The mappings are documented in the package javadoc for generic, specific and reflect API.
From Doug Cutting:
HDFS splits files into blocks, and mapreduce runs a map task for each block. When the task starts, it needs to be able to seek into the file to the start of the block process through the block’s end. If the file were, e.g., a gzip file, this would not be possible, since gzip files must be decompressed from the start. One cannot seek into the middle of a gzip file and start decompressing. So Hadoop’s SequenceFile places a marker periodically (~64k) in the file at record and compression boundaries, where processing can be sensibly started. Then, when a map task starts processing an HDFS block, it finds the first marker after the block’s start and continues through the first marker in the next block of the file. This requires a bit of non-local access (~0.1%). Avro’s data file uses the same method as SequenceFile.
Why isn’t every value in Avro nullable?
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 of every value stored and
corresponding computational costs to create this bit on write and
interpret it on read. These costs are wasted when values may not in
fact be null, as is the case in many datasets. In Avro such costs are
only paid when values may actually be null.
Also, allowing values to be null is a well-known source of errors. In
Avro, a value declared as non-null will always be non-null and programs
need not test for null values when processing it nor will they ever fail
for lack of such tests.
Tony Hoare calls his invention of null references his “Billion Dollar
Mistake”.
http://qconlondon.com/london-2009/presentation/Null+References:+The+Billion+Dollar+Mistake
Also note that in some programming languages not all values are permitted to be null. For example, in Java, values of type boolean, byte, short, char, int, float, long, and double may not be null.
How can I serialize directly to/from a byte array?
As pointed out in the specification, Avro data should always be stored with its schema. The Avro provided classes DataFileWriter, DataFileReader, and DataFileStream all ensure this by serializing the Schema in a container header. In some special cases, such as when implementing a new storage system or writing unit tests, you may need to write and read directly with the bare Avro serialized values. The following examples use code generated for Java from an Avro Schema, specifically the User example from the Quickstart guide.
Serializing to a byte array
This example takes a User object and returns a newly allocated byte array with the Avro serialization of that user.
ByteArrayOutputStream out = new ByteArrayOutputStream();
BinaryEncoder encoder = EncoderFactory.get().binaryEncoder(out, null);
DatumWriter<User> writer = new SpecificDatumWriter<User>(User.getClassSchema());
writer.write(user, encoder);
encoder.flush();
out.close();
byte[] serializedBytes = out.toByteArray();
Deserializing from a byte array
This example takes a byte array containing the Avro serialization of a user and returns a User object.SpecificDatumReader reader = new SpecificDatumReader(User.getClassSchema());
Decoder decoder = DecoderFactory.get().binaryDecoder(bytes, null);
User user = reader.read(null, decoder);
Note: If you are serializing or deserializing in a loop or as a method, you should be reusing objects, readers and/or writers. Check the JavaDoc to see how to reuse the objects.
16 - Apache Avro™ 1.7.3 Documentation
Introduction
Apache Avro™ is a data serialization system.
Avro provides:
- Rich data structures.
- A compact, fast, binary data format.
- A container file, to store persistent data.
- Remote procedure call (RPC).
- Simple integration with dynamic languages. Code generation is not required to read or write data files nor to use or implement RPC protocols. Code generation as an optional optimization, only worth implementing for statically typed languages.
Schemas
Avro relies on schemas. When Avro data is read, the schema used when writing it is always present. This permits each datum to be written with no per-value overheads, making serialization both fast and small. This also facilitates use with dynamic, scripting languages, since data, together with its schema, is fully self-describing.
When Avro data is stored in a file, its schema is stored with it, so that files may be processed later by any program. If the program reading the data expects a different schema this can be easily resolved, since both schemas are present.
When Avro is used in RPC, the client and server exchange schemas in the connection handshake. (This can be optimized so that, for most calls, no schemas are actually transmitted.) Since both client and server both have the other’s full schema, correspondence between same named fields, missing fields, extra fields, etc. can all be easily resolved.
Avro schemas are defined with JSON . This facilitates implementation in languages that already have JSON libraries.
Comparison with other systems
Avro provides functionality similar to systems such as Thrift, Protocol Buffers, etc. Avro differs from these systems in the following fundamental aspects.
- Dynamic typing: Avro does not require that code be generated. Data is always accompanied by a schema that permits full processing of that data without code generation, static datatypes, etc. This facilitates construction of generic data-processing systems and languages.
- Untagged data: Since the schema is present when data is read, considerably less type information need be encoded with data, resulting in smaller serialization size.
- No manually-assigned field IDs: When a schema changes, both the old and new schema are always present when processing data, so differences may be resolved symbolically, using field names.
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
16.1 - Getting Started (Java)
This is a short guide for getting started with Apache Avro™ using Java. This guide only covers using Avro for data serialization; see Patrick Hunt’s Avro RPC Quick Start for a good introduction to using Avro for RPC.
Download
Avro implementations for C, C++, C#, Java, PHP, Python, and Ruby can be downloaded from the Apache Avro™ Releases page. This guide uses Avro 1.7.3, the latest version at the time of writing. For the examples in this guide, download avro-1.7.3.jar and avro-tools-1.7.3.jar. The Avro Java implementation also depends on the Jackson JSON library. From the Jackson download page, download the core-asl and mapper-asl jars. Add avro-1.7.3.jar and the Jackson jars to your project’s classpath (avro-tools will be used for code generation).
Alternatively, if you are using Maven, add the following dependency to your POM:
<dependency>
<groupId>org.apache.avro</groupId>
<artifactId>avro</artifactId>
<version>1.7.3</version>
</dependency>
As well as the Avro Maven plugin (for performing code generation):
<plugin>
<groupId>org.apache.avro</groupId>
<artifactId>avro-maven-plugin</artifactId>
<version>1.7.3</version>
<executions>
<execution>
<phase>generate-sources</phase>
<goals>
<goal>schema</goal>
</goals>
<configuration>
<sourceDirectory>${project.basedir}/src/main/avro/</sourceDirectory>
<outputDirectory>${project.basedir}/src/main/java/</outputDirectory>
</configuration>
</execution>
</executions>
</plugin>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-compiler-plugin</artifactId>
<configuration>
<source>1.6</source>
<target>1.6</target>
</configuration>
</plugin>
You may also build the required Avro jars from source. Building Avro is beyond the scope of this guide; see the Build Documentation page in the wiki for more information.
Defining a schema
Avro schemas are defined using JSON. Schemas are composed of primitive types (null, boolean, int, long, float, double, bytes, and string) and complex types (record, enum, array, map, union, and fixed). You can learn more about Avro schemas and types from the specification, but for now let’s start with a simple schema example, user.avsc:
{"namespace": "example.avro",
"type": "record",
"name": "User",
"fields": [
{"name": "name", "type": "string"},
{"name": "favorite_number", "type": ["int", "null"]},
{"name": "favorite_color", "type": ["string", "null"]}
]
}
This schema defines a record representing a hypothetical user. (Note that a schema file can only contain a single schema definition.) At minimum, a record definition must include its type (“type”: “record”), a name (“name”: “User”), and fields, in this case name, favorite_number, and favorite_color. We also define a namespace (“namespace”: “example.avro”), which together with the name attribute defines the “full name” of the schema (example.avro.User in this case).
Fields are defined via an array of objects, each of which defines a name and type (other attributes are optional, see the record specification for more details). The type attribute of a field is another schema object, which can be either a primitive or complex type. For example, the name field of our User schema is the primitive type string, whereas the favorite_number and favorite_color fields are both unions, represented by JSON arrays. unions are a complex type that can be any of the types listed in the array; e.g., favorite_number can either be an int or null, essentially making it an optional field.
Serializing and deserializing with code generation
Compiling the schema
Code generation allows us to automatically create classes based on our previously-defined schema. Once we have defined the relevant classes, there is no need to use the schema directly in our programs. We use the avro-tools jar to generate code as follows:
java -jar /path/to/avro-tools-1.7.3.jar compile schema <schema file> <destination>
This will generate the appropriate source files in a package based on the schema’s namespace in the provided destination folder. For instance, to generate a User class in package example.avro from the schema defined above, run
java -jar /path/to/avro-tools-1.7.3.jar compile schema user.avsc .
Note that if you using the Avro Maven plugin, there is no need to manually invoke the schema compiler; the plugin automatically performs code generation on any .avsc files present in the configured source directory.
Creating Users
Now that we’ve completed the code generation, let’s create some Users, serialize them to a data file on disk, and then read back the file and deserialize the User objects.
First let’s create some Users and set their fields.
User user1 = new User();
user1.setName("Alyssa");
user1.setFavoriteNumber(256);
// Leave favorite color null
// Alternate constructor
User user2 = new User("Ben", 7, "red");
// Construct via builder
User user3 = User.newBuilder()
.setName("Charlie")
.setFavoriteColor("blue")
.setFavoriteNumber(null)
.build();
As shown in this example, Avro objects can be created either by invoking a constructor directly or by using a builder. Unlike constructors, builders will automatically set any default values specified in the schema. Additionally, builders validate the data as it set, whereas objects constructed directly will not cause an error until the object is serialized. However, using constructors directly generally offers better performance, as builders create a copy of the datastructure before it is written.
Note that we do not set user1’s favorite color. Since that record is of type [“string”, “null”], we can either set it to a string or leave it null; it is essentially optional. Similarly, we set user3’s favorite number to null (using a builder requires setting all fields, even if they are null).
Serializing
Now let’s serialize our Users to disk.
// Serialize user1 and user2 to disk
File file = new File("users.avro");
DatumWriter<User> userDatumWriter = new SpecificDatumWriter<User>(User.class);
DataFileWriter<User> dataFileWriter = new DataFileWriter<User>(userDatumWriter);
dataFileWriter.create(user1.getSchema(), new File("users.avro"));
dataFileWriter.append(user1);
dataFileWriter.append(user2);
dataFileWriter.append(user3);
dataFileWriter.close();
We create a DatumWriter, which converts Java objects into an in-memory serialized format. The SpecificDatumWriter class is used with generated classes and extracts the schema from the specified generated type.
Next we create a DataFileWriter, which writes the serialized records, as well as the schema, to the file specified in the dataFileWriter.create call. We write our users to the file via calls to the dataFileWriter.append method. When we are done writing, we close the data file.
Deserializing
Finally, let’s deserialize the data file we just created.
// Deserialize Users from disk
DatumReader<User> userDatumReader = new SpecificDatumReader<User>(User.class);
DataFileReader<User> dataFileReader = new DataFileReader<User>(file, userDatumReader);
User user = null;
while (dataFileReader.hasNext()) {
// Reuse user object by passing it to next(). This saves us from
// allocating and garbage collecting many objects for files with
// many items.
user = dataFileReader.next(user);
System.out.println(user);
}
This snippet will output:
{"name": "Alyssa", "favorite_number": 256, "favorite_color": null}
{"name": "Ben", "favorite_number": 7, "favorite_color": "red"}
{"name": "Charlie", "favorite_number": null, "favorite_color": "blue"}
Deserializing is very similar to serializing. We create a SpecificDatumReader, analogous to the SpecificDatumWriter we used in serialization, which converts in-memory serialized items into instances of our generated class, in this case User. We pass the DatumReader and the previously created File to a DataFileReader, analogous to the DataFileWriter, which reads the data file on disk.
Next we use the DataFileReader to iterate through the serialized Users and print the deserialized object to stdout. Note how we perform the iteration: we create a single User object which we store the current deserialized user in, and pass this record object to every call of dataFileReader.next. This is a performance optimization that allows the DataFileReader to reuse the same User object rather than allocating a new User for every iteration, which can be very expensive in terms of object allocation and garbage collection if we deserialize a large data file. While this technique is the standard way to iterate through a data file, it’s also possible to use for (User user : dataFileReader) if performance is not a concern.
Compiling and running the example code
This example code is included as a Maven project in the examples/java-example directory in the Avro docs. From this directory, execute the following commands to build and run the example:
$ mvn compile # includes code generation via Avro Maven plugin
$ mvn -q exec:java -Dexec.mainClass=example.SpecificMain
Serializing and deserializing without code generation
Data in Avro is always stored with its corresponding schema, meaning we can always read a serialized item regardless of whether we know the schema ahead of time. This allows us to perform serialization and deserialization without code generation.
Let’s go over the same example as in the previous section, but without using code generation: we’ll create some users, serialize them to a data file on disk, and then read back the file and deserialize the users objects.
Creating users
First, we use a Parser to read our schema definition and create a Schema object.
Schema schema = new Parser().parse(new File("user.avsc"));
Using this schema, let’s create some users.
GenericRecord user1 = new GenericData.Record(schema);
user1.put("name", "Alyssa");
user1.put("favorite_number", 256);
// Leave favorite color null
GenericRecord user2 = new GenericData.Record(schema);
user2.put("name", "Ben");
user2.put("favorite_number", 7);
user2.put("favorite_color", "red");
Since we’re not using code generation, we use GenericRecords to represent users. GenericRecord uses the schema to verify that we only specify valid fields. If we try to set a non-existent field (e.g., user1.put(“favorite_animal”, “cat”)), we’ll get an AvroRuntimeException when we run the program.
Note that we do not set user1’s favorite color. Since that record is of type [“string”, “null”], we can either set it to a string or leave it null; it is essentially optional.
Serializing
Now that we’ve created our user objects, serializing and deserializing them is almost identical to the example above which uses code generation. The main difference is that we use generic instead of specific readers and writers.
First we’ll serialize our users to a data file on disk.
// Serialize user1 and user2 to disk
File file = new File("users.avro");
DatumWriter<GenericRecord> datumWriter = new GenericDatumWriter<GenericRecord>(schema);
DataFileWriter<GenericRecord> dataFileWriter = new DataFileWriter<GenericRecord>(datumWriter);
dataFileWriter.create(schema, file);
dataFileWriter.append(user1);
dataFileWriter.append(user2);
dataFileWriter.close();
We create a DatumWriter, which converts Java objects into an in-memory serialized format. Since we are not using code generation, we create a GenericDatumWriter. It requires the schema both to determine how to write the GenericRecords and to verify that all non-nullable fields are present.
As in the code generation example, we also create a DataFileWriter, which writes the serialized records, as well as the schema, to the file specified in the dataFileWriter.create call. We write our users to the file via calls to the dataFileWriter.append method. When we are done writing, we close the data file.
Deserializing
Finally, we’ll deserialize the data file we just created.
// Deserialize users from disk
DatumReader<GenericRecord> datumReader = new GenericDatumReader<GenericRecord>(schema);
DataFileReader<GenericRecord> dataFileReader = new DataFileReader<GenericRecord>(file, datumReader);
GenericRecord user = null;
while (dataFileReader.hasNext()) {
// Reuse user object by passing it to next(). This saves us from
// allocating and garbage collecting many objects for files with
// many items.
user = dataFileReader.next(user);
System.out.println(user);
This outputs:
{"name": "Alyssa", "favorite_number": 256, "favorite_color": null}
{"name": "Ben", "favorite_number": 7, "favorite_color": "red"}
Deserializing is very similar to serializing. We create a GenericDatumReader, analogous to the GenericDatumWriter we used in serialization, which converts in-memory serialized items into GenericRecords. We pass the DatumReader and the previously created File to a DataFileReader, analogous to the DataFileWriter, which reads the data file on disk.
Next, we use the DataFileReader to iterate through the serialized users and print the deserialized object to stdout. Note how we perform the iteration: we create a single GenericRecord object which we store the current deserialized user in, and pass this record object to every call of dataFileReader.next. This is a performance optimization that allows the DataFileReader to reuse the same record object rather than allocating a new GenericRecord for every iteration, which can be very expensive in terms of object allocation and garbage collection if we deserialize a large data file. While this technique is the standard way to iterate through a data file, it’s also possible to use for (GenericRecord user : dataFileReader) if performance is not a concern.
Compiling and running the example code
This example code is included as a Maven project in the examples/java-example directory in the Avro docs. From this directory, execute the following commands to build and run the example:
$ mvn compile
$ mvn -q exec:java -Dexec.mainClass=example.GenericMain
16.2 - Getting Started (Python)
This is a short guide for getting started with Apache Avro™ using Python. This guide only covers using Avro for data serialization; see Patrick Hunt’s Avro RPC Quick Start for a good introduction to using Avro for RPC.
Download
Avro implementations for C, C++, C#, Java, PHP, Python, and Ruby can be downloaded from the Apache Avro™ Releases page. This guide uses Avro 1.7.3, the latest version at the time of writing. Download and unzip avro-1.7.3.tar.gz, and install via python setup.py (this will probably require root privileges). Ensure that you can import avro from a Python prompt.
$ tar xvf avro-1.7.3.tar.gz
$ cd avro-1.7.3
$ sudo python setup.py install
$ python
>>> import avro # should not raise ImportError
Alternatively, you may build the Avro Python library from source. From your the root Avro directory, run the commands
$ cd lang/py/
$ ant
$ sudo python setup.py install
$ python
>>> import avro # should not raise ImportError
Defining a schema
Avro schemas are defined using JSON. Schemas are composed of primitive types (null, boolean, int, long, float, double, bytes, and string) and complex types (record, enum, array, map, union, and fixed). You can learn more about Avro schemas and types from the specification, but for now let’s start with a simple schema example, user.avsc:
{"namespace": "example.avro",
"type": "record",
"name": "User",
"fields": [
{"name": "name", "type": "string"},
{"name": "favorite_number", "type": ["int", "null"]},
{"name": "favorite_color", "type": ["string", "null"]}
]
}
This schema defines a record representing a hypothetical user. (Note that a schema file can only contain a single schema definition.) At minimum, a record definition must include its type (“type”: “record”), a name (“name”: “User”), and fields, in this case name, favorite_number, and favorite_color. We also define a namespace (“namespace”: “example.avro”), which together with the name attribute defines the “full name” of the schema (example.avro.User in this case).
Fields are defined via an array of objects, each of which defines a name and type (other attributes are optional, see the record specification for more details). The type attribute of a field is another schema object, which can be either a primitive or complex type. For example, the name field of our User schema is the primitive type string, whereas the favorite_number and favorite_color fields are both unions, represented by JSON arrays. unions are a complex type that can be any of the types listed in the array; e.g., favorite_number can either be an int or null, essentially making it an optional field.
Serializing and deserializing without code generation
Data in Avro is always stored with its corresponding schema, meaning we can always read a serialized item, regardless of whether we know the schema ahead of time. This allows us to perform serialization and deserialization without code generation. Note that the Avro Python library does not support code generation.
Try running the following code snippet, which serializes two users to a data file on disk, and then reads back and deserializes the data file:
import avro.schema
from avro.datafile import DataFileReader, DataFileWriter
from avro.io import DatumReader, DatumWriter
schema = avro.schema.parse(open("user.avsc").read())
writer = DataFileWriter(open("users.avro", "w"), DatumWriter(), schema)
writer.append({"name": "Alyssa", "favorite_number": 256})
writer.append({"name": "Ben", "favorite_number": 7, "favorite_color": "red"})
writer.close()
reader = DataFileReader(open("users.avro", "r"), DatumReader())
for user in reader:
print user
reader.close()
This outputs:
{u'favorite_color': None, u'favorite_number': 256, u'name': u'Alyssa'}
{u'favorite_color': u'red', u'favorite_number': 7, u'name': u'Ben'}
Let’s take a closer look at what’s going on here.
schema = avro.schema.parse(open("user.avsc").read())
avro.schema.parse takes a string containing a JSON schema definition as input and outputs a avro.schema.Schema object (specifically a subclass of Schema, in this case RecordSchema). We’re passing in the contents of our user.avsc schema file here.
writer = DataFileWriter(open("users.avro", "w"), DatumWriter(), schema)
We create a DataFileWriter, which we’ll use to write serialized items to a data file on disk. The DataFileWriter constructor takes three arguments:
- The file we’ll serialize to
- A DatumWriter, which is responsible for actually serializing the items to Avro’s binary format (DatumWriters can be used separately from DataFileWriters, e.g., to perform IPC with Avro TODO: is this true??).
- The schema we’re using. The DataFileWriter needs the schema both to write the schema to the data file, and to verify that the items we write are valid items and write the appropriate fields.
writer.append({"name": "Alyssa", "favorite_number": 256})
writer.append({"name": "Ben", "favorite_number": 7, "favorite_color": "red"})
We use DataFileWriter.append to add items to our data file. Avro records are represented as Python dicts. Since the field favorite_color has type [“int”, “null”], we are not required to specify this field, as shown in the first append. Were we to omit the required name field, an exception would be raised. Any extra entries not corresponding to a field are present in the dict are ignored.
reader = DataFileReader(open("users.avro", "r"), DatumReader())
We open the file again, this time for reading back from disk. We use a DataFileReader and DatumReader analagous to the DataFileWriter and DatumWriter above.
for user in reader:
print user
The DataFileReader is an iterator that returns dicts corresponding to the serialized items.
16.3 - Specification
Introduction
This document defines Apache Avro. It is intended to be the authoritative specification. Implementations of Avro must adhere to this document.
Schema Declaration
A Schema is represented in JSON by one of:
- A JSON string, naming a defined type.
- A JSON object, of the form:
{"type": "typeName" ...attributes...}
where typeName is either a primitive or derived type name, as defined below. Attributes not defined in this document are permitted as metadata, but must not affect the format of serialized data.
- A JSON array, representing a union of embedded types.
Primitive Types
The set of primitive type names is:
- null: no value
- boolean: a binary value
- int: 32-bit signed integer
- long: 64-bit signed integer
- float: single precision (32-bit) IEEE 754 floating-point number
- double: double precision (64-bit) IEEE 754 floating-point number
- bytes: sequence of 8-bit unsigned bytes
- string: unicode character sequence
Primitive types have no specified attributes.
Primitive type names are also defined type names. Thus, for example, the schema “string” is equivalent to:
Complex Types
Avro supports six kinds of complex types: records, enums, arrays, maps, unions and fixed.
Records
Records use the type name “record” and support three attributes:
- name: a JSON string providing the name of the record (required).
- namespace, a JSON string that qualifies the name;
- doc: a JSON string providing documentation to the user of this schema (optional).
- aliases: a JSON array of strings, providing alternate names for this record (optional).
- fields: a JSON array, listing fields (required). Each field is a JSON object with the following attributes:
- name: a JSON string providing the name of the field (required), and
- doc: a JSON string describing this field for users (optional).
- type: A JSON object defining a schema, or a JSON string naming a record definition (required).
- default: A default value for this field, used when reading instances that lack this field (optional). 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. Default values for bytes and fixed fields are JSON strings, where Unicode code points 0-255 are mapped to unsigned 8-bit byte values 0-255.
field default values
| avro type |
json type |
example |
| null |
null |
null |
| boolean |
boolean |
true |
| int,long |
integer |
1 |
| float,double |
number |
1.1 |
| bytes |
string |
"\u00FF" |
| string |
string |
"foo" |
| record |
object |
{"a": 1} |
| enum |
string |
"FOO" |
| array |
array |
[1] |
| map |
object |
{"a": 1} |
| fixed |
string |
"\u00ff" |
- order: specifies how this field impacts sort ordering of this record (optional). Valid values are “ascending” (the default), “descending”, or “ignore”. For more details on how this is used, see the sort order section below.
- aliases: a JSON array of strings, providing alternate names for this field (optional).
For example, a linked-list of 64-bit values may be defined with:
{
"type": "record",
"name": "LongList",
"aliases": ["LinkedLongs"], // old name for this
"fields" : [
{"name": "value", "type": "long"}, // each element has a long
{"name": "next", "type": ["LongList", "null"]} // optional next element
]
}
Enums
Enums use the type name “enum” and support the following attributes:
- name: a JSON string providing the name of the enum (required).
- namespace, a JSON string that qualifies the name ;
- aliases: a JSON array of strings, providing alternate names for this enum (optional).
- doc: a JSON string providing documentation to the user of this schema (optional).
- symbols: a JSON array, listing symbols, as JSON strings (required). All symbols in an enum must be unique; duplicates are prohibited.
For example, playing card suits might be defined with:
{
"type": "enum",
"name": "Suit",
"symbols" : ["SPADES", "HEARTS", "DIAMONDS", "CLUBS"]
}
Arrays
Arrays use the type name “array” and support a single attribute:
- items: the schema of the array’s items.
For example, an array of strings is declared with:
{
"type": "array",
"items" : "string"
}
Maps
Maps use the type name “map” and support one attribute:
- values: the schema of the map’s values.
Map keys are assumed to be strings.
For example, a map from string to long is declared with:
{
"type": "map",
"values" : "long"
}
Unions
Unions, as mentioned above, are represented using JSON arrays. For example, ["string", "null"] declares a schema which may be either a string or null.
Unions may not contain more than one schema with the same type, except for the named types record, fixed and enum. For example, unions containing two array types or two map types are not permitted, but two types with different names are permitted. (Names permit efficient resolution when reading and writing unions.)
Unions may not immediately contain other unions.
Fixed
Fixed uses the type name “fixed” and supports two attributes:
- name: a string naming this fixed (required).
- namespace, a string that qualifies the name;
- aliases: a JSON array of strings, providing alternate names for this enum (optional).
- size: an integer, specifying the number of bytes per value (required).
For example, 16-byte quantity may be declared with:
{"type": "fixed", "size": 16, "name": "md5"}
Names
Record, enums and fixed are named types. Each has a fullname that is composed of two parts; a name and a namespace. Equality of names is defined on the fullname.
The name portion of a fullname, record field names, and enum symbols must:
- start with [A-Za-z_]
- subsequently contain only [A-Za-z0-9_]
A namespace is a dot-separated sequence of such names. Equality of names (including field names and enum symbols) as well as fullnames is case-sensitive.
In record, enum and fixed definitions, the fullname is determined in one of the following ways:
A name and namespace are both specified. For example, one might use “name”: “X”, “namespace”: “org.foo” to indicate the fullname org.foo.X.
-
A fullname is specified. If the name specified contains a dot, then it is assumed to be a fullname, and any namespace also specified is ignored. For example, use “name”: “org.foo.X” to indicate the fullname org.foo.X.
-
A name only is specified, i.e., a name that contains no dots. In this case the namespace is taken from the most tightly enclosing schema or protocol. For example, if “name”: “X” is specified, and this occurs within a field of the record definition of org.foo.Y, then the fullname is org.foo.X.
References to previously defined names are as in the latter two cases above: if they contain a dot they are a fullname, if they do not contain a dot, the namespace is the namespace of the enclosing definition.
Primitive type names have no namespace and their names may not be defined in any namespace.
A schema or protocol may not contain multiple definitions of a fullname. Further, a name must be defined before it is used (“before” in the depth-first, left-to-right traversal of the JSON parse tree, where the types attribute of a protocol is always deemed to come “before” the messages attribute.)
Aliases
Named types and fields may have aliases. An implementation may optionally use aliases to map a writer’s schema to the reader’s. This facilitates both schema evolution as well as processing disparate datasets.
Aliases function by re-writing the writer’s schema using aliases from the reader’s schema. For example, if the writer’s schema was named “Foo” and the reader’s schema is named “Bar” and has an alias of “Foo”, then the implementation would act as though “Foo” were named “Bar” when reading. Similarly, if data was written as a record with a field named “x” and is read as a record with a field named “y” with alias “x”, then the implementation would act as though “x” were named “y” when reading.
A type alias may be specified either as a fully namespace-qualified, or relative to the namespace of the name it is an alias for. For example, if a type named “a.b” has aliases of “c” and “x.y”, then the fully qualified names of its aliases are “a.c” and “x.y”.
Data Serialization
Avro data is always serialized with its schema. Files that store Avro data should always also include the schema for that data in the same file. Avro-based remote procedure call (RPC) systems must also guarantee that remote recipients of data have a copy of the schema used to write that data.
Because the schema used to write data is always available when the data is read, Avro data itself is not tagged with type information. The schema is required to parse data.
In general, both serialization and deserialization proceed as a depth-first, left-to-right traversal of the schema, serializing primitive types as they are encountered.
Encodings
Avro specifies two serialization encodings: binary and JSON. Most applications will use the binary encoding, as it is smaller and faster. But, for debugging and web-based applications, the JSON encoding may sometimes be appropriate.
Binary Encoding
Primitive Types
Primitive types are encoded in binary as follows:
- null is written as zero bytes.
- a boolean is written as a single byte whose value is either 0 (false) or 1 (true).
- int and long values are written using variable-length zig-zag coding. Some examples:
| value |
hex |
| 0 |
00 |
| -1 |
01 |
| 1 |
02 |
| -2 |
03 |
| 2 |
04 |
| … |
|
| -64 |
7f |
| 64 |
80 01 |
| … |
|
- a float is written as 4 bytes. The float is converted into a 32-bit integer using a method equivalent to Java’s floatToIntBits and then encoded in little-endian format.
- a double is written as 8 bytes. The double is converted into a 64-bit integer using a method equivalent to Java’s doubleToLongBits and then encoded in little-endian format.
- bytes are encoded as a long followed by that many bytes of data.
- a string is encoded as a long followed by that many bytes of UTF-8 encoded character data.
For example, the three-character string “foo” would be encoded as the long value 3 (encoded as hex 06) followed by the UTF-8 encoding of ‘f’, ‘o’, and ‘o’ (the hex bytes 66 6f 6f):
06 66 6f 6f
Complex Types
Complex types are encoded in binary as follows:
Records
A record is encoded by encoding the values of its fields in the order that they are declared. In other words, a record is encoded as just the concatenation of the encodings of its fields. Field values are encoded per their schema.
For example, the record schema
{
"type": "record",
"name": "test",
"fields" : [
{"name": "a", "type": "long"},
{"name": "b", "type": "string"}
]
}
An instance of this record whose a field has value 27 (encoded as hex 36) and whose b field has value “foo” (encoded as hex bytes 06 66 6f 6f), would be encoded simply as the concatenation of these, namely the hex byte sequence:
36 06 66 6f 6f
Enums
An enum is encoded by a int, representing the zero-based position of the symbol in the schema.
For example, consider the enum:
{"type": "enum", "name": "Foo", "symbols": ["A", "B", "C", "D"] }
This would be encoded by an int between zero and three, with zero indicating “A”, and 3 indicating “D”.
Arrays
Arrays are encoded as a series of blocks. Each block consists of a long count value, followed by that many array items. A block with count zero indicates the end of the array. Each item is encoded per the array’s item schema.
If a block’s count is negative, its absolute value is used, and the count is followed immediately by a long block size indicating the number of bytes in the block. This block size permits fast skipping through data, e.g., when projecting a record to a subset of its fields.
For example, the array schema
{"type": "array", "items": "long"}
an array containing the items 3 and 27 could be encoded as the long value 2 (encoded as hex 04) followed by long values 3 and 27 (encoded as hex 06 36) terminated by zero:
04 06 36 00
The blocked representation permits one to read and write arrays larger than can be buffered in memory, since one can start writing items without knowing the full length of the array.
Maps
Maps are encoded as a series of blocks. Each block consists of a long count value, followed by that many key/value pairs. A block with count zero indicates the end of the map. Each item is encoded per the map’s value schema.
If a block’s count is negative, its absolute value is used, and the count is followed immediately by a long block size indicating the number of bytes in the block. This block size permits fast skipping through data, e.g., when projecting a record to a subset of its fields.
The blocked representation permits one to read and write maps larger than can be buffered in memory, since one can start writing items without knowing the full length of the map.
Unions
A union is encoded by first writing a long value indicating the zero-based position within the union of the schema of its value. The value is then encoded per the indicated schema within the union.
For example, the union schema ["string","null"] would encode:
-
null as the integer 1 (the index of “null” in the union, encoded as hex 02):
02
-
the string “a” as zero (the index of “string” in the union), followed by the serialized string:
02 02 61
Fixed
Fixed instances are encoded using the number of bytes declared in the schema.
JSON Encoding
Except for unions, the JSON encoding is the same as is used to encode field default values.
The value of a union is encoded in JSON as follows:
- if its type is null, then it is encoded as a JSON null;
- otherwise it is encoded as a JSON object with one name/value pair whose name is the type’s name and whose value is the recursively encoded value. For Avro’s named types (record, fixed or enum) the user-specified name is used, for other types the type name is used.
For example, the union schema ["null","string","Foo"], where Foo is a record name, would encode:
- null as null;
- the string “a” as
{"string": "a"} and
- a Foo instance as
{"Foo": {...}}, where {...} indicates the JSON encoding of a Foo instance.
Note that a schema is still required to correctly process JSON-encoded data. For example, the JSON encoding does not distinguish between int and long, float and double, records and maps, enums and strings, etc.
Sort Order
Avro defines a standard sort order for data. This permits data written by one system to be efficiently sorted by another system. This can be an important optimization, as sort order comparisons are sometimes the most frequent per-object operation. Note also that Avro binary-encoded data can be efficiently ordered without deserializing it to objects.
Data items may only be compared if they have identical schemas. Pairwise comparisons are implemented recursively with a depth-first, left-to-right traversal of the schema. The first mismatch encountered determines the order of the items.
Two items with the same schema are compared according to the following rules.
- null data is always equal.
- boolean data is ordered with false before true.
- int, long, float and double data is ordered by ascending numeric value.
- bytes and fixed data are compared lexicographically by unsigned 8-bit values.
- string data is compared lexicographically by Unicode code point. Note that since UTF-8 is used as the binary encoding for strings, sorting of bytes and string binary data is identical.
- array data is compared lexicographically by element.
- enum data is ordered by the symbol’s position in the enum schema. For example, an enum whose symbols are
["z", "a"] would sort “z” values before “a” values.
- union data is first ordered by the branch within the union, and, within that, by the type of the branch. For example, an
["int", "string"] union would order all int values before all string values, with the ints and strings themselves ordered as defined above.
- record data is ordered lexicographically by field. If a field specifies that its order is:
- “ascending”, then the order of its values is unaltered.
- “descending”, then the order of its values is reversed.
- “ignore”, then its values are ignored when sorting.
- map data may not be compared. It is an error to attempt to compare data containing maps unless those maps are in an
"order":"ignore" record field.
Object Container Files
Avro includes a simple object container file format. A file has a schema, and all objects stored in the file must be written according to that schema, using binary encoding. Objects are stored in blocks that may be compressed. Syncronization markers are used between blocks to permit efficient splitting of files for MapReduce processing.
Files may include arbitrary user-specified metadata.
A file consists of:
- A file header, followed by
- one or more file data blocks.
A file header consists of:
- Four bytes, ASCII ‘O’, ‘b’, ‘j’, followed by 1.
- file metadata, including the schema.
- The 16-byte, randomly-generated sync marker for this file.
File metadata consists of:
A long indicating the number of metadata key/value pairs.
For each pair, a string key and bytes value.
All metadata properties that start with “avro.” are reserved. The following file metadata properties are currently used:
- avro.schema contains the schema of objects stored in the file, as JSON data (required).
- avro.codec the name of the compression codec used to compress blocks, as a string. Implementations are required to support the following codecs: “null” and “deflate”. If codec is absent, it is assumed to be “null”. The codecs are described with more detail below.
A file header is thus described by the following schema:
{"type": "record", "name": "org.apache.avro.file.Header",
"fields" : [
{"name": "magic", "type": {"type": "fixed", "name": "Magic", "size": 4}},
{"name": "meta", "type": {"type": "map", "values": "bytes"}},
{"name": "sync", "type": {"type": "fixed", "name": "Sync", "size": 16}}
]
}
A file data block consists of:
- A long indicating the count of objects in this block.
- A long indicating the size in bytes of the serialized objects in the current block, after any codec is applied
- The serialized objects. If a codec is specified, this is compressed by that codec.
- The file’s 16-byte sync marker.
Thus, each block’s binary data can be efficiently extracted or skipped without deserializing the contents. The combination of block size, object counts, and sync markers enable detection of corrupt blocks and help ensure data integrity.
Required Codecs
null
The “null” codec simply passes through data uncompressed.
deflate
The “deflate” codec writes the data block using the deflate algorithm as specified in RFC 1951, and typically implemented using the zlib library. Note that this format (unlike the “zlib format” in RFC 1950) does not have a checksum.
Optional Codecs
snappy
The “snappy” codec uses Google’s Snappy compression library. Each compressed block is followed by the 4-byte, big-endian CRC32 checksum of the uncompressed data in the block.
Protocol Declaration
Avro protocols describe RPC interfaces. Like schemas, they are defined with JSON text.
A protocol is a JSON object with the following attributes:
- protocol, a string, the name of the protocol (required);
- namespace, an optional string that qualifies the name;
- doc, an optional string describing this protocol;
- types, an optional list of definitions of named types (records, enums, fixed and errors). An error definition is just like a record definition except it uses “error” instead of “record”. Note that forward references to named types are not permitted.
- messages, an optional JSON object whose keys are message names and whose values are objects whose attributes are described below. No two messages may have the same name.
The name and namespace qualification rules defined for schema objects apply to protocols as well.
Messages
A message has attributes:
- a doc, an optional description of the message,
- a request, a list of named, typed parameter schemas (this has the same form as the fields of a record declaration);
- a response schema;
- an optional union of declared error schemas. The effective union has “string” prepended to the declared union, to permit transmission of undeclared “system” errors. For example, if the declared error union is
["AccessError"], then the effective union is ["string", "AccessError"]. When no errors are declared, the effective error union is ["string"]. Errors are serialized using the effective union; however, a protocol’s JSON declaration contains only the declared union.
- an optional one-way boolean parameter.
A request parameter list is processed equivalently to an anonymous record. Since record field lists may vary between reader and writer, request parameters may also differ between the caller and responder, and such differences are resolved in the same manner as record field differences.
The one-way parameter may only be true when the response type is "null" and no errors are listed.
Sample Protocol
For example, one may define a simple HelloWorld protocol with:
{
"namespace": "com.acme",
"protocol": "HelloWorld",
"doc": "Protocol Greetings",
"types": [
{"name": "Greeting", "type": "record", "fields": [
{"name": "message", "type": "string"}]},
{"name": "Curse", "type": "error", "fields": [
{"name": "message", "type": "string"}]}
],
"messages": {
"hello": {
"doc": "Say hello.",
"request": [{"name": "greeting", "type": "Greeting" }],
"response": "Greeting",
"errors": ["Curse"]
}
}
}
Message Transport
Messages may be transmitted via different transport mechanisms.
To the transport, a message is an opaque byte sequence.
A transport is a system that supports:
- transmission of request messages
- receipt of corresponding response messages
Servers may send a response message back to the client corresponding to a request message. The mechanism of correspondance is transport-specific. For example, in HTTP it is implicit, since HTTP directly supports requests and responses. But a transport that multiplexes many client threads over a single socket would need to tag messages with unique identifiers.
Transports may be either stateless or stateful. In a stateless transport, messaging assumes no established connection state, while stateful transports establish connections that may be used for multiple messages. This distinction is discussed further in the handshake section below.
HTTP as Transport
When HTTP is used as a transport, each Avro message exchange is an HTTP request/response pair. All messages of an Avro protocol should share a single URL at an HTTP server. Other protocols may also use that URL. Both normal and error Avro response messages should use the 200 (OK) response code. The chunked encoding may be used for requests and responses, but, regardless the Avro request and response are the entire content of an HTTP request and response. The HTTP Content-Type of requests and responses should be specified as “avro/binary”. Requests should be made using the POST method.
HTTP is used by Avro as a stateless transport.
Message Framing
Avro messages are framed as a list of buffers.
Framing is a layer between messages and the transport. It exists to optimize certain operations.
The format of framed message data is:
- a series of buffers, where each buffer consists of:
- a four-byte, big-endian buffer length, followed by
- that many bytes of buffer data.
- A message is always terminated by a zero-lenghted buffer.
Framing is transparent to request and response message formats (described below). Any message may be presented as a single or multiple buffers.
Framing can permit readers to more efficiently get different buffers from different sources and for writers to more efficiently store different buffers to different destinations. In particular, it can reduce the number of times large binary objects are copied. For example, if an RPC parameter consists of a megabyte of file data, that data can be copied directly to a socket from a file descriptor, and, on the other end, it could be written directly to a file descriptor, never entering user space.
A simple, recommended, framing policy is for writers to create a new segment whenever a single binary object is written that is larger than a normal output buffer. Small objects are then appended in buffers, while larger objects are written as their own buffers. When a reader then tries to read a large object the runtime can hand it an entire buffer directly, without having to copy it.
Handshake
The purpose of the handshake is to ensure that the client and the server have each other’s protocol definition, so that the client can correctly deserialize responses, and the server can correctly deserialize requests. Both clients and servers should maintain a cache of recently seen protocols, so that, in most cases, a handshake will be completed without extra round-trip network exchanges or the transmission of full protocol text.
RPC requests and responses may not be processed until a handshake has been completed. With a stateless transport, all requests and responses are prefixed by handshakes. With a stateful transport, handshakes are only attached to requests and responses until a successful handshake response has been returned over a connection. After this, request and response payloads are sent without handshakes for the lifetime of that connection.
The handshake process uses the following record schemas:
{
"type": "record",
"name": "HandshakeRequest", "namespace":"org.apache.avro.ipc",
"fields": [
{"name": "clientHash",
"type": {"type": "fixed", "name": "MD5", "size": 16}},
{"name": "clientProtocol", "type": ["null", "string"]},
{"name": "serverHash", "type": "MD5"},
{"name": "meta", "type": ["null", {"type": "map", "values": "bytes"}]}
]
}
{
"type": "record",
"name": "HandshakeResponse", "namespace": "org.apache.avro.ipc",
"fields": [
{"name": "match",
"type": {"type": "enum", "name": "HandshakeMatch",
"symbols": ["BOTH", "CLIENT", "NONE"]}},
{"name": "serverProtocol",
"type": ["null", "string"]},
{"name": "serverHash",
"type": ["null", {"type": "fixed", "name": "MD5", "size": 16}]},
{"name": "meta",
"type": ["null", {"type": "map", "values": "bytes"}]}
]
}
- A client first prefixes each request with a
HandshakeRequest containing just the hash of its protocol and of the server’s protocol (clientHash!=null, clientProtocol=null, serverHash!=null), where the hashes are 128-bit MD5 hashes of the JSON protocol text. If a client has never connected to a given server, it sends its hash as a guess of the server’s hash, otherwise it sends the hash that it previously obtained from this server.
The server responds with a HandshakeResponse containing one of:
-
match=BOTH, serverProtocol=null, serverHash=null if the client sent the valid hash of the server’s protocol and the server knows what protocol corresponds to the client’s hash. In this case, the request is complete and the response data immediately follows the HandshakeResponse.
-
match=CLIENT, serverProtocol!=null, serverHash!=null if the server has previously seen the client’s protocol, but the client sent an incorrect hash of the server’s protocol. The request is complete and the response data immediately follows the HandshakeResponse. The client must use the returned protocol to process the response and should also cache that protocol and its hash for future interactions with this server.
-
match=NONE if the server has not previously seen the client’s protocol. The serverHash and serverProtocol may also be non-null if the server’s protocol hash was incorrect.
In this case the client must then re-submit its request with its protocol text (clientHash!=null, clientProtocol!=null, serverHash!=null) and the server should respond with a successful match (match=BOTH, serverProtocol=null, serverHash=null) as above.
The meta field is reserved for future handshake enhancements.
A call consists of a request message paired with its resulting response or error message. Requests and responses contain extensible metadata, and both kinds of messages are framed as described above.
The format of a call request is:
- request metadata, a map with values of type bytes
- the message name, an Avro string, followed by
- the message parameters. Parameters are serialized according to the message’s request declaration.
When a message is declared one-way and a stateful connection has been established by a successful handshake response, no response data is sent. Otherwise the format of the call response is:
- response metadata, a map with values of type bytes
- a one-byte error flag boolean, followed by either:
- if the error flag is false, the message response, serialized per the message’s response schema.
- if the error flag is true, the error, serialized per the message’s effective error union schema.
Schema Resolution
A reader of Avro data, whether from an RPC or a file, can always parse that data because its schema is provided. But that schema may not be exactly the schema that was expected. For example, if the data was written with a different version of the software than it is read, then records may have had fields added or removed. This section specifies how such schema differences should be resolved.
We refer to the schema used to write the data as the writer’s schema, and the schema that the application expects the reader’s schema. Differences between these should be resolved as follows:
- It is an error if the two schemas do not match.
To match, one of the following must hold:
-
both schemas are arrays whose item types match
-
both schemas are maps whose value types match
-
both schemas are enums whose names match
-
both schemas are fixed whose sizes and names match
-
both schemas are records with the same name
-
either schema is a union
-
both schemas have same primitive type
-
the writer’s schema may be promoted to the reader’s as follows:
- int is promotable to long, float, or double
- long is promotable to float or double
- float is promotable to double
- string is promotable to bytes
- bytes is promotable to string
-
if both are records:
- the ordering of fields may be different: fields are matched by name.
- schemas for fields with the same name in both records are resolved recursively.
- if the writer’s record contains a field with a name not present in the reader’s record, the writer’s value for that field is ignored.
- if the reader’s record schema has a field that contains a default value, and writer’s schema does not have a field with the same name, then the reader should use the default value from its field.
- if the reader’s record schema has a field with no default value, and writer’s schema does not have a field with the same name, an error is signalled.
-
if both are enums:
if the writer’s symbol is not present in the reader’s enum, then an error is signalled.
This resolution algorithm is applied recursively to the reader’s and writer’s array item schemas.
This resolution algorithm is applied recursively to the reader’s and writer’s value schemas.
The first schema in the reader’s union that matches the selected writer’s union schema is recursively resolved against it. if none match, an error is signalled.
- if reader’s is a union, but writer’s is not
The first schema in the reader’s union that matches the writer’s schema is recursively resolved against it. If none match, an error is signalled.
- if writer’s is a union, but reader’s is not
If the reader’s schema matches the selected writer’s schema, it is recursively resolved against it. If they do not match, an error is signalled.
A schema’s doc fields are ignored for the purposes of schema resolution. Hence, the doc portion of a schema may be dropped at serialization.
One of the defining characteristics of Avro is that a reader is assumed to have the “same” schema used by the writer of the data the reader is reading. This assumption leads to a data format that’s compact and also amenable to many forms of schema evolution. However, the specification so far has not defined what it means for the reader to have the “same” schema as the writer. Does the schema need to be textually identical? Well, clearly adding or removing some whitespace to a JSON expression does not change its meaning. At the same time, reordering the fields of records clearly does change the meaning. So what does it mean for a reader to have “the same” schema as a writer?
Parsing Canonical Form is a transformation of a writer’s schema that let’s us define what it means for two schemas to be “the same” for the purpose of reading data written against the schema. It is called Parsing Canonical Form because the transformations strip away parts of the schema, like “doc” attributes, that are irrelevant to readers trying to parse incoming data. It is called Canonical Form because the transformations normalize the JSON text (such as the order of attributes) in a way that eliminates unimportant differences between schemas. If the Parsing Canonical Forms of two different schemas are textually equal, then those schemas are “the same” as far as any reader is concerned, i.e., there is no serialized data that would allow a reader to distinguish data generated by a writer using one of the original schemas from data generated by a writing using the other original schema. (We sketch a proof of this property in a companion document.)
The next subsection specifies the transformations that define Parsing Canonical Form. But with a well-defined canonical form, it can be convenient to go one step further, transforming these canonical forms into simple integers (“fingerprints”) that can be used to uniquely identify schemas. The subsection after next recommends some standard practices for generating such fingerprints.
Assuming an input schema (in JSON form) that’s already UTF-8 text for a valid Avro schema (including all quotes as required by JSON), the following transformations will produce its Parsing Canonical Form:
- [PRIMITIVES] Convert primitive schemas to their simple form (e.g., int instead of
{"type":"int"}).
- [FULLNAMES] Replace short names with fullnames, using applicable namespaces to do so. Then eliminate namespace attributes, which are now redundant.
- [STRIP] Keep only attributes that are relevant to parsing data, which are: type, name, fields, symbols, items, values, size. Strip all others (e.g., doc and aliases).
- [ORDER] Order the appearance of fields of JSON objects as follows: name, type, fields, symbols, items, values, size. For example, if an object has type, name, and size fields, then the name field should appear first, followed by the type and then the size fields.
- [STRINGS] For all JSON string literals in the schema text, replace any escaped characters (e.g., \uXXXX escapes) with their UTF-8 equivalents.
- [INTEGERS] Eliminate quotes around and any leading zeros in front of JSON integer literals (which appear in the size attributes of fixed schemas).
- [WHITESPACE] Eliminate all whitespace in JSON outside of string literals.
Schema Fingerprints
“[A] fingerprinting algorithm is a procedure that maps an arbitrarily large data item (such as a computer file) to a much shorter bit string, its fingerprint, that uniquely identifies the original data for all practical purposes” (quoted from Wikipedia). In the Avro context, fingerprints of Parsing Canonical Form can be useful in a number of applications; for example, to cache encoder and decoder objects, to tag data items with a short substitute for the writer’s full schema, and to quickly negotiate common-case schemas between readers and writers.
In designing fingerprinting algorithms, there is a fundamental trade-off between the length of the fingerprint and the probability of collisions. To help application designers find appropriate points within this trade-off space, while encouraging interoperability and ease of implementation, we recommend using one of the following three algorithms when fingerprinting Avro schemas:
- When applications can tolerate longer fingerprints, we recommend using the SHA-256 digest algorithm to generate 256-bit fingerprints of Parsing Canonical Forms. Most languages today have SHA-256 implementations in their libraries.
- At the opposite extreme, the smallest fingerprint we recommend is a 64-bit Rabin fingerprint. Below, we provide pseudo-code for this algorithm that can be easily translated into any programming language. 64-bit fingerprints should guarantee uniqueness for schema caches of up to a million entries (for such a cache, the chance of a collision is 3E-8). We don’t recommend shorter fingerprints, as the chances of collisions is too great (for example, with 32-bit fingerprints, a cache with as few as 100,000 schemas has a 50% chance of having a collision).
- Between these two extremes, we recommend using the MD5 message digest to generate 128-bit fingerprints. These make sense only where very large numbers of schemas are being manipulated (tens of millions); otherwise, 64-bit fingerprints should be sufficient. As with SHA-256, MD5 implementations are found in most libraries today.
These fingerprints are not meant to provide any security guarantees, even the longer SHA-256-based ones. Most Avro applications should be surrounded by security measures that prevent attackers from writing random data and otherwise interfering with the consumers of schemas. We recommend that these surrounding mechanisms be used to prevent collision and pre-image attacks (i.e., “forgery”) on schema fingerprints, rather than relying on the security properties of the fingerprints themselves.
Rabin fingerprints are cyclic redundancy checks computed using irreducible polynomials. In the style of the Appendix of RFC 1952 (pg 10), which defines the CRC-32 algorithm, here’s our definition of the 64-bit AVRO fingerprinting algorithm:
long fingerprint64(byte[] buf) {
if (FP_TABLE == null) initFPTable();
long fp = EMPTY;
for (int i = 0; i < buf.length; i++)
fp = (fp >>> 8) ^ FP_TABLE[(int)(fp ^ buf[i]) & 0xff];
return fp;
}
static long EMPTY = 0xc15d213aa4d7a795L;
static long[] FP_TABLE = null;
void initFPTable() {
FP_TABLE = new long[256];
for (int i = 0; i < 256; i++) {
long fp = i;
for (int j = 0; j < 8; j++)
fp = (fp >>> 1) ^ (EMPTY & -(fp & 1L));
FP_TABLE[i] = fp;
}
}
Readers interested in the mathematics behind this algorithm may want to read this book chapter. (Unlike RFC-1952 and the book chapter, we prepend a single one bit to messages. We do this because CRCs ignore leading zero bits, which can be problematic. Our code prepends a one-bit by initializing fingerprints using EMPTY, rather than initializing using zero as in RFC-1952 and the book chapter.)
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
16.4 - Trevni
Version 0.1
DRAFT
This document is the authoritative specification of a file format. Its intent is to permit compatible, independent implementations that read and/or write files in this format.
Introduction
Data sets are often described as a table composed of rows and columns. Each record in the dataset is considered a row, with each field of the record occupying a different column. Writing records to a file one-by-one as they are created results in a row-major format, like Hadoop’s SequenceFile or Avro data files.
In many cases higher query performance may be achieved if the data is instead organized in a column-major format, where multiple values of a given column are stored adjacently. This document defines such a column-major file format for datasets.
To permit scalable, distributed query evaluation, datasets are partitioned into row groups, containing distinct collections of rows. Each row group is organized in column-major order, while row groups form a row-major partitioning of the entire dataset.
Rationale
Goals
The format is meant satisfy the following goals:
Maximize the size of row groups. Disc drives are used most efficiently when sequentially accessing data. Consider a drive that takes 10ms to seek and transfers at 100MB/second. If a 10-column dataset whose values are all the same size is split into 10MB row groups, then accessing a single column will require a sequence of seek+1MB reads, for a cost of 20ms/MB processed. If the same dataset is split into 100MB row groups then this drops to 11ms/MB processed. This effect is exaggerated for datasets with larger numbers of columns and with columns whose values are smaller than average. So we’d prefer row groups that are 100MB or greater.
Permit random access within a row group. Some queries will first examine one column, and, only when certain relatively rare criteria are met, examine other columns. Rather than iterating through selected columns of the row-group in parallel, one might iterate through one column and randomly access another. This is called support for WHERE clauses, after the SQL operator of that name.
Minimize the number of files per dataset. HDFS is a primary intended deployment platform for these files. The HDFS Namenode requires memory for each file in the filesystem, thus for a format to be HDFS-friendly it should strive to require the minimum number of distinct files.
Support co-location of columns within row-groups. Row groups are the unit of parallel operation on a column dataset. For efficient file i/o, the entirety of a row-group should ideally reside on the host that is evaluating the query in order to avoid network latencies and bottlenecks.
Data integrity. The format should permit applications to detect data corruption. Many file systems may prevent corruption, but files may be moved between filesystems and be subject to corruption at points in that process. It is best if the data in a file can be validated independently.
Extensibility. The format should permit applications to store additional annotations about a datasets in the files, such as type information, origin, etc. Some environments may have metadata stores for such information, but not all do, and files might be moved among systems with different metadata systems. The ability to keep such information within the file simplifies the coordination of such information.
Minimal overhead. The column format should not make datasets appreciably larger. Storage is a primary cost and a choice to use this format should not require additional storage.
Primary format. The column format should be usable as a primary format for datasets, not as an auxiliary, accelerated format. Applications that process a dataset in row-major order should be able to easily consume column files and applications that produce datasets in row-major order should be able to easily generate column files.
Design
To meet these goals we propose the following design.
Each row group is a separate file. All values of a column in a file are written contiguously. This maximizes the row group size, optimizing performance when querying few and small columns.
Each file occupies a single HDFS block. A larger than normal block size may be specified, e.g., ~1GB instead of the typical ~100MB. This guarantees co-location and eliminates network use when query processing can be co-located with the file. This also moderates the memory impact on the HDFS Namenode since no small files are written.
Each column in a file is written as a sequence of ~64kB compressed blocks. The sequence is prefixed by a table describing all of the blocks in the column to permit random access within the column.
Application-specific metadata may be added at the file, column, and block levels.
Checksums are included with each block, providing data integrity.
Discussion
The use of a single block per file achieves the same effect as the custom block placement policy described in the CIF paper, but while still permitting HDFS rebalancing and not increasing the number of files in the namespace.
This section formally describes the proposed column file format.
Data Model
We assume a simple data model, where a record is a set of named fields, and the value of each field is a sequence of untyped bytes. A type system may be layered on top of this, as specified in the Type Mapping section below.
Primitive Values
We define the following primitive value types:
- Signed 64-bit long values are written using a variable-length zig-zag coding, where the high-order bit in each byte determines whether subsequent bytes are present. For example:
decimal value hex bytes
| decimal value |
hex bytes |
|
| 0 |
00 |
|
| -1 |
01 |
|
| 1 |
02 |
|
| … |
|
|
| -64 |
7f |
|
| 64 |
80 01 |
|
| … |
|
|
- bytes are encoded as a long followed by that many bytes of data.
- a string is encoded as a long followed by that many bytes of UTF-8 encoded character data.
For example, the three-character string “foo” would be encoded as the long value 3 (encoded as hex 06) followed by the UTF-8 encoding of ‘f’, ‘o’, and ‘o’ (the hex bytes 66 6f 6f): 06 66 6f 6f
Type Names
The following type names are used to describe column values:
- null, requires zero bytes. Sometimes used in array columns.
- boolean, one bit, packed into bytes, little-endian;
- int, like long, but restricted to 32-bit signed values
- long 64-bit signed values, represented as above
- fixed32 32-bit values stored as four bytes, little-endian.
- fixed64 64-bit values stored as eight bytes, little-endian.
- float 32-bit IEEE floating point value, little-endian
- double 64-bit IEEE floating point value, little-endian
- string as above
- bytes as above, may be used to encapsulate more complex objects
Type names are represented as strings (UTF-8 encoded, length-prefixed).
Metadata consists of:
- A long indicating the number of metadata key/value pairs.
- For each pair, a string key and bytes value.
All metadata properties that start with “trevni.” are reserved.
The following file metadata properties are defined:
- trevni.codec the name of the default compression codec used to compress blocks, as a string. Implementations are required to support the “null” codec. Optional. If absent, it is assumed to be “null”. Codecs are described in more detail below.
- trevni.checksum the name of the checksum algorithm used in this file, as a string. Implementations are required to support the “crc-32” checksum. Optional. If absent, it is assumed to be “null”. Checksums are described in more detail below.
The following column metadata properties are defined:
- trevni.codec the name of the compression codec used to compress the blocks of this column, as a string. Implementations are required to support the “null” codec. Optional. If absent, it is assumed to be “null”. Codecs are described in more detail below.
- trevni.name the name of the column, as a string. Required.
- trevni.type the type of data in the column. One of the type names above. Required.
- trevni.values if present, indicates that the initial value of each block in this column will be stored in the block’s descriptor. Not permitted for array columns or columns that specify a parent.
- trevni.array if present, indicates that each row in this column contains a sequence of values of the named type rather than just a single value. An integer length precedes each sequence of values indicating the count of values in the sequence.
- trevni.parent if present, the name of an array column whose lengths are also used by this column. Thus values of this column are sequences but no lengths are stored in this column.
For example, consider the following row, as JSON, where all values are primitive types, but one has multiple values.
{"id"=566, "date"=23423234234
"from"="foo@bar.com",
"to"=["bar@baz.com", "bang@foo.com"],
"content"="Hi!"}
The columns for this might be specified as:
name=id type=int
name=date type=long
name=from type=string
name=to type=string array=true
name=content type=string
If a row contains an array of records, e.g. “received” in the following:
{"id"=566, "date"=23423234234
"from"="foo@bar.com",
"to"=["bar@baz.com", "bang@foo.com"],
"content"="Hi!"
"received"=[{"date"=234234234234, "host"="192.168.0.0.1"},
{"date"=234234545645, "host"="192.168.0.0.2"}]
}
Then one can define a parent column followed by a column for each field in the record, adding the following columns:
name=received type=null array=true
name=date type=long parent=received
name=host type=string parent=received
If an array value itself contains an array, e.g. the “sigs” below:
{"id"=566, "date"=23423234234
"from"="foo@bar.com",
"to"=["bar@baz.com", "bang@foo.com"],
"content"="Hi!"
"received"=[{"date"=234234234234, "host"="192.168.0.0.1",
"sigs"=[{"algo"="weak", "value"="0af345de"}]},
{"date"=234234545645, "host"="192.168.0.0.2",
"sigs"=[]}]
}
Then a parent column may be defined that itself has a parent column.
name=sigs type=null array=true parent=received
name=algo type=string parent=sigs
name=value type=string parent=sigs
No block metadata properties are currently defined.
A file consists of:
- A file header, followed by
- one or more columns.
A file header consists of:
- Four bytes, ASCII ‘T’, ‘r’, ‘v’, followed by 1.
- a fixed64 indicating the number of rows in the file
- a fixed32 indicating the number of columns in the file
- file metadata.
- for each column, its column metadata
- for each column, its starting position in the file as a fixed64.
A column consists of:
- A fixed32 indicating the number of blocks in this column.
- For each block, a block descriptor
- One or more blocks.
A block descriptor consists of:
- A fixed32 indicating the number of rows in the block
- A fixed32 indicating the size in bytes of the block before the codec is applied (excluding checksum).
- A fixed32 indicating the size in bytes of the block after the codec is applied (excluding checksum).
- If this column’s metadata declares it to include values, the first value in the column, serialized according to this column’s type.
A block consists of:
- The serialized column values. If a column is an array column then value sequences are preceded by their length, as an int. If a codec is specified, the values and lengths are compressed by that codec.
- The checksum, as determined by the file metadata.
Codecs
null
The “null” codec simply passes data through uncompressed.
deflate
The “deflate” codec writes the data block using the deflate algorithm as specified in RFC 1951.
snappy
The “snappy” codec uses Google’s Snappy compression library.
Checksum algorithms
null
The “null” checksum contains zero bytes.
crc-32
Each “crc-32” checksum contains the four bytes of an ISO 3309 CRC-32 checksum of the uncompressed block data as a fixed32.
Type Mappings
We define a standard mapping for how types defined in various serialization systems are represented in a column file. Records from these systems are shredded into columns. When records are nested, a depth-first recursive walk can assign a separate column for each primitive value.
Avro
Protocol Buffers
Thrift
Implementation Notes
Some possible techniques for writing column files include:
- Use a standard ~100MB block, buffer in memory up to the block size, then flush the file directly to HDFS. A single reduce task might create multiple output files. The namenode requires memory proportional to the number of names and blocks*replication. This would increase the number of names but not blocks, so this should still be much better than a file per column.
- Spill each column to a separate local, temporary file then, when the file is closed, append these files, writing a single file to HDFS whose block size is set to be that of the entire file. This would be a bit slower than and may have trouble when the local disk is full, but it would better use HDFS namespace and further reduce seeks when processing columns whose values are small.
- Use a separate mapreduce job to convert row-major files to column-major. The map output would output a by (row#, column#, value) tuple, partitioned by row# but sorted by column# then row#. The reducer could directly write the column file. But the column file format would need to be changed to write counts, descriptors, etc. at the end of files rather than at the front.
(1) is the simplest to implement and most implementations should start with it.
References
CIF Column-Oriented Storage Techniques for MapReduce, Floratou, Patel, Shekita, & Tata, VLDB 2011.
DREMEL Dremel: Interactive Analysis of Web-Scale Datasets, Melnik, Gubarev, Long, Romer, Shivakumar, & Tolton, VLDB 2010.
16.5 - IDL Language
Introduction
This document defines Avro IDL, a higher-level language for authoring Avro schemata. Before reading this document, you should have familiarity with the concepts of schemata and protocols, as well as the various primitive and complex types available in Avro.
Overview
Purpose
The aim of the Avro IDL language is to enable developers to author schemata in a way that feels more similar to common programming languages like Java, C++, or Python. Additionally, the Avro IDL language may feel more familiar for those users who have previously used the interface description languages (IDLs) in other frameworks like Thrift, Protocol Buffers, or CORBA.
Usage
Each Avro IDL file defines a single Avro Protocol, and thus generates as its output a JSON-format Avro Protocol file with extension .avpr.
To convert a .avdl file into a .avpr file, it may be processed by the idl tool. For example:
$ java -jar avroj-tools.jar idl src/test/idl/input/namespaces.avdl /tmp/namespaces.avpr
$ head /tmp/namespaces.avpr
{
"protocol" : "TestNamespace",
"namespace" : "avro.test.protocol",
The idl tool can also process input to and from stdin and stdout. See idl --help for full usage information.
A Maven plugin is also provided to compile .avdl files. To use it, add something like the following to your pom.xml:
<build>
<plugins>
<plugin>
<groupId>org.apache.avro</groupId>
<artifactId>avro-maven-plugin</artifactId>
<executions>
<execution>
<goals>
<goal>idl-protocol</goal>
</goals>
</execution>
</executions>
</plugin>
</plugins>
</build>
Defining a Protocol in Avro IDL
An Avro IDL file consists of exactly one protocol definition. The minimal protocol is defined by the following code:
This is equivalent to (and generates) the following JSON protocol definition:
{
"protocol" : "MyProtocol",
"types" : [ ],
"messages" : {
}
}
The namespace of the protocol may be changed using the @namespace annotation:
@namespace("mynamespace")
protocol MyProtocol {
}
This notation is used throughout Avro IDL as a way of specifying properties for the annotated element, as will be described later in this document.
Protocols in Avro IDL can contain the following items:
- Imports of external protocol and schema files.
- Definitions of named schemata, including records, errors, enums, and fixeds.
- Definitions of RPC messages
Imports
Files may be imported in one of three formats:
-
An IDL file may be imported with a statement like:
import idl "foo.avdl";
-
A JSON protocol file may be imported with a statement like:
import protocol "foo.avpr";
-
A JSON schema file may be imported with a statement like:
import schema "foo.avsc";
Messages and types in the imported file are added to this file’s protocol.
Imported file names are resolved relative to the current IDL file.
Defining an Enumeration
Enums are defined in Avro IDL using a syntax similar to C or Java:
enum Suit {
SPADES, DIAMONDS, CLUBS, HEARTS
}
Note that, unlike the JSON format, anonymous enums cannot be defined.
Defining a Fixed Length Field
Fixed fields are defined using the following syntax:
fixed MD5(16);
This example defines a fixed-length type called MD5 which contains 16 bytes.
Defining Records and Errors
Records are defined in Avro IDL using a syntax similar to a struct definition in C:
record Employee {
string name;
boolean active = true;
long salary;
}
The above example defines a record with the name “Employee” with three fields.
To define an error, simply use the keyword error instead of record. For example:
error Kaboom {
string explanation;
int result_code = -1;
}
Each field in a record or error consists of a type and a name, optional property annotations and an optional default value.
A type reference in Avro IDL must be one of:
- A primitive type
- A logical type
- A named schema defined prior to this usage in the same Protocol
- A complex type (array, map, or union)
Primitive Types
The primitive types supported by Avro IDL are the same as those supported by Avro’s JSON format. This list includes int, long, string, boolean, float, double, null, and bytes.
References to Named Schemata
If a named schema has already been defined in the same Avro IDL file, it may be referenced by name as if it were a primitive type:
record Card {
Suit suit; // refers to the enum Card defined above
int number;
}
Default Values
Default values for fields may be optionally specified by using an equals sign after the field name followed by a JSON expression indicating the default value. This JSON is interpreted as described in the spec.
Complex Types
Arrays
Array types are written in a manner that will seem familiar to C++ or Java programmers. An array of any type t is denoted array<t>. For example, an array of strings is denoted array<string>, and a multidimensional array of Foo records would be array<array<Foo>>.
Maps
Map types are written similarly to array types. An array that contains values of type t is written map<t>. As in the JSON schema format, all maps contain string-type keys.
Unions
Union types are denoted as union { typeA, typeB, typeC, … }. For example, this record contains a string field that is optional (unioned with null):
record RecordWithUnion {
union { null, string } optionalString;
}
Note that the same restrictions apply to Avro IDL unions as apply to unions defined in the JSON format; namely, a record may not contain multiple elements of the same type.
Defining RPC Messages
The syntax to define an RPC message within a Avro IDL protocol is similar to the syntax for a method declaration within a C header file or a Java interface. To define an RPC message add which takes two arguments named foo and bar, returning an int, simply include the following definition within the protocol:
int add(int foo, int bar = 0);
Message arguments, like record fields, may specify default values.
To define a message with no response, you may use the alias void, equivalent to the Avro null type:
void logMessage(string message);
If you have previously defined an error type within the same protocol, you may declare that a message can throw this error using the syntax:
void goKaboom() throws Kaboom;
To define a one-way message, use the keyword oneway after the parameter list, for example:
void fireAndForget(string message) oneway;
Other Language Features
All Java-style comments are supported within a Avro IDL file. Any text following // on a line is ignored, as is any text between /* and */, possibly spanning multiple lines.
Comments that begin with /** are used as the documentation string for the type or field definition that follows the comment.
Escaping Identifiers
Occasionally, one will need to use a reserved language keyword as an identifier. In order to do so, backticks (`) may be used to escape the identifier. For example, to define a message with the literal name error, you may write:
This syntax is allowed anywhere an identifier is expected.
Annotations for Ordering and Namespaces
Java-style annotations may be used to add additional properties to types and fields throughout Avro IDL.
For example, to specify the sort order of a field within a record, one may use the @order annotation before the field name as follows:
record MyRecord {
string @order("ascending") myAscendingSortField;
string @order("descending") myDescendingField;
string @order("ignore") myIgnoredField;
}
A field’s type may also be preceded by annotations, e.g.:
record MyRecord {
@java-class("java.util.ArrayList") array<string> myStrings;
}
Similarly, a @namespace annotation may be used to modify the namespace when defining a named schema. For example:
@namespace("org.apache.avro.firstNamespace")
protocol MyProto {
@namespace("org.apache.avro.someOtherNamespace")
record Foo {}
record Bar {}
}
will define a protocol in the firstNamespace namespace. The record Foo will be defined in someOtherNamespace and Bar will be defined in firstNamespace as it inherits its default from its container.
Type and field aliases are specified with the @aliases annotation as follows:
@aliases(["org.old.OldRecord", "org.ancient.AncientRecord"])
record MyRecord {
string @aliases(["oldField", "ancientField"]) myNewField;
}
Some annotations like those listed above are handled specially. All other annotations are added as properties to the protocol, message, schema or field.
Complete Example
The following is a complete example of a Avro IDL file that shows most of the above features:
/**
* An example protocol in Avro IDL
*/
@namespace("org.apache.avro.test")
protocol Simple {
@aliases(["org.foo.KindOf"])
enum Kind {
FOO,
BAR, // the bar enum value
BAZ
}
fixed MD5(16);
record TestRecord {
@order("ignore")
string name;
@order("descending")
Kind kind;
MD5 hash;
union { MD5, null} @aliases(["hash"]) nullableHash;
array<long> arrayOfLongs;
}
error TestError {
string message;
}
string hello(string greeting);
TestRecord echo(TestRecord `record`);
int add(int arg1, int arg2);
bytes echoBytes(bytes data);
void `error`() throws TestError;
void ping() oneway;
}
Additional examples may be found in the Avro source tree under the src/test/idl/input directory.
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
16.6 - SASL profile
Introduction
SASL (RFC 2222) provides a framework for authentication and security of network protocols. Each protocol that uses SASL is meant to define a SASL profile. This document provides a SASL profile for connection-based Avro RPC.
Overview
SASL negotiation proceeds as a series of message interactions over a connection between a client and server using a selected SASL mechanism. The client starts this negotiation by sending its chosen mechanism name with an initial (possibly empty) message. Negotiation proceeds with the exchange of messages until either side indicates success or failure. The content of the messages is mechanism-specific. If the negotiation succeeds, then the session can proceed over the connection, otherwise it must be abandoned.
Some mechanisms continue to process session data after negotiation (e.g., encrypting it), while some specify that further session data is transmitted unmodified.
Negotiation
Commands
Avro SASL negotiation uses four one-byte commands.
- 0: START Used in a client’s initial message.
- 1: CONTINUE Used while negotiation is ongoing.
- 2: FAIL Terminates negotiation unsuccessfully.
- 3: COMPLETE Terminates negotiation successfully.
The format of a START message is:
| 0 | 4-byte mechanism name length | mechanism name | 4-byte payload length | payload data |
The format of a CONTINUE message is:
| 1 | 4-byte payload length | payload data |
The format of a FAIL message is:
| 2 | 4-byte message length | UTF-8 message |
The format of a COMPLETE message is:
| 3 | 4-byte payload length | payload data |
Process
Negotiation is initiated by a client sending a START command containing the client’s chosen mechanism name and any mechanism-specific payload data.
The server and client then interchange some number (possibly zero) of CONTINUE messages. Each message contains payload data that is processed by the security mechanism to generate the next message.
Once either the client or server send a FAIL message then negotiation has failed. UTF-8-encoded text is included in the failure message. Once either a FAIL message has been sent or received, or any other error occurs in the negotiation, further communication on this connection must cease.
Once either the client or server send a COMPLETE message then negotiation has completed successfully. Session data may now be transmitted over the connection until it is closed by either side.
Session Data
If no SASL QOP (quality of protection) is negotiated, then all subsequent writes to/reads over this connection are written/read unmodified. In particular, messages use Avro framing, and are of the form:
| 4-byte frame length | frame data | ... | 4 zero bytes |
If a SASL QOP is negotiated, then it must be used by the connection for all subsequent messages. This is done by wrapping each non-empty frame written using the security mechanism and unwrapping each non-empty frame read. The length written in each non-empty frame is the length of the wrapped data. Complete frames must be passed to the security mechanism for unwrapping. Unwrapped data is then passed to the application as the content of the frame.
If at any point processing fails due to wrapping, unwrapping or framing errors, then all further communication on this connection must cease.
Anonymous Mechanism
The SASL anonymous mechanism (RFC 2245) is quite simple to implement. In particular, an initial anonymous request may be prefixed by the following static sequence:
| 0 | 0009 | ANONYMOUS | 0000 |
If a server uses the anonymous mechanism, it should check that the mechanism name in the start message prefixing the first request received is ‘ANONYMOUS’, then simply prefix its initial response with a COMPLETE message of:
| 3 | 0000 |
If an anonymous server recieves some other mechanism name, then it may respond with a FAIL message as simple as:
| 2 | 0000 |
Note that the anonymous mechanism need add no additional round-trip messages between client and server. The START message can be piggybacked on the initial request and the COMPLETE or FAIL message can be piggybacked on the initial response.
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
16.7 - Wiki
The Wiki page can be found here.
16.8 - FAQ
What is Avro?
Avro is a data serialization system.
Avro provides:
- Rich data structures.
- A compact, fast, binary data format.
- A container file, to store persistent data.
- Remote procedure call (RPC).
- Simple integration with dynamic languages. Code generation is not required to read or write data files nor to use or implement RPC protocols. Code generation as an optional optimization, only worth implementing for statically typed languages.
How can I get started quickly with Avro?
Check out the Quick Start Guide.
How do I statically compile a schema or protocol into generated code?
In Java
- Add the avro jar, the jackson-mapper-asl.jar and jackson-core-asl.jar to your CLASSPATH.
- Run java org.apache.avro.specific.SpecificCompiler .
This appears to be out of date, the SpecificCompiler requires two arguments, presumably an input and and output file, but it isn’t clear that this does.
Or use the Schema or Protocol Ant tasks. Avro’s build.xml provides examples of how these are used.
Lastly, you can also use the “avro-tools” jar which ships with an Avro release. Just use the “compile (schema|protocol)” command.
How are Strings represented in Java?
They use org.apache.avro.util.Utf8, not java.lang.String.
More generally, how do Avro types map to Java types?
The mappings are documented in the package javadoc for generic, specific and reflect API.
From Doug Cutting:
HDFS splits files into blocks, and mapreduce runs a map task for each block. When the task starts, it needs to be able to seek into the file to the start of the block process through the block’s end. If the file were, e.g., a gzip file, this would not be possible, since gzip files must be decompressed from the start. One cannot seek into the middle of a gzip file and start decompressing. So Hadoop’s SequenceFile places a marker periodically (~64k) in the file at record and compression boundaries, where processing can be sensibly started. Then, when a map task starts processing an HDFS block, it finds the first marker after the block’s start and continues through the first marker in the next block of the file. This requires a bit of non-local access (~0.1%). Avro’s data file uses the same method as SequenceFile.
Why isn’t every value in Avro nullable?
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 of every value stored and
corresponding computational costs to create this bit on write and
interpret it on read. These costs are wasted when values may not in
fact be null, as is the case in many datasets. In Avro such costs are
only paid when values may actually be null.
Also, allowing values to be null is a well-known source of errors. In
Avro, a value declared as non-null will always be non-null and programs
need not test for null values when processing it nor will they ever fail
for lack of such tests.
Tony Hoare calls his invention of null references his “Billion Dollar
Mistake”.
http://qconlondon.com/london-2009/presentation/Null+References:+The+Billion+Dollar+Mistake
Also note that in some programming languages not all values are permitted to be null. For example, in Java, values of type boolean, byte, short, char, int, float, long, and double may not be null.
How can I serialize directly to/from a byte array?
As pointed out in the specification, Avro data should always be stored with its schema. The Avro provided classes DataFileWriter, DataFileReader, and DataFileStream all ensure this by serializing the Schema in a container header. In some special cases, such as when implementing a new storage system or writing unit tests, you may need to write and read directly with the bare Avro serialized values. The following examples use code generated for Java from an Avro Schema, specifically the User example from the Quickstart guide.
Serializing to a byte array
This example takes a User object and returns a newly allocated byte array with the Avro serialization of that user.
ByteArrayOutputStream out = new ByteArrayOutputStream();
BinaryEncoder encoder = EncoderFactory.get().binaryEncoder(out, null);
DatumWriter<User> writer = new SpecificDatumWriter<User>(User.getClassSchema());
writer.write(user, encoder);
encoder.flush();
out.close();
byte[] serializedBytes = out.toByteArray();
Deserializing from a byte array
This example takes a byte array containing the Avro serialization of a user and returns a User object.SpecificDatumReader reader = new SpecificDatumReader(User.getClassSchema());
Decoder decoder = DecoderFactory.get().binaryDecoder(bytes, null);
User user = reader.read(null, decoder);
Note: If you are serializing or deserializing in a loop or as a method, you should be reusing objects, readers and/or writers. Check the JavaDoc to see how to reuse the objects.
17 - Apache Avro™ 1.7.2 Documentation
Introduction
Apache Avro™ is a data serialization system.
Avro provides:
- Rich data structures.
- A compact, fast, binary data format.
- A container file, to store persistent data.
- Remote procedure call (RPC).
- Simple integration with dynamic languages. Code generation is not required to read or write data files nor to use or implement RPC protocols. Code generation as an optional optimization, only worth implementing for statically typed languages.
Schemas
Avro relies on schemas. When Avro data is read, the schema used when writing it is always present. This permits each datum to be written with no per-value overheads, making serialization both fast and small. This also facilitates use with dynamic, scripting languages, since data, together with its schema, is fully self-describing.
When Avro data is stored in a file, its schema is stored with it, so that files may be processed later by any program. If the program reading the data expects a different schema this can be easily resolved, since both schemas are present.
When Avro is used in RPC, the client and server exchange schemas in the connection handshake. (This can be optimized so that, for most calls, no schemas are actually transmitted.) Since both client and server both have the other’s full schema, correspondence between same named fields, missing fields, extra fields, etc. can all be easily resolved.
Avro schemas are defined with JSON . This facilitates implementation in languages that already have JSON libraries.
Comparison with other systems
Avro provides functionality similar to systems such as Thrift, Protocol Buffers, etc. Avro differs from these systems in the following fundamental aspects.
- Dynamic typing: Avro does not require that code be generated. Data is always accompanied by a schema that permits full processing of that data without code generation, static datatypes, etc. This facilitates construction of generic data-processing systems and languages.
- Untagged data: Since the schema is present when data is read, considerably less type information need be encoded with data, resulting in smaller serialization size.
- No manually-assigned field IDs: When a schema changes, both the old and new schema are always present when processing data, so differences may be resolved symbolically, using field names.
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
17.1 - Specification
Introduction
This document defines Apache Avro. It is intended to be the authoritative specification. Implementations of Avro must adhere to this document.
Schema Declaration
A Schema is represented in JSON by one of:
- A JSON string, naming a defined type.
- A JSON object, of the form:
{"type": "typeName" ...attributes...}
where typeName is either a primitive or derived type name, as defined below. Attributes not defined in this document are permitted as metadata, but must not affect the format of serialized data.
- A JSON array, representing a union of embedded types.
Primitive Types
The set of primitive type names is:
- null: no value
- boolean: a binary value
- int: 32-bit signed integer
- long: 64-bit signed integer
- float: single precision (32-bit) IEEE 754 floating-point number
- double: double precision (64-bit) IEEE 754 floating-point number
- bytes: sequence of 8-bit unsigned bytes
- string: unicode character sequence
Primitive types have no specified attributes.
Primitive type names are also defined type names. Thus, for example, the schema “string” is equivalent to:
Complex Types
Avro supports six kinds of complex types: records, enums, arrays, maps, unions and fixed.
Records
Records use the type name “record” and support three attributes:
- name: a JSON string providing the name of the record (required).
- namespace, a JSON string that qualifies the name;
- doc: a JSON string providing documentation to the user of this schema (optional).
- aliases: a JSON array of strings, providing alternate names for this record (optional).
- fields: a JSON array, listing fields (required). Each field is a JSON object with the following attributes:
- name: a JSON string providing the name of the field (required), and
- doc: a JSON string describing this field for users (optional).
- type: A JSON object defining a schema, or a JSON string naming a record definition (required).
- default: A default value for this field, used when reading instances that lack this field (optional). 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. Default values for bytes and fixed fields are JSON strings, where Unicode code points 0-255 are mapped to unsigned 8-bit byte values 0-255.
field default values
| avro type |
json type |
example |
| null |
null |
null |
| boolean |
boolean |
true |
| int,long |
integer |
1 |
| float,double |
number |
1.1 |
| bytes |
string |
"\u00FF" |
| string |
string |
"foo" |
| record |
object |
{"a": 1} |
| enum |
string |
"FOO" |
| array |
array |
[1] |
| map |
object |
{"a": 1} |
| fixed |
string |
"\u00ff" |
- order: specifies how this field impacts sort ordering of this record (optional). Valid values are “ascending” (the default), “descending”, or “ignore”. For more details on how this is used, see the sort order section below.
- aliases: a JSON array of strings, providing alternate names for this field (optional).
For example, a linked-list of 64-bit values may be defined with:
{
"type": "record",
"name": "LongList",
"aliases": ["LinkedLongs"], // old name for this
"fields" : [
{"name": "value", "type": "long"}, // each element has a long
{"name": "next", "type": ["LongList", "null"]} // optional next element
]
}
Enums
Enums use the type name “enum” and support the following attributes:
- name: a JSON string providing the name of the enum (required).
- namespace, a JSON string that qualifies the name ;
- aliases: a JSON array of strings, providing alternate names for this enum (optional).
- doc: a JSON string providing documentation to the user of this schema (optional).
- symbols: a JSON array, listing symbols, as JSON strings (required). All symbols in an enum must be unique; duplicates are prohibited.
For example, playing card suits might be defined with:
{
"type": "enum",
"name": "Suit",
"symbols" : ["SPADES", "HEARTS", "DIAMONDS", "CLUBS"]
}
Arrays
Arrays use the type name “array” and support a single attribute:
- items: the schema of the array’s items.
For example, an array of strings is declared with:
{
"type": "array",
"items" : "string"
}
Maps
Maps use the type name “map” and support one attribute:
- values: the schema of the map’s values.
Map keys are assumed to be strings.
For example, a map from string to long is declared with:
{
"type": "map",
"values" : "long"
}
Unions
Unions, as mentioned above, are represented using JSON arrays. For example, ["string", "null"] declares a schema which may be either a string or null.
Unions may not contain more than one schema with the same type, except for the named types record, fixed and enum. For example, unions containing two array types or two map types are not permitted, but two types with different names are permitted. (Names permit efficient resolution when reading and writing unions.)
Unions may not immediately contain other unions.
Fixed
Fixed uses the type name “fixed” and supports two attributes:
- name: a string naming this fixed (required).
- namespace, a string that qualifies the name;
- aliases: a JSON array of strings, providing alternate names for this enum (optional).
- size: an integer, specifying the number of bytes per value (required).
For example, 16-byte quantity may be declared with:
{"type": "fixed", "size": 16, "name": "md5"}
Names
Record, enums and fixed are named types. Each has a fullname that is composed of two parts; a name and a namespace. Equality of names is defined on the fullname.
The name portion of a fullname, record field names, and enum symbols must:
- start with [A-Za-z_]
- subsequently contain only [A-Za-z0-9_]
A namespace is a dot-separated sequence of such names. Equality of names (including field names and enum symbols) as well as fullnames is case-sensitive.
In record, enum and fixed definitions, the fullname is determined in one of the following ways:
A name and namespace are both specified. For example, one might use “name”: “X”, “namespace”: “org.foo” to indicate the fullname org.foo.X.
-
A fullname is specified. If the name specified contains a dot, then it is assumed to be a fullname, and any namespace also specified is ignored. For example, use “name”: “org.foo.X” to indicate the fullname org.foo.X.
-
A name only is specified, i.e., a name that contains no dots. In this case the namespace is taken from the most tightly enclosing schema or protocol. For example, if “name”: “X” is specified, and this occurs within a field of the record definition of org.foo.Y, then the fullname is org.foo.X.
References to previously defined names are as in the latter two cases above: if they contain a dot they are a fullname, if they do not contain a dot, the namespace is the namespace of the enclosing definition.
Primitive type names have no namespace and their names may not be defined in any namespace.
A schema or protocol may not contain multiple definitions of a fullname. Further, a name must be defined before it is used (“before” in the depth-first, left-to-right traversal of the JSON parse tree, where the types attribute of a protocol is always deemed to come “before” the messages attribute.)
Aliases
Named types and fields may have aliases. An implementation may optionally use aliases to map a writer’s schema to the reader’s. This facilitates both schema evolution as well as processing disparate datasets.
Aliases function by re-writing the writer’s schema using aliases from the reader’s schema. For example, if the writer’s schema was named “Foo” and the reader’s schema is named “Bar” and has an alias of “Foo”, then the implementation would act as though “Foo” were named “Bar” when reading. Similarly, if data was written as a record with a field named “x” and is read as a record with a field named “y” with alias “x”, then the implementation would act as though “x” were named “y” when reading.
A type alias may be specified either as a fully namespace-qualified, or relative to the namespace of the name it is an alias for. For example, if a type named “a.b” has aliases of “c” and “x.y”, then the fully qualified names of its aliases are “a.c” and “x.y”.
Data Serialization
Avro data is always serialized with its schema. Files that store Avro data should always also include the schema for that data in the same file. Avro-based remote procedure call (RPC) systems must also guarantee that remote recipients of data have a copy of the schema used to write that data.
Because the schema used to write data is always available when the data is read, Avro data itself is not tagged with type information. The schema is required to parse data.
In general, both serialization and deserialization proceed as a depth-first, left-to-right traversal of the schema, serializing primitive types as they are encountered.
Encodings
Avro specifies two serialization encodings: binary and JSON. Most applications will use the binary encoding, as it is smaller and faster. But, for debugging and web-based applications, the JSON encoding may sometimes be appropriate.
Binary Encoding
Primitive Types
Primitive types are encoded in binary as follows:
- null is written as zero bytes.
- a boolean is written as a single byte whose value is either 0 (false) or 1 (true).
- int and long values are written using variable-length zig-zag coding. Some examples:
| value |
hex |
| 0 |
00 |
| -1 |
01 |
| 1 |
02 |
| -2 |
03 |
| 2 |
04 |
| … |
|
| -64 |
7f |
| 64 |
80 01 |
| … |
|
- a float is written as 4 bytes. The float is converted into a 32-bit integer using a method equivalent to Java’s floatToIntBits and then encoded in little-endian format.
- a double is written as 8 bytes. The double is converted into a 64-bit integer using a method equivalent to Java’s doubleToLongBits and then encoded in little-endian format.
- bytes are encoded as a long followed by that many bytes of data.
- a string is encoded as a long followed by that many bytes of UTF-8 encoded character data.
For example, the three-character string “foo” would be encoded as the long value 3 (encoded as hex 06) followed by the UTF-8 encoding of ‘f’, ‘o’, and ‘o’ (the hex bytes 66 6f 6f):
06 66 6f 6f
Complex Types
Complex types are encoded in binary as follows:
Records
A record is encoded by encoding the values of its fields in the order that they are declared. In other words, a record is encoded as just the concatenation of the encodings of its fields. Field values are encoded per their schema.
For example, the record schema
{
"type": "record",
"name": "test",
"fields" : [
{"name": "a", "type": "long"},
{"name": "b", "type": "string"}
]
}
An instance of this record whose a field has value 27 (encoded as hex 36) and whose b field has value “foo” (encoded as hex bytes 06 66 6f 6f), would be encoded simply as the concatenation of these, namely the hex byte sequence:
36 06 66 6f 6f
Enums
An enum is encoded by a int, representing the zero-based position of the symbol in the schema.
For example, consider the enum:
{"type": "enum", "name": "Foo", "symbols": ["A", "B", "C", "D"] }
This would be encoded by an int between zero and three, with zero indicating “A”, and 3 indicating “D”.
Arrays
Arrays are encoded as a series of blocks. Each block consists of a long count value, followed by that many array items. A block with count zero indicates the end of the array. Each item is encoded per the array’s item schema.
If a block’s count is negative, its absolute value is used, and the count is followed immediately by a long block size indicating the number of bytes in the block. This block size permits fast skipping through data, e.g., when projecting a record to a subset of its fields.
For example, the array schema
{"type": "array", "items": "long"}
an array containing the items 3 and 27 could be encoded as the long value 2 (encoded as hex 04) followed by long values 3 and 27 (encoded as hex 06 36) terminated by zero:
04 06 36 00
The blocked representation permits one to read and write arrays larger than can be buffered in memory, since one can start writing items without knowing the full length of the array.
Maps
Maps are encoded as a series of blocks. Each block consists of a long count value, followed by that many key/value pairs. A block with count zero indicates the end of the map. Each item is encoded per the map’s value schema.
If a block’s count is negative, its absolute value is used, and the count is followed immediately by a long block size indicating the number of bytes in the block. This block size permits fast skipping through data, e.g., when projecting a record to a subset of its fields.
The blocked representation permits one to read and write maps larger than can be buffered in memory, since one can start writing items without knowing the full length of the map.
Unions
A union is encoded by first writing a long value indicating the zero-based position within the union of the schema of its value. The value is then encoded per the indicated schema within the union.
For example, the union schema ["string","null"] would encode:
-
null as the integer 1 (the index of “null” in the union, encoded as hex 02):
02
-
the string “a” as zero (the index of “string” in the union), followed by the serialized string:
02 02 61
Fixed
Fixed instances are encoded using the number of bytes declared in the schema.
JSON Encoding
Except for unions, the JSON encoding is the same as is used to encode field default values.
The value of a union is encoded in JSON as follows:
- if its type is null, then it is encoded as a JSON null;
- otherwise it is encoded as a JSON object with one name/value pair whose name is the type’s name and whose value is the recursively encoded value. For Avro’s named types (record, fixed or enum) the user-specified name is used, for other types the type name is used.
For example, the union schema ["null","string","Foo"], where Foo is a record name, would encode:
- null as null;
- the string “a” as
{"string": "a"} and
- a Foo instance as
{"Foo": {...}}, where {...} indicates the JSON encoding of a Foo instance.
Note that a schema is still required to correctly process JSON-encoded data. For example, the JSON encoding does not distinguish between int and long, float and double, records and maps, enums and strings, etc.
Sort Order
Avro defines a standard sort order for data. This permits data written by one system to be efficiently sorted by another system. This can be an important optimization, as sort order comparisons are sometimes the most frequent per-object operation. Note also that Avro binary-encoded data can be efficiently ordered without deserializing it to objects.
Data items may only be compared if they have identical schemas. Pairwise comparisons are implemented recursively with a depth-first, left-to-right traversal of the schema. The first mismatch encountered determines the order of the items.
Two items with the same schema are compared according to the following rules.
- null data is always equal.
- boolean data is ordered with false before true.
- int, long, float and double data is ordered by ascending numeric value.
- bytes and fixed data are compared lexicographically by unsigned 8-bit values.
- string data is compared lexicographically by Unicode code point. Note that since UTF-8 is used as the binary encoding for strings, sorting of bytes and string binary data is identical.
- array data is compared lexicographically by element.
- enum data is ordered by the symbol’s position in the enum schema. For example, an enum whose symbols are
["z", "a"] would sort “z” values before “a” values.
- union data is first ordered by the branch within the union, and, within that, by the type of the branch. For example, an
["int", "string"] union would order all int values before all string values, with the ints and strings themselves ordered as defined above.
- record data is ordered lexicographically by field. If a field specifies that its order is:
- “ascending”, then the order of its values is unaltered.
- “descending”, then the order of its values is reversed.
- “ignore”, then its values are ignored when sorting.
- map data may not be compared. It is an error to attempt to compare data containing maps unless those maps are in an
"order":"ignore" record field.
Object Container Files
Avro includes a simple object container file format. A file has a schema, and all objects stored in the file must be written according to that schema, using binary encoding. Objects are stored in blocks that may be compressed. Syncronization markers are used between blocks to permit efficient splitting of files for MapReduce processing.
Files may include arbitrary user-specified metadata.
A file consists of:
- A file header, followed by
- one or more file data blocks.
A file header consists of:
- Four bytes, ASCII ‘O’, ‘b’, ‘j’, followed by 1.
- file metadata, including the schema.
- The 16-byte, randomly-generated sync marker for this file.
File metadata consists of:
A long indicating the number of metadata key/value pairs.
For each pair, a string key and bytes value.
All metadata properties that start with “avro.” are reserved. The following file metadata properties are currently used:
- avro.schema contains the schema of objects stored in the file, as JSON data (required).
- avro.codec the name of the compression codec used to compress blocks, as a string. Implementations are required to support the following codecs: “null” and “deflate”. If codec is absent, it is assumed to be “null”. The codecs are described with more detail below.
A file header is thus described by the following schema:
{"type": "record", "name": "org.apache.avro.file.Header",
"fields" : [
{"name": "magic", "type": {"type": "fixed", "name": "Magic", "size": 4}},
{"name": "meta", "type": {"type": "map", "values": "bytes"}},
{"name": "sync", "type": {"type": "fixed", "name": "Sync", "size": 16}}
]
}
A file data block consists of:
- A long indicating the count of objects in this block.
- A long indicating the size in bytes of the serialized objects in the current block, after any codec is applied
- The serialized objects. If a codec is specified, this is compressed by that codec.
- The file’s 16-byte sync marker.
Thus, each block’s binary data can be efficiently extracted or skipped without deserializing the contents. The combination of block size, object counts, and sync markers enable detection of corrupt blocks and help ensure data integrity.
Required Codecs
null
The “null” codec simply passes through data uncompressed.
deflate
The “deflate” codec writes the data block using the deflate algorithm as specified in RFC 1951, and typically implemented using the zlib library. Note that this format (unlike the “zlib format” in RFC 1950) does not have a checksum.
Optional Codecs
snappy
The “snappy” codec uses Google’s Snappy compression library. Each compressed block is followed by the 4-byte, big-endian CRC32 checksum of the uncompressed data in the block.
Protocol Declaration
Avro protocols describe RPC interfaces. Like schemas, they are defined with JSON text.
A protocol is a JSON object with the following attributes:
- protocol, a string, the name of the protocol (required);
- namespace, an optional string that qualifies the name;
- doc, an optional string describing this protocol;
- types, an optional list of definitions of named types (records, enums, fixed and errors). An error definition is just like a record definition except it uses “error” instead of “record”. Note that forward references to named types are not permitted.
- messages, an optional JSON object whose keys are message names and whose values are objects whose attributes are described below. No two messages may have the same name.
The name and namespace qualification rules defined for schema objects apply to protocols as well.
Messages
A message has attributes:
- a doc, an optional description of the message,
- a request, a list of named, typed parameter schemas (this has the same form as the fields of a record declaration);
- a response schema;
- an optional union of declared error schemas. The effective union has “string” prepended to the declared union, to permit transmission of undeclared “system” errors. For example, if the declared error union is
["AccessError"], then the effective union is ["string", "AccessError"]. When no errors are declared, the effective error union is ["string"]. Errors are serialized using the effective union; however, a protocol’s JSON declaration contains only the declared union.
- an optional one-way boolean parameter.
A request parameter list is processed equivalently to an anonymous record. Since record field lists may vary between reader and writer, request parameters may also differ between the caller and responder, and such differences are resolved in the same manner as record field differences.
The one-way parameter may only be true when the response type is "null" and no errors are listed.
Sample Protocol
For example, one may define a simple HelloWorld protocol with:
{
"namespace": "com.acme",
"protocol": "HelloWorld",
"doc": "Protocol Greetings",
"types": [
{"name": "Greeting", "type": "record", "fields": [
{"name": "message", "type": "string"}]},
{"name": "Curse", "type": "error", "fields": [
{"name": "message", "type": "string"}]}
],
"messages": {
"hello": {
"doc": "Say hello.",
"request": [{"name": "greeting", "type": "Greeting" }],
"response": "Greeting",
"errors": ["Curse"]
}
}
}
Message Transport
Messages may be transmitted via different transport mechanisms.
To the transport, a message is an opaque byte sequence.
A transport is a system that supports:
- transmission of request messages
- receipt of corresponding response messages
Servers may send a response message back to the client corresponding to a request message. The mechanism of correspondance is transport-specific. For example, in HTTP it is implicit, since HTTP directly supports requests and responses. But a transport that multiplexes many client threads over a single socket would need to tag messages with unique identifiers.
Transports may be either stateless or stateful. In a stateless transport, messaging assumes no established connection state, while stateful transports establish connections that may be used for multiple messages. This distinction is discussed further in the handshake section below.
HTTP as Transport
When HTTP is used as a transport, each Avro message exchange is an HTTP request/response pair. All messages of an Avro protocol should share a single URL at an HTTP server. Other protocols may also use that URL. Both normal and error Avro response messages should use the 200 (OK) response code. The chunked encoding may be used for requests and responses, but, regardless the Avro request and response are the entire content of an HTTP request and response. The HTTP Content-Type of requests and responses should be specified as “avro/binary”. Requests should be made using the POST method.
HTTP is used by Avro as a stateless transport.
Message Framing
Avro messages are framed as a list of buffers.
Framing is a layer between messages and the transport. It exists to optimize certain operations.
The format of framed message data is:
- a series of buffers, where each buffer consists of:
- a four-byte, big-endian buffer length, followed by
- that many bytes of buffer data.
- A message is always terminated by a zero-lenghted buffer.
Framing is transparent to request and response message formats (described below). Any message may be presented as a single or multiple buffers.
Framing can permit readers to more efficiently get different buffers from different sources and for writers to more efficiently store different buffers to different destinations. In particular, it can reduce the number of times large binary objects are copied. For example, if an RPC parameter consists of a megabyte of file data, that data can be copied directly to a socket from a file descriptor, and, on the other end, it could be written directly to a file descriptor, never entering user space.
A simple, recommended, framing policy is for writers to create a new segment whenever a single binary object is written that is larger than a normal output buffer. Small objects are then appended in buffers, while larger objects are written as their own buffers. When a reader then tries to read a large object the runtime can hand it an entire buffer directly, without having to copy it.
Handshake
The purpose of the handshake is to ensure that the client and the server have each other’s protocol definition, so that the client can correctly deserialize responses, and the server can correctly deserialize requests. Both clients and servers should maintain a cache of recently seen protocols, so that, in most cases, a handshake will be completed without extra round-trip network exchanges or the transmission of full protocol text.
RPC requests and responses may not be processed until a handshake has been completed. With a stateless transport, all requests and responses are prefixed by handshakes. With a stateful transport, handshakes are only attached to requests and responses until a successful handshake response has been returned over a connection. After this, request and response payloads are sent without handshakes for the lifetime of that connection.
The handshake process uses the following record schemas:
{
"type": "record",
"name": "HandshakeRequest", "namespace":"org.apache.avro.ipc",
"fields": [
{"name": "clientHash",
"type": {"type": "fixed", "name": "MD5", "size": 16}},
{"name": "clientProtocol", "type": ["null", "string"]},
{"name": "serverHash", "type": "MD5"},
{"name": "meta", "type": ["null", {"type": "map", "values": "bytes"}]}
]
}
{
"type": "record",
"name": "HandshakeResponse", "namespace": "org.apache.avro.ipc",
"fields": [
{"name": "match",
"type": {"type": "enum", "name": "HandshakeMatch",
"symbols": ["BOTH", "CLIENT", "NONE"]}},
{"name": "serverProtocol",
"type": ["null", "string"]},
{"name": "serverHash",
"type": ["null", {"type": "fixed", "name": "MD5", "size": 16}]},
{"name": "meta",
"type": ["null", {"type": "map", "values": "bytes"}]}
]
}
- A client first prefixes each request with a
HandshakeRequest containing just the hash of its protocol and of the server’s protocol (clientHash!=null, clientProtocol=null, serverHash!=null), where the hashes are 128-bit MD5 hashes of the JSON protocol text. If a client has never connected to a given server, it sends its hash as a guess of the server’s hash, otherwise it sends the hash that it previously obtained from this server.
The server responds with a HandshakeResponse containing one of:
-
match=BOTH, serverProtocol=null, serverHash=null if the client sent the valid hash of the server’s protocol and the server knows what protocol corresponds to the client’s hash. In this case, the request is complete and the response data immediately follows the HandshakeResponse.
-
match=CLIENT, serverProtocol!=null, serverHash!=null if the server has previously seen the client’s protocol, but the client sent an incorrect hash of the server’s protocol. The request is complete and the response data immediately follows the HandshakeResponse. The client must use the returned protocol to process the response and should also cache that protocol and its hash for future interactions with this server.
-
match=NONE if the server has not previously seen the client’s protocol. The serverHash and serverProtocol may also be non-null if the server’s protocol hash was incorrect.
In this case the client must then re-submit its request with its protocol text (clientHash!=null, clientProtocol!=null, serverHash!=null) and the server should respond with a successful match (match=BOTH, serverProtocol=null, serverHash=null) as above.
The meta field is reserved for future handshake enhancements.
A call consists of a request message paired with its resulting response or error message. Requests and responses contain extensible metadata, and both kinds of messages are framed as described above.
The format of a call request is:
- request metadata, a map with values of type bytes
- the message name, an Avro string, followed by
- the message parameters. Parameters are serialized according to the message’s request declaration.
When a message is declared one-way and a stateful connection has been established by a successful handshake response, no response data is sent. Otherwise the format of the call response is:
- response metadata, a map with values of type bytes
- a one-byte error flag boolean, followed by either:
- if the error flag is false, the message response, serialized per the message’s response schema.
- if the error flag is true, the error, serialized per the message’s effective error union schema.
Schema Resolution
A reader of Avro data, whether from an RPC or a file, can always parse that data because its schema is provided. But that schema may not be exactly the schema that was expected. For example, if the data was written with a different version of the software than it is read, then records may have had fields added or removed. This section specifies how such schema differences should be resolved.
We refer to the schema used to write the data as the writer’s schema, and the schema that the application expects the reader’s schema. Differences between these should be resolved as follows:
- It is an error if the two schemas do not match.
To match, one of the following must hold:
-
both schemas are arrays whose item types match
-
both schemas are maps whose value types match
-
both schemas are enums whose names match
-
both schemas are fixed whose sizes and names match
-
both schemas are records with the same name
-
either schema is a union
-
both schemas have same primitive type
-
the writer’s schema may be promoted to the reader’s as follows:
- int is promotable to long, float, or double
- long is promotable to float or double
- float is promotable to double
- string is promotable to bytes
- bytes is promotable to string
-
if both are records:
- the ordering of fields may be different: fields are matched by name.
- schemas for fields with the same name in both records are resolved recursively.
- if the writer’s record contains a field with a name not present in the reader’s record, the writer’s value for that field is ignored.
- if the reader’s record schema has a field that contains a default value, and writer’s schema does not have a field with the same name, then the reader should use the default value from its field.
- if the reader’s record schema has a field with no default value, and writer’s schema does not have a field with the same name, an error is signalled.
-
if both are enums:
if the writer’s symbol is not present in the reader’s enum, then an error is signalled.
This resolution algorithm is applied recursively to the reader’s and writer’s array item schemas.
This resolution algorithm is applied recursively to the reader’s and writer’s value schemas.
The first schema in the reader’s union that matches the selected writer’s union schema is recursively resolved against it. if none match, an error is signalled.
- if reader’s is a union, but writer’s is not
The first schema in the reader’s union that matches the writer’s schema is recursively resolved against it. If none match, an error is signalled.
- if writer’s is a union, but reader’s is not
If the reader’s schema matches the selected writer’s schema, it is recursively resolved against it. If they do not match, an error is signalled.
A schema’s doc fields are ignored for the purposes of schema resolution. Hence, the doc portion of a schema may be dropped at serialization.
One of the defining characteristics of Avro is that a reader is assumed to have the “same” schema used by the writer of the data the reader is reading. This assumption leads to a data format that’s compact and also amenable to many forms of schema evolution. However, the specification so far has not defined what it means for the reader to have the “same” schema as the writer. Does the schema need to be textually identical? Well, clearly adding or removing some whitespace to a JSON expression does not change its meaning. At the same time, reordering the fields of records clearly does change the meaning. So what does it mean for a reader to have “the same” schema as a writer?
Parsing Canonical Form is a transformation of a writer’s schema that let’s us define what it means for two schemas to be “the same” for the purpose of reading data written against the schema. It is called Parsing Canonical Form because the transformations strip away parts of the schema, like “doc” attributes, that are irrelevant to readers trying to parse incoming data. It is called Canonical Form because the transformations normalize the JSON text (such as the order of attributes) in a way that eliminates unimportant differences between schemas. If the Parsing Canonical Forms of two different schemas are textually equal, then those schemas are “the same” as far as any reader is concerned, i.e., there is no serialized data that would allow a reader to distinguish data generated by a writer using one of the original schemas from data generated by a writing using the other original schema. (We sketch a proof of this property in a companion document.)
The next subsection specifies the transformations that define Parsing Canonical Form. But with a well-defined canonical form, it can be convenient to go one step further, transforming these canonical forms into simple integers (“fingerprints”) that can be used to uniquely identify schemas. The subsection after next recommends some standard practices for generating such fingerprints.
Assuming an input schema (in JSON form) that’s already UTF-8 text for a valid Avro schema (including all quotes as required by JSON), the following transformations will produce its Parsing Canonical Form:
- [PRIMITIVES] Convert primitive schemas to their simple form (e.g., int instead of
{"type":"int"}).
- [FULLNAMES] Replace short names with fullnames, using applicable namespaces to do so. Then eliminate namespace attributes, which are now redundant.
- [STRIP] Keep only attributes that are relevant to parsing data, which are: type, name, fields, symbols, items, values, size. Strip all others (e.g., doc and aliases).
- [ORDER] Order the appearance of fields of JSON objects as follows: name, type, fields, symbols, items, values, size. For example, if an object has type, name, and size fields, then the name field should appear first, followed by the type and then the size fields.
- [STRINGS] For all JSON string literals in the schema text, replace any escaped characters (e.g., \uXXXX escapes) with their UTF-8 equivalents.
- [INTEGERS] Eliminate quotes around and any leading zeros in front of JSON integer literals (which appear in the size attributes of fixed schemas).
- [WHITESPACE] Eliminate all whitespace in JSON outside of string literals.
Schema Fingerprints
“[A] fingerprinting algorithm is a procedure that maps an arbitrarily large data item (such as a computer file) to a much shorter bit string, its fingerprint, that uniquely identifies the original data for all practical purposes” (quoted from Wikipedia). In the Avro context, fingerprints of Parsing Canonical Form can be useful in a number of applications; for example, to cache encoder and decoder objects, to tag data items with a short substitute for the writer’s full schema, and to quickly negotiate common-case schemas between readers and writers.
In designing fingerprinting algorithms, there is a fundamental trade-off between the length of the fingerprint and the probability of collisions. To help application designers find appropriate points within this trade-off space, while encouraging interoperability and ease of implementation, we recommend using one of the following three algorithms when fingerprinting Avro schemas:
- When applications can tolerate longer fingerprints, we recommend using the SHA-256 digest algorithm to generate 256-bit fingerprints of Parsing Canonical Forms. Most languages today have SHA-256 implementations in their libraries.
- At the opposite extreme, the smallest fingerprint we recommend is a 64-bit Rabin fingerprint. Below, we provide pseudo-code for this algorithm that can be easily translated into any programming language. 64-bit fingerprints should guarantee uniqueness for schema caches of up to a million entries (for such a cache, the chance of a collision is 3E-8). We don’t recommend shorter fingerprints, as the chances of collisions is too great (for example, with 32-bit fingerprints, a cache with as few as 100,000 schemas has a 50% chance of having a collision).
- Between these two extremes, we recommend using the MD5 message digest to generate 128-bit fingerprints. These make sense only where very large numbers of schemas are being manipulated (tens of millions); otherwise, 64-bit fingerprints should be sufficient. As with SHA-256, MD5 implementations are found in most libraries today.
These fingerprints are not meant to provide any security guarantees, even the longer SHA-256-based ones. Most Avro applications should be surrounded by security measures that prevent attackers from writing random data and otherwise interfering with the consumers of schemas. We recommend that these surrounding mechanisms be used to prevent collision and pre-image attacks (i.e., “forgery”) on schema fingerprints, rather than relying on the security properties of the fingerprints themselves.
Rabin fingerprints are cyclic redundancy checks computed using irreducible polynomials. In the style of the Appendix of RFC 1952 (pg 10), which defines the CRC-32 algorithm, here’s our definition of the 64-bit AVRO fingerprinting algorithm:
long fingerprint64(byte[] buf) {
if (FP_TABLE == null) initFPTable();
long fp = EMPTY;
for (int i = 0; i < buf.length; i++)
fp = (fp >>> 8) ^ FP_TABLE[(int)(fp ^ buf[i]) & 0xff];
return fp;
}
static long EMPTY = 0xc15d213aa4d7a795L;
static long[] FP_TABLE = null;
void initFPTable() {
FP_TABLE = new long[256];
for (int i = 0; i < 256; i++) {
long fp = i;
for (int j = 0; j < 8; j++)
fp = (fp >>> 1) ^ (EMPTY & -(fp & 1L));
FP_TABLE[i] = fp;
}
}
Readers interested in the mathematics behind this algorithm may want to read this book chapter. (Unlike RFC-1952 and the book chapter, we prepend a single one bit to messages. We do this because CRCs ignore leading zero bits, which can be problematic. Our code prepends a one-bit by initializing fingerprints using EMPTY, rather than initializing using zero as in RFC-1952 and the book chapter.)
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
17.2 - Trevni
Version 0.1
DRAFT
This document is the authoritative specification of a file format. Its intent is to permit compatible, independent implementations that read and/or write files in this format.
Introduction
Data sets are often described as a table composed of rows and columns. Each record in the dataset is considered a row, with each field of the record occupying a different column. Writing records to a file one-by-one as they are created results in a row-major format, like Hadoop’s SequenceFile or Avro data files.
In many cases higher query performance may be achieved if the data is instead organized in a column-major format, where multiple values of a given column are stored adjacently. This document defines such a column-major file format for datasets.
To permit scalable, distributed query evaluation, datasets are partitioned into row groups, containing distinct collections of rows. Each row group is organized in column-major order, while row groups form a row-major partitioning of the entire dataset.
Rationale
Goals
The format is meant satisfy the following goals:
Maximize the size of row groups. Disc drives are used most efficiently when sequentially accessing data. Consider a drive that takes 10ms to seek and transfers at 100MB/second. If a 10-column dataset whose values are all the same size is split into 10MB row groups, then accessing a single column will require a sequence of seek+1MB reads, for a cost of 20ms/MB processed. If the same dataset is split into 100MB row groups then this drops to 11ms/MB processed. This effect is exaggerated for datasets with larger numbers of columns and with columns whose values are smaller than average. So we’d prefer row groups that are 100MB or greater.
Permit random access within a row group. Some queries will first examine one column, and, only when certain relatively rare criteria are met, examine other columns. Rather than iterating through selected columns of the row-group in parallel, one might iterate through one column and randomly access another. This is called support for WHERE clauses, after the SQL operator of that name.
Minimize the number of files per dataset. HDFS is a primary intended deployment platform for these files. The HDFS Namenode requires memory for each file in the filesystem, thus for a format to be HDFS-friendly it should strive to require the minimum number of distinct files.
Support co-location of columns within row-groups. Row groups are the unit of parallel operation on a column dataset. For efficient file i/o, the entirety of a row-group should ideally reside on the host that is evaluating the query in order to avoid network latencies and bottlenecks.
Data integrity. The format should permit applications to detect data corruption. Many file systems may prevent corruption, but files may be moved between filesystems and be subject to corruption at points in that process. It is best if the data in a file can be validated independently.
Extensibility. The format should permit applications to store additional annotations about a datasets in the files, such as type information, origin, etc. Some environments may have metadata stores for such information, but not all do, and files might be moved among systems with different metadata systems. The ability to keep such information within the file simplifies the coordination of such information.
Minimal overhead. The column format should not make datasets appreciably larger. Storage is a primary cost and a choice to use this format should not require additional storage.
Primary format. The column format should be usable as a primary format for datasets, not as an auxiliary, accelerated format. Applications that process a dataset in row-major order should be able to easily consume column files and applications that produce datasets in row-major order should be able to easily generate column files.
Design
To meet these goals we propose the following design.
Each row group is a separate file. All values of a column in a file are written contiguously. This maximizes the row group size, optimizing performance when querying few and small columns.
Each file occupies a single HDFS block. A larger than normal block size may be specified, e.g., ~1GB instead of the typical ~100MB. This guarantees co-location and eliminates network use when query processing can be co-located with the file. This also moderates the memory impact on the HDFS Namenode since no small files are written.
Each column in a file is written as a sequence of ~64kB compressed blocks. The sequence is prefixed by a table describing all of the blocks in the column to permit random access within the column.
Application-specific metadata may be added at the file, column, and block levels.
Checksums are included with each block, providing data integrity.
Discussion
The use of a single block per file achieves the same effect as the custom block placement policy described in the CIF paper, but while still permitting HDFS rebalancing and not increasing the number of files in the namespace.
This section formally describes the proposed column file format.
Data Model
We assume a simple data model, where a record is a set of named fields, and the value of each field is a sequence of untyped bytes. A type system may be layered on top of this, as specified in the Type Mapping section below.
Primitive Values
We define the following primitive value types:
- Signed 64-bit long values are written using a variable-length zig-zag coding, where the high-order bit in each byte determines whether subsequent bytes are present. For example:
decimal value hex bytes
| decimal value |
hex bytes |
|
| 0 |
00 |
|
| -1 |
01 |
|
| 1 |
02 |
|
| … |
|
|
| -64 |
7f |
|
| 64 |
80 01 |
|
| … |
|
|
- bytes are encoded as a long followed by that many bytes of data.
- a string is encoded as a long followed by that many bytes of UTF-8 encoded character data.
For example, the three-character string “foo” would be encoded as the long value 3 (encoded as hex 06) followed by the UTF-8 encoding of ‘f’, ‘o’, and ‘o’ (the hex bytes 66 6f 6f): 06 66 6f 6f
Type Names
The following type names are used to describe column values:
- null, requires zero bytes. Sometimes used in array columns.
- boolean, one bit, packed into bytes, little-endian;
- int, like long, but restricted to 32-bit signed values
- long 64-bit signed values, represented as above
- fixed32 32-bit values stored as four bytes, little-endian.
- fixed64 64-bit values stored as eight bytes, little-endian.
- float 32-bit IEEE floating point value, little-endian
- double 64-bit IEEE floating point value, little-endian
- string as above
- bytes as above, may be used to encapsulate more complex objects
Type names are represented as strings (UTF-8 encoded, length-prefixed).
Metadata consists of:
- A long indicating the number of metadata key/value pairs.
- For each pair, a string key and bytes value.
All metadata properties that start with “trevni.” are reserved.
The following file metadata properties are defined:
- trevni.codec the name of the default compression codec used to compress blocks, as a string. Implementations are required to support the “null” codec. Optional. If absent, it is assumed to be “null”. Codecs are described in more detail below.
- trevni.checksum the name of the checksum algorithm used in this file, as a string. Implementations are required to support the “crc-32” checksum. Optional. If absent, it is assumed to be “null”. Checksums are described in more detail below.
The following column metadata properties are defined:
- trevni.codec the name of the compression codec used to compress the blocks of this column, as a string. Implementations are required to support the “null” codec. Optional. If absent, it is assumed to be “null”. Codecs are described in more detail below.
- trevni.name the name of the column, as a string. Required.
- trevni.type the type of data in the column. One of the type names above. Required.
- trevni.values if present, indicates that the initial value of each block in this column will be stored in the block’s descriptor. Not permitted for array columns or columns that specify a parent.
- trevni.array if present, indicates that each row in this column contains a sequence of values of the named type rather than just a single value. An integer length precedes each sequence of values indicating the count of values in the sequence.
- trevni.parent if present, the name of an array column whose lengths are also used by this column. Thus values of this column are sequences but no lengths are stored in this column.
For example, consider the following row, as JSON, where all values are primitive types, but one has multiple values.
{"id"=566, "date"=23423234234
"from"="foo@bar.com",
"to"=["bar@baz.com", "bang@foo.com"],
"content"="Hi!"}
The columns for this might be specified as:
name=id type=int
name=date type=long
name=from type=string
name=to type=string array=true
name=content type=string
If a row contains an array of records, e.g. “received” in the following:
{"id"=566, "date"=23423234234
"from"="foo@bar.com",
"to"=["bar@baz.com", "bang@foo.com"],
"content"="Hi!"
"received"=[{"date"=234234234234, "host"="192.168.0.0.1"},
{"date"=234234545645, "host"="192.168.0.0.2"}]
}
Then one can define a parent column followed by a column for each field in the record, adding the following columns:
name=received type=null array=true
name=date type=long parent=received
name=host type=string parent=received
If an array value itself contains an array, e.g. the “sigs” below:
{"id"=566, "date"=23423234234
"from"="foo@bar.com",
"to"=["bar@baz.com", "bang@foo.com"],
"content"="Hi!"
"received"=[{"date"=234234234234, "host"="192.168.0.0.1",
"sigs"=[{"algo"="weak", "value"="0af345de"}]},
{"date"=234234545645, "host"="192.168.0.0.2",
"sigs"=[]}]
}
Then a parent column may be defined that itself has a parent column.
name=sigs type=null array=true parent=received
name=algo type=string parent=sigs
name=value type=string parent=sigs
No block metadata properties are currently defined.
A file consists of:
- A file header, followed by
- one or more columns.
A file header consists of:
- Four bytes, ASCII ‘T’, ‘r’, ‘v’, followed by 1.
- a fixed64 indicating the number of rows in the file
- a fixed32 indicating the number of columns in the file
- file metadata.
- for each column, its column metadata
- for each column, its starting position in the file as a fixed64.
A column consists of:
- A fixed32 indicating the number of blocks in this column.
- For each block, a block descriptor
- One or more blocks.
A block descriptor consists of:
- A fixed32 indicating the number of rows in the block
- A fixed32 indicating the size in bytes of the block before the codec is applied (excluding checksum).
- A fixed32 indicating the size in bytes of the block after the codec is applied (excluding checksum).
- If this column’s metadata declares it to include values, the first value in the column, serialized according to this column’s type.
A block consists of:
- The serialized column values. If a column is an array column then value sequences are preceded by their length, as an int. If a codec is specified, the values and lengths are compressed by that codec.
- The checksum, as determined by the file metadata.
Codecs
null
The “null” codec simply passes data through uncompressed.
deflate
The “deflate” codec writes the data block using the deflate algorithm as specified in RFC 1951.
snappy
The “snappy” codec uses Google’s Snappy compression library.
Checksum algorithms
null
The “null” checksum contains zero bytes.
crc-32
Each “crc-32” checksum contains the four bytes of an ISO 3309 CRC-32 checksum of the uncompressed block data as a fixed32.
Type Mappings
We define a standard mapping for how types defined in various serialization systems are represented in a column file. Records from these systems are shredded into columns. When records are nested, a depth-first recursive walk can assign a separate column for each primitive value.
Avro
Protocol Buffers
Thrift
Implementation Notes
Some possible techniques for writing column files include:
- Use a standard ~100MB block, buffer in memory up to the block size, then flush the file directly to HDFS. A single reduce task might create multiple output files. The namenode requires memory proportional to the number of names and blocks*replication. This would increase the number of names but not blocks, so this should still be much better than a file per column.
- Spill each column to a separate local, temporary file then, when the file is closed, append these files, writing a single file to HDFS whose block size is set to be that of the entire file. This would be a bit slower than and may have trouble when the local disk is full, but it would better use HDFS namespace and further reduce seeks when processing columns whose values are small.
- Use a separate mapreduce job to convert row-major files to column-major. The map output would output a by (row#, column#, value) tuple, partitioned by row# but sorted by column# then row#. The reducer could directly write the column file. But the column file format would need to be changed to write counts, descriptors, etc. at the end of files rather than at the front.
(1) is the simplest to implement and most implementations should start with it.
References
CIF Column-Oriented Storage Techniques for MapReduce, Floratou, Patel, Shekita, & Tata, VLDB 2011.
DREMEL Dremel: Interactive Analysis of Web-Scale Datasets, Melnik, Gubarev, Long, Romer, Shivakumar, & Tolton, VLDB 2010.
17.3 - IDL Language
Introduction
This document defines Avro IDL, a higher-level language for authoring Avro schemata. Before reading this document, you should have familiarity with the concepts of schemata and protocols, as well as the various primitive and complex types available in Avro.
Overview
Purpose
The aim of the Avro IDL language is to enable developers to author schemata in a way that feels more similar to common programming languages like Java, C++, or Python. Additionally, the Avro IDL language may feel more familiar for those users who have previously used the interface description languages (IDLs) in other frameworks like Thrift, Protocol Buffers, or CORBA.
Usage
Each Avro IDL file defines a single Avro Protocol, and thus generates as its output a JSON-format Avro Protocol file with extension .avpr.
To convert a .avdl file into a .avpr file, it may be processed by the idl tool. For example:
$ java -jar avroj-tools.jar idl src/test/idl/input/namespaces.avdl /tmp/namespaces.avpr
$ head /tmp/namespaces.avpr
{
"protocol" : "TestNamespace",
"namespace" : "avro.test.protocol",
The idl tool can also process input to and from stdin and stdout. See idl --help for full usage information.
A Maven plugin is also provided to compile .avdl files. To use it, add something like the following to your pom.xml:
<build>
<plugins>
<plugin>
<groupId>org.apache.avro</groupId>
<artifactId>avro-maven-plugin</artifactId>
<executions>
<execution>
<goals>
<goal>idl-protocol</goal>
</goals>
</execution>
</executions>
</plugin>
</plugins>
</build>
Defining a Protocol in Avro IDL
An Avro IDL file consists of exactly one protocol definition. The minimal protocol is defined by the following code:
This is equivalent to (and generates) the following JSON protocol definition:
{
"protocol" : "MyProtocol",
"types" : [ ],
"messages" : {
}
}
The namespace of the protocol may be changed using the @namespace annotation:
@namespace("mynamespace")
protocol MyProtocol {
}
This notation is used throughout Avro IDL as a way of specifying properties for the annotated element, as will be described later in this document.
Protocols in Avro IDL can contain the following items:
- Imports of external protocol and schema files.
- Definitions of named schemata, including records, errors, enums, and fixeds.
- Definitions of RPC messages
Imports
Files may be imported in one of three formats:
-
An IDL file may be imported with a statement like:
import idl "foo.avdl";
-
A JSON protocol file may be imported with a statement like:
import protocol "foo.avpr";
-
A JSON schema file may be imported with a statement like:
import schema "foo.avsc";
Messages and types in the imported file are added to this file’s protocol.
Imported file names are resolved relative to the current IDL file.
Defining an Enumeration
Enums are defined in Avro IDL using a syntax similar to C or Java:
enum Suit {
SPADES, DIAMONDS, CLUBS, HEARTS
}
Note that, unlike the JSON format, anonymous enums cannot be defined.
Defining a Fixed Length Field
Fixed fields are defined using the following syntax:
fixed MD5(16);
This example defines a fixed-length type called MD5 which contains 16 bytes.
Defining Records and Errors
Records are defined in Avro IDL using a syntax similar to a struct definition in C:
record Employee {
string name;
boolean active = true;
long salary;
}
The above example defines a record with the name “Employee” with three fields.
To define an error, simply use the keyword error instead of record. For example:
error Kaboom {
string explanation;
int result_code = -1;
}
Each field in a record or error consists of a type and a name, optional property annotations and an optional default value.
A type reference in Avro IDL must be one of:
- A primitive type
- A logical type
- A named schema defined prior to this usage in the same Protocol
- A complex type (array, map, or union)
Primitive Types
The primitive types supported by Avro IDL are the same as those supported by Avro’s JSON format. This list includes int, long, string, boolean, float, double, null, and bytes.
References to Named Schemata
If a named schema has already been defined in the same Avro IDL file, it may be referenced by name as if it were a primitive type:
record Card {
Suit suit; // refers to the enum Card defined above
int number;
}
Default Values
Default values for fields may be optionally specified by using an equals sign after the field name followed by a JSON expression indicating the default value. This JSON is interpreted as described in the spec.
Complex Types
Arrays
Array types are written in a manner that will seem familiar to C++ or Java programmers. An array of any type t is denoted array<t>. For example, an array of strings is denoted array<string>, and a multidimensional array of Foo records would be array<array<Foo>>.
Maps
Map types are written similarly to array types. An array that contains values of type t is written map<t>. As in the JSON schema format, all maps contain string-type keys.
Unions
Union types are denoted as union { typeA, typeB, typeC, … }. For example, this record contains a string field that is optional (unioned with null):
record RecordWithUnion {
union { null, string } optionalString;
}
Note that the same restrictions apply to Avro IDL unions as apply to unions defined in the JSON format; namely, a record may not contain multiple elements of the same type.
Defining RPC Messages
The syntax to define an RPC message within a Avro IDL protocol is similar to the syntax for a method declaration within a C header file or a Java interface. To define an RPC message add which takes two arguments named foo and bar, returning an int, simply include the following definition within the protocol:
int add(int foo, int bar = 0);
Message arguments, like record fields, may specify default values.
To define a message with no response, you may use the alias void, equivalent to the Avro null type:
void logMessage(string message);
If you have previously defined an error type within the same protocol, you may declare that a message can throw this error using the syntax:
void goKaboom() throws Kaboom;
To define a one-way message, use the keyword oneway after the parameter list, for example:
void fireAndForget(string message) oneway;
Other Language Features
All Java-style comments are supported within a Avro IDL file. Any text following // on a line is ignored, as is any text between /* and */, possibly spanning multiple lines.
Comments that begin with /** are used as the documentation string for the type or field definition that follows the comment.
Escaping Identifiers
Occasionally, one will need to use a reserved language keyword as an identifier. In order to do so, backticks (`) may be used to escape the identifier. For example, to define a message with the literal name error, you may write:
This syntax is allowed anywhere an identifier is expected.
Annotations for Ordering and Namespaces
Java-style annotations may be used to add additional properties to types and fields throughout Avro IDL.
For example, to specify the sort order of a field within a record, one may use the @order annotation before the field name as follows:
record MyRecord {
string @order("ascending") myAscendingSortField;
string @order("descending") myDescendingField;
string @order("ignore") myIgnoredField;
}
A field’s type may also be preceded by annotations, e.g.:
record MyRecord {
@java-class("java.util.ArrayList") array<string> myStrings;
}
Similarly, a @namespace annotation may be used to modify the namespace when defining a named schema. For example:
@namespace("org.apache.avro.firstNamespace")
protocol MyProto {
@namespace("org.apache.avro.someOtherNamespace")
record Foo {}
record Bar {}
}
will define a protocol in the firstNamespace namespace. The record Foo will be defined in someOtherNamespace and Bar will be defined in firstNamespace as it inherits its default from its container.
Type and field aliases are specified with the @aliases annotation as follows:
@aliases(["org.old.OldRecord", "org.ancient.AncientRecord"])
record MyRecord {
string @aliases(["oldField", "ancientField"]) myNewField;
}
Some annotations like those listed above are handled specially. All other annotations are added as properties to the protocol, message, schema or field.
Complete Example
The following is a complete example of a Avro IDL file that shows most of the above features:
/**
* An example protocol in Avro IDL
*/
@namespace("org.apache.avro.test")
protocol Simple {
@aliases(["org.foo.KindOf"])
enum Kind {
FOO,
BAR, // the bar enum value
BAZ
}
fixed MD5(16);
record TestRecord {
@order("ignore")
string name;
@order("descending")
Kind kind;
MD5 hash;
union { MD5, null} @aliases(["hash"]) nullableHash;
array<long> arrayOfLongs;
}
error TestError {
string message;
}
string hello(string greeting);
TestRecord echo(TestRecord `record`);
int add(int arg1, int arg2);
bytes echoBytes(bytes data);
void `error`() throws TestError;
void ping() oneway;
}
Additional examples may be found in the Avro source tree under the src/test/idl/input directory.
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
17.4 - SASL profile
Introduction
SASL (RFC 2222) provides a framework for authentication and security of network protocols. Each protocol that uses SASL is meant to define a SASL profile. This document provides a SASL profile for connection-based Avro RPC.
Overview
SASL negotiation proceeds as a series of message interactions over a connection between a client and server using a selected SASL mechanism. The client starts this negotiation by sending its chosen mechanism name with an initial (possibly empty) message. Negotiation proceeds with the exchange of messages until either side indicates success or failure. The content of the messages is mechanism-specific. If the negotiation succeeds, then the session can proceed over the connection, otherwise it must be abandoned.
Some mechanisms continue to process session data after negotiation (e.g., encrypting it), while some specify that further session data is transmitted unmodified.
Negotiation
Commands
Avro SASL negotiation uses four one-byte commands.
- 0: START Used in a client’s initial message.
- 1: CONTINUE Used while negotiation is ongoing.
- 2: FAIL Terminates negotiation unsuccessfully.
- 3: COMPLETE Terminates negotiation successfully.
The format of a START message is:
| 0 | 4-byte mechanism name length | mechanism name | 4-byte payload length | payload data |
The format of a CONTINUE message is:
| 1 | 4-byte payload length | payload data |
The format of a FAIL message is:
| 2 | 4-byte message length | UTF-8 message |
The format of a COMPLETE message is:
| 3 | 4-byte payload length | payload data |
Process
Negotiation is initiated by a client sending a START command containing the client’s chosen mechanism name and any mechanism-specific payload data.
The server and client then interchange some number (possibly zero) of CONTINUE messages. Each message contains payload data that is processed by the security mechanism to generate the next message.
Once either the client or server send a FAIL message then negotiation has failed. UTF-8-encoded text is included in the failure message. Once either a FAIL message has been sent or received, or any other error occurs in the negotiation, further communication on this connection must cease.
Once either the client or server send a COMPLETE message then negotiation has completed successfully. Session data may now be transmitted over the connection until it is closed by either side.
Session Data
If no SASL QOP (quality of protection) is negotiated, then all subsequent writes to/reads over this connection are written/read unmodified. In particular, messages use Avro framing, and are of the form:
| 4-byte frame length | frame data | ... | 4 zero bytes |
If a SASL QOP is negotiated, then it must be used by the connection for all subsequent messages. This is done by wrapping each non-empty frame written using the security mechanism and unwrapping each non-empty frame read. The length written in each non-empty frame is the length of the wrapped data. Complete frames must be passed to the security mechanism for unwrapping. Unwrapped data is then passed to the application as the content of the frame.
If at any point processing fails due to wrapping, unwrapping or framing errors, then all further communication on this connection must cease.
Anonymous Mechanism
The SASL anonymous mechanism (RFC 2245) is quite simple to implement. In particular, an initial anonymous request may be prefixed by the following static sequence:
| 0 | 0009 | ANONYMOUS | 0000 |
If a server uses the anonymous mechanism, it should check that the mechanism name in the start message prefixing the first request received is ‘ANONYMOUS’, then simply prefix its initial response with a COMPLETE message of:
| 3 | 0000 |
If an anonymous server recieves some other mechanism name, then it may respond with a FAIL message as simple as:
| 2 | 0000 |
Note that the anonymous mechanism need add no additional round-trip messages between client and server. The START message can be piggybacked on the initial request and the COMPLETE or FAIL message can be piggybacked on the initial response.
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
17.5 - Wiki
The Wiki page can be found here.
17.6 - FAQ
What is Avro?
Avro is a data serialization system.
Avro provides:
- Rich data structures.
- A compact, fast, binary data format.
- A container file, to store persistent data.
- Remote procedure call (RPC).
- Simple integration with dynamic languages. Code generation is not required to read or write data files nor to use or implement RPC protocols. Code generation as an optional optimization, only worth implementing for statically typed languages.
How can I get started quickly with Avro?
Check out the Quick Start Guide.
How do I statically compile a schema or protocol into generated code?
In Java
- Add the avro jar, the jackson-mapper-asl.jar and jackson-core-asl.jar to your CLASSPATH.
- Run java org.apache.avro.specific.SpecificCompiler .
This appears to be out of date, the SpecificCompiler requires two arguments, presumably an input and and output file, but it isn’t clear that this does.
Or use the Schema or Protocol Ant tasks. Avro’s build.xml provides examples of how these are used.
Lastly, you can also use the “avro-tools” jar which ships with an Avro release. Just use the “compile (schema|protocol)” command.
How are Strings represented in Java?
They use org.apache.avro.util.Utf8, not java.lang.String.
More generally, how do Avro types map to Java types?
The mappings are documented in the package javadoc for generic, specific and reflect API.
From Doug Cutting:
HDFS splits files into blocks, and mapreduce runs a map task for each block. When the task starts, it needs to be able to seek into the file to the start of the block process through the block’s end. If the file were, e.g., a gzip file, this would not be possible, since gzip files must be decompressed from the start. One cannot seek into the middle of a gzip file and start decompressing. So Hadoop’s SequenceFile places a marker periodically (~64k) in the file at record and compression boundaries, where processing can be sensibly started. Then, when a map task starts processing an HDFS block, it finds the first marker after the block’s start and continues through the first marker in the next block of the file. This requires a bit of non-local access (~0.1%). Avro’s data file uses the same method as SequenceFile.
Why isn’t every value in Avro nullable?
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 of every value stored and
corresponding computational costs to create this bit on write and
interpret it on read. These costs are wasted when values may not in
fact be null, as is the case in many datasets. In Avro such costs are
only paid when values may actually be null.
Also, allowing values to be null is a well-known source of errors. In
Avro, a value declared as non-null will always be non-null and programs
need not test for null values when processing it nor will they ever fail
for lack of such tests.
Tony Hoare calls his invention of null references his “Billion Dollar
Mistake”.
http://qconlondon.com/london-2009/presentation/Null+References:+The+Billion+Dollar+Mistake
Also note that in some programming languages not all values are permitted to be null. For example, in Java, values of type boolean, byte, short, char, int, float, long, and double may not be null.
How can I serialize directly to/from a byte array?
As pointed out in the specification, Avro data should always be stored with its schema. The Avro provided classes DataFileWriter, DataFileReader, and DataFileStream all ensure this by serializing the Schema in a container header. In some special cases, such as when implementing a new storage system or writing unit tests, you may need to write and read directly with the bare Avro serialized values. The following examples use code generated for Java from an Avro Schema, specifically the User example from the Quickstart guide.
Serializing to a byte array
This example takes a User object and returns a newly allocated byte array with the Avro serialization of that user.
ByteArrayOutputStream out = new ByteArrayOutputStream();
BinaryEncoder encoder = EncoderFactory.get().binaryEncoder(out, null);
DatumWriter<User> writer = new SpecificDatumWriter<User>(User.getClassSchema());
writer.write(user, encoder);
encoder.flush();
out.close();
byte[] serializedBytes = out.toByteArray();
Deserializing from a byte array
This example takes a byte array containing the Avro serialization of a user and returns a User object.SpecificDatumReader reader = new SpecificDatumReader(User.getClassSchema());
Decoder decoder = DecoderFactory.get().binaryDecoder(bytes, null);
User user = reader.read(null, decoder);
Note: If you are serializing or deserializing in a loop or as a method, you should be reusing objects, readers and/or writers. Check the JavaDoc to see how to reuse the objects.
18 - Apache Avro™ 1.7.1 Documentation
Introduction
Apache Avro™ is a data serialization system.
Avro provides:
- Rich data structures.
- A compact, fast, binary data format.
- A container file, to store persistent data.
- Remote procedure call (RPC).
- Simple integration with dynamic languages. Code generation is not required to read or write data files nor to use or implement RPC protocols. Code generation as an optional optimization, only worth implementing for statically typed languages.
Schemas
Avro relies on schemas. When Avro data is read, the schema used when writing it is always present. This permits each datum to be written with no per-value overheads, making serialization both fast and small. This also facilitates use with dynamic, scripting languages, since data, together with its schema, is fully self-describing.
When Avro data is stored in a file, its schema is stored with it, so that files may be processed later by any program. If the program reading the data expects a different schema this can be easily resolved, since both schemas are present.
When Avro is used in RPC, the client and server exchange schemas in the connection handshake. (This can be optimized so that, for most calls, no schemas are actually transmitted.) Since both client and server both have the other’s full schema, correspondence between same named fields, missing fields, extra fields, etc. can all be easily resolved.
Avro schemas are defined with JSON . This facilitates implementation in languages that already have JSON libraries.
Comparison with other systems
Avro provides functionality similar to systems such as Thrift, Protocol Buffers, etc. Avro differs from these systems in the following fundamental aspects.
- Dynamic typing: Avro does not require that code be generated. Data is always accompanied by a schema that permits full processing of that data without code generation, static datatypes, etc. This facilitates construction of generic data-processing systems and languages.
- Untagged data: Since the schema is present when data is read, considerably less type information need be encoded with data, resulting in smaller serialization size.
- No manually-assigned field IDs: When a schema changes, both the old and new schema are always present when processing data, so differences may be resolved symbolically, using field names.
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
18.1 - Specification
Introduction
This document defines Apache Avro. It is intended to be the authoritative specification. Implementations of Avro must adhere to this document.
Schema Declaration
A Schema is represented in JSON by one of:
- A JSON string, naming a defined type.
- A JSON object, of the form:
{"type": "typeName" ...attributes...}
where typeName is either a primitive or derived type name, as defined below. Attributes not defined in this document are permitted as metadata, but must not affect the format of serialized data.
- A JSON array, representing a union of embedded types.
Primitive Types
The set of primitive type names is:
- null: no value
- boolean: a binary value
- int: 32-bit signed integer
- long: 64-bit signed integer
- float: single precision (32-bit) IEEE 754 floating-point number
- double: double precision (64-bit) IEEE 754 floating-point number
- bytes: sequence of 8-bit unsigned bytes
- string: unicode character sequence
Primitive types have no specified attributes.
Primitive type names are also defined type names. Thus, for example, the schema “string” is equivalent to:
Complex Types
Avro supports six kinds of complex types: records, enums, arrays, maps, unions and fixed.
Records
Records use the type name “record” and support three attributes:
- name: a JSON string providing the name of the record (required).
- namespace, a JSON string that qualifies the name;
- doc: a JSON string providing documentation to the user of this schema (optional).
- aliases: a JSON array of strings, providing alternate names for this record (optional).
- fields: a JSON array, listing fields (required). Each field is a JSON object with the following attributes:
- name: a JSON string providing the name of the field (required), and
- doc: a JSON string describing this field for users (optional).
- type: A JSON object defining a schema, or a JSON string naming a record definition (required).
- default: A default value for this field, used when reading instances that lack this field (optional). 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. Default values for bytes and fixed fields are JSON strings, where Unicode code points 0-255 are mapped to unsigned 8-bit byte values 0-255.
field default values
| avro type |
json type |
example |
| null |
null |
null |
| boolean |
boolean |
true |
| int,long |
integer |
1 |
| float,double |
number |
1.1 |
| bytes |
string |
"\u00FF" |
| string |
string |
"foo" |
| record |
object |
{"a": 1} |
| enum |
string |
"FOO" |
| array |
array |
[1] |
| map |
object |
{"a": 1} |
| fixed |
string |
"\u00ff" |
- order: specifies how this field impacts sort ordering of this record (optional). Valid values are “ascending” (the default), “descending”, or “ignore”. For more details on how this is used, see the sort order section below.
- aliases: a JSON array of strings, providing alternate names for this field (optional).
For example, a linked-list of 64-bit values may be defined with:
{
"type": "record",
"name": "LongList",
"aliases": ["LinkedLongs"], // old name for this
"fields" : [
{"name": "value", "type": "long"}, // each element has a long
{"name": "next", "type": ["LongList", "null"]} // optional next element
]
}
Enums
Enums use the type name “enum” and support the following attributes:
- name: a JSON string providing the name of the enum (required).
- namespace, a JSON string that qualifies the name ;
- aliases: a JSON array of strings, providing alternate names for this enum (optional).
- doc: a JSON string providing documentation to the user of this schema (optional).
- symbols: a JSON array, listing symbols, as JSON strings (required). All symbols in an enum must be unique; duplicates are prohibited.
For example, playing card suits might be defined with:
{
"type": "enum",
"name": "Suit",
"symbols" : ["SPADES", "HEARTS", "DIAMONDS", "CLUBS"]
}
Arrays
Arrays use the type name “array” and support a single attribute:
- items: the schema of the array’s items.
For example, an array of strings is declared with:
{
"type": "array",
"items" : "string"
}
Maps
Maps use the type name “map” and support one attribute:
- values: the schema of the map’s values.
Map keys are assumed to be strings.
For example, a map from string to long is declared with:
{
"type": "map",
"values" : "long"
}
Unions
Unions, as mentioned above, are represented using JSON arrays. For example, ["string", "null"] declares a schema which may be either a string or null.
Unions may not contain more than one schema with the same type, except for the named types record, fixed and enum. For example, unions containing two array types or two map types are not permitted, but two types with different names are permitted. (Names permit efficient resolution when reading and writing unions.)
Unions may not immediately contain other unions.
Fixed
Fixed uses the type name “fixed” and supports two attributes:
- name: a string naming this fixed (required).
- namespace, a string that qualifies the name;
- aliases: a JSON array of strings, providing alternate names for this enum (optional).
- size: an integer, specifying the number of bytes per value (required).
For example, 16-byte quantity may be declared with:
{"type": "fixed", "size": 16, "name": "md5"}
Names
Record, enums and fixed are named types. Each has a fullname that is composed of two parts; a name and a namespace. Equality of names is defined on the fullname.
The name portion of a fullname, record field names, and enum symbols must:
- start with [A-Za-z_]
- subsequently contain only [A-Za-z0-9_]
A namespace is a dot-separated sequence of such names. Equality of names (including field names and enum symbols) as well as fullnames is case-sensitive.
In record, enum and fixed definitions, the fullname is determined in one of the following ways:
A name and namespace are both specified. For example, one might use “name”: “X”, “namespace”: “org.foo” to indicate the fullname org.foo.X.
-
A fullname is specified. If the name specified contains a dot, then it is assumed to be a fullname, and any namespace also specified is ignored. For example, use “name”: “org.foo.X” to indicate the fullname org.foo.X.
-
A name only is specified, i.e., a name that contains no dots. In this case the namespace is taken from the most tightly enclosing schema or protocol. For example, if “name”: “X” is specified, and this occurs within a field of the record definition of org.foo.Y, then the fullname is org.foo.X.
References to previously defined names are as in the latter two cases above: if they contain a dot they are a fullname, if they do not contain a dot, the namespace is the namespace of the enclosing definition.
Primitive type names have no namespace and their names may not be defined in any namespace.
A schema or protocol may not contain multiple definitions of a fullname. Further, a name must be defined before it is used (“before” in the depth-first, left-to-right traversal of the JSON parse tree, where the types attribute of a protocol is always deemed to come “before” the messages attribute.)
Aliases
Named types and fields may have aliases. An implementation may optionally use aliases to map a writer’s schema to the reader’s. This facilitates both schema evolution as well as processing disparate datasets.
Aliases function by re-writing the writer’s schema using aliases from the reader’s schema. For example, if the writer’s schema was named “Foo” and the reader’s schema is named “Bar” and has an alias of “Foo”, then the implementation would act as though “Foo” were named “Bar” when reading. Similarly, if data was written as a record with a field named “x” and is read as a record with a field named “y” with alias “x”, then the implementation would act as though “x” were named “y” when reading.
A type alias may be specified either as a fully namespace-qualified, or relative to the namespace of the name it is an alias for. For example, if a type named “a.b” has aliases of “c” and “x.y”, then the fully qualified names of its aliases are “a.c” and “x.y”.
Data Serialization
Avro data is always serialized with its schema. Files that store Avro data should always also include the schema for that data in the same file. Avro-based remote procedure call (RPC) systems must also guarantee that remote recipients of data have a copy of the schema used to write that data.
Because the schema used to write data is always available when the data is read, Avro data itself is not tagged with type information. The schema is required to parse data.
In general, both serialization and deserialization proceed as a depth-first, left-to-right traversal of the schema, serializing primitive types as they are encountered.
Encodings
Avro specifies two serialization encodings: binary and JSON. Most applications will use the binary encoding, as it is smaller and faster. But, for debugging and web-based applications, the JSON encoding may sometimes be appropriate.
Binary Encoding
Primitive Types
Primitive types are encoded in binary as follows:
- null is written as zero bytes.
- a boolean is written as a single byte whose value is either 0 (false) or 1 (true).
- int and long values are written using variable-length zig-zag coding. Some examples:
| value |
hex |
| 0 |
00 |
| -1 |
01 |
| 1 |
02 |
| -2 |
03 |
| 2 |
04 |
| … |
|
| -64 |
7f |
| 64 |
80 01 |
| … |
|
- a float is written as 4 bytes. The float is converted into a 32-bit integer using a method equivalent to Java’s floatToIntBits and then encoded in little-endian format.
- a double is written as 8 bytes. The double is converted into a 64-bit integer using a method equivalent to Java’s doubleToLongBits and then encoded in little-endian format.
- bytes are encoded as a long followed by that many bytes of data.
- a string is encoded as a long followed by that many bytes of UTF-8 encoded character data.
For example, the three-character string “foo” would be encoded as the long value 3 (encoded as hex 06) followed by the UTF-8 encoding of ‘f’, ‘o’, and ‘o’ (the hex bytes 66 6f 6f):
06 66 6f 6f
Complex Types
Complex types are encoded in binary as follows:
Records
A record is encoded by encoding the values of its fields in the order that they are declared. In other words, a record is encoded as just the concatenation of the encodings of its fields. Field values are encoded per their schema.
For example, the record schema
{
"type": "record",
"name": "test",
"fields" : [
{"name": "a", "type": "long"},
{"name": "b", "type": "string"}
]
}
An instance of this record whose a field has value 27 (encoded as hex 36) and whose b field has value “foo” (encoded as hex bytes 06 66 6f 6f), would be encoded simply as the concatenation of these, namely the hex byte sequence:
36 06 66 6f 6f
Enums
An enum is encoded by a int, representing the zero-based position of the symbol in the schema.
For example, consider the enum:
{"type": "enum", "name": "Foo", "symbols": ["A", "B", "C", "D"] }
This would be encoded by an int between zero and three, with zero indicating “A”, and 3 indicating “D”.
Arrays
Arrays are encoded as a series of blocks. Each block consists of a long count value, followed by that many array items. A block with count zero indicates the end of the array. Each item is encoded per the array’s item schema.
If a block’s count is negative, its absolute value is used, and the count is followed immediately by a long block size indicating the number of bytes in the block. This block size permits fast skipping through data, e.g., when projecting a record to a subset of its fields.
For example, the array schema
{"type": "array", "items": "long"}
an array containing the items 3 and 27 could be encoded as the long value 2 (encoded as hex 04) followed by long values 3 and 27 (encoded as hex 06 36) terminated by zero:
04 06 36 00
The blocked representation permits one to read and write arrays larger than can be buffered in memory, since one can start writing items without knowing the full length of the array.
Maps
Maps are encoded as a series of blocks. Each block consists of a long count value, followed by that many key/value pairs. A block with count zero indicates the end of the map. Each item is encoded per the map’s value schema.
If a block’s count is negative, its absolute value is used, and the count is followed immediately by a long block size indicating the number of bytes in the block. This block size permits fast skipping through data, e.g., when projecting a record to a subset of its fields.
The blocked representation permits one to read and write maps larger than can be buffered in memory, since one can start writing items without knowing the full length of the map.
Unions
A union is encoded by first writing a long value indicating the zero-based position within the union of the schema of its value. The value is then encoded per the indicated schema within the union.
For example, the union schema ["string","null"] would encode:
-
null as the integer 1 (the index of “null” in the union, encoded as hex 02):
02
-
the string “a” as zero (the index of “string” in the union), followed by the serialized string:
02 02 61
Fixed
Fixed instances are encoded using the number of bytes declared in the schema.
JSON Encoding
Except for unions, the JSON encoding is the same as is used to encode field default values.
The value of a union is encoded in JSON as follows:
- if its type is null, then it is encoded as a JSON null;
- otherwise it is encoded as a JSON object with one name/value pair whose name is the type’s name and whose value is the recursively encoded value. For Avro’s named types (record, fixed or enum) the user-specified name is used, for other types the type name is used.
For example, the union schema ["null","string","Foo"], where Foo is a record name, would encode:
- null as null;
- the string “a” as
{"string": "a"} and
- a Foo instance as
{"Foo": {...}}, where {...} indicates the JSON encoding of a Foo instance.
Note that a schema is still required to correctly process JSON-encoded data. For example, the JSON encoding does not distinguish between int and long, float and double, records and maps, enums and strings, etc.
Sort Order
Avro defines a standard sort order for data. This permits data written by one system to be efficiently sorted by another system. This can be an important optimization, as sort order comparisons are sometimes the most frequent per-object operation. Note also that Avro binary-encoded data can be efficiently ordered without deserializing it to objects.
Data items may only be compared if they have identical schemas. Pairwise comparisons are implemented recursively with a depth-first, left-to-right traversal of the schema. The first mismatch encountered determines the order of the items.
Two items with the same schema are compared according to the following rules.
- null data is always equal.
- boolean data is ordered with false before true.
- int, long, float and double data is ordered by ascending numeric value.
- bytes and fixed data are compared lexicographically by unsigned 8-bit values.
- string data is compared lexicographically by Unicode code point. Note that since UTF-8 is used as the binary encoding for strings, sorting of bytes and string binary data is identical.
- array data is compared lexicographically by element.
- enum data is ordered by the symbol’s position in the enum schema. For example, an enum whose symbols are
["z", "a"] would sort “z” values before “a” values.
- union data is first ordered by the branch within the union, and, within that, by the type of the branch. For example, an
["int", "string"] union would order all int values before all string values, with the ints and strings themselves ordered as defined above.
- record data is ordered lexicographically by field. If a field specifies that its order is:
- “ascending”, then the order of its values is unaltered.
- “descending”, then the order of its values is reversed.
- “ignore”, then its values are ignored when sorting.
- map data may not be compared. It is an error to attempt to compare data containing maps unless those maps are in an
"order":"ignore" record field.
Object Container Files
Avro includes a simple object container file format. A file has a schema, and all objects stored in the file must be written according to that schema, using binary encoding. Objects are stored in blocks that may be compressed. Syncronization markers are used between blocks to permit efficient splitting of files for MapReduce processing.
Files may include arbitrary user-specified metadata.
A file consists of:
- A file header, followed by
- one or more file data blocks.
A file header consists of:
- Four bytes, ASCII ‘O’, ‘b’, ‘j’, followed by 1.
- file metadata, including the schema.
- The 16-byte, randomly-generated sync marker for this file.
File metadata consists of:
A long indicating the number of metadata key/value pairs.
For each pair, a string key and bytes value.
All metadata properties that start with “avro.” are reserved. The following file metadata properties are currently used:
- avro.schema contains the schema of objects stored in the file, as JSON data (required).
- avro.codec the name of the compression codec used to compress blocks, as a string. Implementations are required to support the following codecs: “null” and “deflate”. If codec is absent, it is assumed to be “null”. The codecs are described with more detail below.
A file header is thus described by the following schema:
{"type": "record", "name": "org.apache.avro.file.Header",
"fields" : [
{"name": "magic", "type": {"type": "fixed", "name": "Magic", "size": 4}},
{"name": "meta", "type": {"type": "map", "values": "bytes"}},
{"name": "sync", "type": {"type": "fixed", "name": "Sync", "size": 16}}
]
}
A file data block consists of:
- A long indicating the count of objects in this block.
- A long indicating the size in bytes of the serialized objects in the current block, after any codec is applied
- The serialized objects. If a codec is specified, this is compressed by that codec.
- The file’s 16-byte sync marker.
Thus, each block’s binary data can be efficiently extracted or skipped without deserializing the contents. The combination of block size, object counts, and sync markers enable detection of corrupt blocks and help ensure data integrity.
Required Codecs
null
The “null” codec simply passes through data uncompressed.
deflate
The “deflate” codec writes the data block using the deflate algorithm as specified in RFC 1951, and typically implemented using the zlib library. Note that this format (unlike the “zlib format” in RFC 1950) does not have a checksum.
Optional Codecs
snappy
The “snappy” codec uses Google’s Snappy compression library. Each compressed block is followed by the 4-byte, big-endian CRC32 checksum of the uncompressed data in the block.
Protocol Declaration
Avro protocols describe RPC interfaces. Like schemas, they are defined with JSON text.
A protocol is a JSON object with the following attributes:
- protocol, a string, the name of the protocol (required);
- namespace, an optional string that qualifies the name;
- doc, an optional string describing this protocol;
- types, an optional list of definitions of named types (records, enums, fixed and errors). An error definition is just like a record definition except it uses “error” instead of “record”. Note that forward references to named types are not permitted.
- messages, an optional JSON object whose keys are message names and whose values are objects whose attributes are described below. No two messages may have the same name.
The name and namespace qualification rules defined for schema objects apply to protocols as well.
Messages
A message has attributes:
- a doc, an optional description of the message,
- a request, a list of named, typed parameter schemas (this has the same form as the fields of a record declaration);
- a response schema;
- an optional union of declared error schemas. The effective union has “string” prepended to the declared union, to permit transmission of undeclared “system” errors. For example, if the declared error union is
["AccessError"], then the effective union is ["string", "AccessError"]. When no errors are declared, the effective error union is ["string"]. Errors are serialized using the effective union; however, a protocol’s JSON declaration contains only the declared union.
- an optional one-way boolean parameter.
A request parameter list is processed equivalently to an anonymous record. Since record field lists may vary between reader and writer, request parameters may also differ between the caller and responder, and such differences are resolved in the same manner as record field differences.
The one-way parameter may only be true when the response type is "null" and no errors are listed.
Sample Protocol
For example, one may define a simple HelloWorld protocol with:
{
"namespace": "com.acme",
"protocol": "HelloWorld",
"doc": "Protocol Greetings",
"types": [
{"name": "Greeting", "type": "record", "fields": [
{"name": "message", "type": "string"}]},
{"name": "Curse", "type": "error", "fields": [
{"name": "message", "type": "string"}]}
],
"messages": {
"hello": {
"doc": "Say hello.",
"request": [{"name": "greeting", "type": "Greeting" }],
"response": "Greeting",
"errors": ["Curse"]
}
}
}
Message Transport
Messages may be transmitted via different transport mechanisms.
To the transport, a message is an opaque byte sequence.
A transport is a system that supports:
- transmission of request messages
- receipt of corresponding response messages
Servers may send a response message back to the client corresponding to a request message. The mechanism of correspondance is transport-specific. For example, in HTTP it is implicit, since HTTP directly supports requests and responses. But a transport that multiplexes many client threads over a single socket would need to tag messages with unique identifiers.
Transports may be either stateless or stateful. In a stateless transport, messaging assumes no established connection state, while stateful transports establish connections that may be used for multiple messages. This distinction is discussed further in the handshake section below.
HTTP as Transport
When HTTP is used as a transport, each Avro message exchange is an HTTP request/response pair. All messages of an Avro protocol should share a single URL at an HTTP server. Other protocols may also use that URL. Both normal and error Avro response messages should use the 200 (OK) response code. The chunked encoding may be used for requests and responses, but, regardless the Avro request and response are the entire content of an HTTP request and response. The HTTP Content-Type of requests and responses should be specified as “avro/binary”. Requests should be made using the POST method.
HTTP is used by Avro as a stateless transport.
Message Framing
Avro messages are framed as a list of buffers.
Framing is a layer between messages and the transport. It exists to optimize certain operations.
The format of framed message data is:
- a series of buffers, where each buffer consists of:
- a four-byte, big-endian buffer length, followed by
- that many bytes of buffer data.
- A message is always terminated by a zero-lenghted buffer.
Framing is transparent to request and response message formats (described below). Any message may be presented as a single or multiple buffers.
Framing can permit readers to more efficiently get different buffers from different sources and for writers to more efficiently store different buffers to different destinations. In particular, it can reduce the number of times large binary objects are copied. For example, if an RPC parameter consists of a megabyte of file data, that data can be copied directly to a socket from a file descriptor, and, on the other end, it could be written directly to a file descriptor, never entering user space.
A simple, recommended, framing policy is for writers to create a new segment whenever a single binary object is written that is larger than a normal output buffer. Small objects are then appended in buffers, while larger objects are written as their own buffers. When a reader then tries to read a large object the runtime can hand it an entire buffer directly, without having to copy it.
Handshake
The purpose of the handshake is to ensure that the client and the server have each other’s protocol definition, so that the client can correctly deserialize responses, and the server can correctly deserialize requests. Both clients and servers should maintain a cache of recently seen protocols, so that, in most cases, a handshake will be completed without extra round-trip network exchanges or the transmission of full protocol text.
RPC requests and responses may not be processed until a handshake has been completed. With a stateless transport, all requests and responses are prefixed by handshakes. With a stateful transport, handshakes are only attached to requests and responses until a successful handshake response has been returned over a connection. After this, request and response payloads are sent without handshakes for the lifetime of that connection.
The handshake process uses the following record schemas:
{
"type": "record",
"name": "HandshakeRequest", "namespace":"org.apache.avro.ipc",
"fields": [
{"name": "clientHash",
"type": {"type": "fixed", "name": "MD5", "size": 16}},
{"name": "clientProtocol", "type": ["null", "string"]},
{"name": "serverHash", "type": "MD5"},
{"name": "meta", "type": ["null", {"type": "map", "values": "bytes"}]}
]
}
{
"type": "record",
"name": "HandshakeResponse", "namespace": "org.apache.avro.ipc",
"fields": [
{"name": "match",
"type": {"type": "enum", "name": "HandshakeMatch",
"symbols": ["BOTH", "CLIENT", "NONE"]}},
{"name": "serverProtocol",
"type": ["null", "string"]},
{"name": "serverHash",
"type": ["null", {"type": "fixed", "name": "MD5", "size": 16}]},
{"name": "meta",
"type": ["null", {"type": "map", "values": "bytes"}]}
]
}
- A client first prefixes each request with a
HandshakeRequest containing just the hash of its protocol and of the server’s protocol (clientHash!=null, clientProtocol=null, serverHash!=null), where the hashes are 128-bit MD5 hashes of the JSON protocol text. If a client has never connected to a given server, it sends its hash as a guess of the server’s hash, otherwise it sends the hash that it previously obtained from this server.
The server responds with a HandshakeResponse containing one of:
-
match=BOTH, serverProtocol=null, serverHash=null if the client sent the valid hash of the server’s protocol and the server knows what protocol corresponds to the client’s hash. In this case, the request is complete and the response data immediately follows the HandshakeResponse.
-
match=CLIENT, serverProtocol!=null, serverHash!=null if the server has previously seen the client’s protocol, but the client sent an incorrect hash of the server’s protocol. The request is complete and the response data immediately follows the HandshakeResponse. The client must use the returned protocol to process the response and should also cache that protocol and its hash for future interactions with this server.
-
match=NONE if the server has not previously seen the client’s protocol. The serverHash and serverProtocol may also be non-null if the server’s protocol hash was incorrect.
In this case the client must then re-submit its request with its protocol text (clientHash!=null, clientProtocol!=null, serverHash!=null) and the server should respond with a successful match (match=BOTH, serverProtocol=null, serverHash=null) as above.
The meta field is reserved for future handshake enhancements.
A call consists of a request message paired with its resulting response or error message. Requests and responses contain extensible metadata, and both kinds of messages are framed as described above.
The format of a call request is:
- request metadata, a map with values of type bytes
- the message name, an Avro string, followed by
- the message parameters. Parameters are serialized according to the message’s request declaration.
When a message is declared one-way and a stateful connection has been established by a successful handshake response, no response data is sent. Otherwise the format of the call response is:
- response metadata, a map with values of type bytes
- a one-byte error flag boolean, followed by either:
- if the error flag is false, the message response, serialized per the message’s response schema.
- if the error flag is true, the error, serialized per the message’s effective error union schema.
Schema Resolution
A reader of Avro data, whether from an RPC or a file, can always parse that data because its schema is provided. But that schema may not be exactly the schema that was expected. For example, if the data was written with a different version of the software than it is read, then records may have had fields added or removed. This section specifies how such schema differences should be resolved.
We refer to the schema used to write the data as the writer’s schema, and the schema that the application expects the reader’s schema. Differences between these should be resolved as follows:
- It is an error if the two schemas do not match.
To match, one of the following must hold:
-
both schemas are arrays whose item types match
-
both schemas are maps whose value types match
-
both schemas are enums whose names match
-
both schemas are fixed whose sizes and names match
-
both schemas are records with the same name
-
either schema is a union
-
both schemas have same primitive type
-
the writer’s schema may be promoted to the reader’s as follows:
- int is promotable to long, float, or double
- long is promotable to float or double
- float is promotable to double
- string is promotable to bytes
- bytes is promotable to string
-
if both are records:
- the ordering of fields may be different: fields are matched by name.
- schemas for fields with the same name in both records are resolved recursively.
- if the writer’s record contains a field with a name not present in the reader’s record, the writer’s value for that field is ignored.
- if the reader’s record schema has a field that contains a default value, and writer’s schema does not have a field with the same name, then the reader should use the default value from its field.
- if the reader’s record schema has a field with no default value, and writer’s schema does not have a field with the same name, an error is signalled.
-
if both are enums:
if the writer’s symbol is not present in the reader’s enum, then an error is signalled.
This resolution algorithm is applied recursively to the reader’s and writer’s array item schemas.
This resolution algorithm is applied recursively to the reader’s and writer’s value schemas.
The first schema in the reader’s union that matches the selected writer’s union schema is recursively resolved against it. if none match, an error is signalled.
- if reader’s is a union, but writer’s is not
The first schema in the reader’s union that matches the writer’s schema is recursively resolved against it. If none match, an error is signalled.
- if writer’s is a union, but reader’s is not
If the reader’s schema matches the selected writer’s schema, it is recursively resolved against it. If they do not match, an error is signalled.
A schema’s doc fields are ignored for the purposes of schema resolution. Hence, the doc portion of a schema may be dropped at serialization.
One of the defining characteristics of Avro is that a reader is assumed to have the “same” schema used by the writer of the data the reader is reading. This assumption leads to a data format that’s compact and also amenable to many forms of schema evolution. However, the specification so far has not defined what it means for the reader to have the “same” schema as the writer. Does the schema need to be textually identical? Well, clearly adding or removing some whitespace to a JSON expression does not change its meaning. At the same time, reordering the fields of records clearly does change the meaning. So what does it mean for a reader to have “the same” schema as a writer?
Parsing Canonical Form is a transformation of a writer’s schema that let’s us define what it means for two schemas to be “the same” for the purpose of reading data written against the schema. It is called Parsing Canonical Form because the transformations strip away parts of the schema, like “doc” attributes, that are irrelevant to readers trying to parse incoming data. It is called Canonical Form because the transformations normalize the JSON text (such as the order of attributes) in a way that eliminates unimportant differences between schemas. If the Parsing Canonical Forms of two different schemas are textually equal, then those schemas are “the same” as far as any reader is concerned, i.e., there is no serialized data that would allow a reader to distinguish data generated by a writer using one of the original schemas from data generated by a writing using the other original schema. (We sketch a proof of this property in a companion document.)
The next subsection specifies the transformations that define Parsing Canonical Form. But with a well-defined canonical form, it can be convenient to go one step further, transforming these canonical forms into simple integers (“fingerprints”) that can be used to uniquely identify schemas. The subsection after next recommends some standard practices for generating such fingerprints.
Assuming an input schema (in JSON form) that’s already UTF-8 text for a valid Avro schema (including all quotes as required by JSON), the following transformations will produce its Parsing Canonical Form:
- [PRIMITIVES] Convert primitive schemas to their simple form (e.g., int instead of
{"type":"int"}).
- [FULLNAMES] Replace short names with fullnames, using applicable namespaces to do so. Then eliminate namespace attributes, which are now redundant.
- [STRIP] Keep only attributes that are relevant to parsing data, which are: type, name, fields, symbols, items, values, size. Strip all others (e.g., doc and aliases).
- [ORDER] Order the appearance of fields of JSON objects as follows: name, type, fields, symbols, items, values, size. For example, if an object has type, name, and size fields, then the name field should appear first, followed by the type and then the size fields.
- [STRINGS] For all JSON string literals in the schema text, replace any escaped characters (e.g., \uXXXX escapes) with their UTF-8 equivalents.
- [INTEGERS] Eliminate quotes around and any leading zeros in front of JSON integer literals (which appear in the size attributes of fixed schemas).
- [WHITESPACE] Eliminate all whitespace in JSON outside of string literals.
Schema Fingerprints
“[A] fingerprinting algorithm is a procedure that maps an arbitrarily large data item (such as a computer file) to a much shorter bit string, its fingerprint, that uniquely identifies the original data for all practical purposes” (quoted from Wikipedia). In the Avro context, fingerprints of Parsing Canonical Form can be useful in a number of applications; for example, to cache encoder and decoder objects, to tag data items with a short substitute for the writer’s full schema, and to quickly negotiate common-case schemas between readers and writers.
In designing fingerprinting algorithms, there is a fundamental trade-off between the length of the fingerprint and the probability of collisions. To help application designers find appropriate points within this trade-off space, while encouraging interoperability and ease of implementation, we recommend using one of the following three algorithms when fingerprinting Avro schemas:
- When applications can tolerate longer fingerprints, we recommend using the SHA-256 digest algorithm to generate 256-bit fingerprints of Parsing Canonical Forms. Most languages today have SHA-256 implementations in their libraries.
- At the opposite extreme, the smallest fingerprint we recommend is a 64-bit Rabin fingerprint. Below, we provide pseudo-code for this algorithm that can be easily translated into any programming language. 64-bit fingerprints should guarantee uniqueness for schema caches of up to a million entries (for such a cache, the chance of a collision is 3E-8). We don’t recommend shorter fingerprints, as the chances of collisions is too great (for example, with 32-bit fingerprints, a cache with as few as 100,000 schemas has a 50% chance of having a collision).
- Between these two extremes, we recommend using the MD5 message digest to generate 128-bit fingerprints. These make sense only where very large numbers of schemas are being manipulated (tens of millions); otherwise, 64-bit fingerprints should be sufficient. As with SHA-256, MD5 implementations are found in most libraries today.
These fingerprints are not meant to provide any security guarantees, even the longer SHA-256-based ones. Most Avro applications should be surrounded by security measures that prevent attackers from writing random data and otherwise interfering with the consumers of schemas. We recommend that these surrounding mechanisms be used to prevent collision and pre-image attacks (i.e., “forgery”) on schema fingerprints, rather than relying on the security properties of the fingerprints themselves.
Rabin fingerprints are cyclic redundancy checks computed using irreducible polynomials. In the style of the Appendix of RFC 1952 (pg 10), which defines the CRC-32 algorithm, here’s our definition of the 64-bit AVRO fingerprinting algorithm:
long fingerprint64(byte[] buf) {
if (FP_TABLE == null) initFPTable();
long fp = EMPTY;
for (int i = 0; i < buf.length; i++)
fp = (fp >>> 8) ^ FP_TABLE[(int)(fp ^ buf[i]) & 0xff];
return fp;
}
static long EMPTY = 0xc15d213aa4d7a795L;
static long[] FP_TABLE = null;
void initFPTable() {
FP_TABLE = new long[256];
for (int i = 0; i < 256; i++) {
long fp = i;
for (int j = 0; j < 8; j++)
fp = (fp >>> 1) ^ (EMPTY & -(fp & 1L));
FP_TABLE[i] = fp;
}
}
Readers interested in the mathematics behind this algorithm may want to read this book chapter. (Unlike RFC-1952 and the book chapter, we prepend a single one bit to messages. We do this because CRCs ignore leading zero bits, which can be problematic. Our code prepends a one-bit by initializing fingerprints using EMPTY, rather than initializing using zero as in RFC-1952 and the book chapter.)
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
18.2 - IDL Language
Introduction
This document defines Avro IDL, a higher-level language for authoring Avro schemata. Before reading this document, you should have familiarity with the concepts of schemata and protocols, as well as the various primitive and complex types available in Avro.
Overview
Purpose
The aim of the Avro IDL language is to enable developers to author schemata in a way that feels more similar to common programming languages like Java, C++, or Python. Additionally, the Avro IDL language may feel more familiar for those users who have previously used the interface description languages (IDLs) in other frameworks like Thrift, Protocol Buffers, or CORBA.
Usage
Each Avro IDL file defines a single Avro Protocol, and thus generates as its output a JSON-format Avro Protocol file with extension .avpr.
To convert a .avdl file into a .avpr file, it may be processed by the idl tool. For example:
$ java -jar avroj-tools.jar idl src/test/idl/input/namespaces.avdl /tmp/namespaces.avpr
$ head /tmp/namespaces.avpr
{
"protocol" : "TestNamespace",
"namespace" : "avro.test.protocol",
The idl tool can also process input to and from stdin and stdout. See idl --help for full usage information.
A Maven plugin is also provided to compile .avdl files. To use it, add something like the following to your pom.xml:
<build>
<plugins>
<plugin>
<groupId>org.apache.avro</groupId>
<artifactId>avro-maven-plugin</artifactId>
<executions>
<execution>
<goals>
<goal>idl-protocol</goal>
</goals>
</execution>
</executions>
</plugin>
</plugins>
</build>
Defining a Protocol in Avro IDL
An Avro IDL file consists of exactly one protocol definition. The minimal protocol is defined by the following code:
This is equivalent to (and generates) the following JSON protocol definition:
{
"protocol" : "MyProtocol",
"types" : [ ],
"messages" : {
}
}
The namespace of the protocol may be changed using the @namespace annotation:
@namespace("mynamespace")
protocol MyProtocol {
}
This notation is used throughout Avro IDL as a way of specifying properties for the annotated element, as will be described later in this document.
Protocols in Avro IDL can contain the following items:
- Imports of external protocol and schema files.
- Definitions of named schemata, including records, errors, enums, and fixeds.
- Definitions of RPC messages
Imports
Files may be imported in one of three formats:
-
An IDL file may be imported with a statement like:
import idl "foo.avdl";
-
A JSON protocol file may be imported with a statement like:
import protocol "foo.avpr";
-
A JSON schema file may be imported with a statement like:
import schema "foo.avsc";
Messages and types in the imported file are added to this file’s protocol.
Imported file names are resolved relative to the current IDL file.
Defining an Enumeration
Enums are defined in Avro IDL using a syntax similar to C or Java:
enum Suit {
SPADES, DIAMONDS, CLUBS, HEARTS
}
Note that, unlike the JSON format, anonymous enums cannot be defined.
Defining a Fixed Length Field
Fixed fields are defined using the following syntax:
fixed MD5(16);
This example defines a fixed-length type called MD5 which contains 16 bytes.
Defining Records and Errors
Records are defined in Avro IDL using a syntax similar to a struct definition in C:
record Employee {
string name;
boolean active = true;
long salary;
}
The above example defines a record with the name “Employee” with three fields.
To define an error, simply use the keyword error instead of record. For example:
error Kaboom {
string explanation;
int result_code = -1;
}
Each field in a record or error consists of a type and a name, optional property annotations and an optional default value.
A type reference in Avro IDL must be one of:
- A primitive type
- A logical type
- A named schema defined prior to this usage in the same Protocol
- A complex type (array, map, or union)
Primitive Types
The primitive types supported by Avro IDL are the same as those supported by Avro’s JSON format. This list includes int, long, string, boolean, float, double, null, and bytes.
References to Named Schemata
If a named schema has already been defined in the same Avro IDL file, it may be referenced by name as if it were a primitive type:
record Card {
Suit suit; // refers to the enum Card defined above
int number;
}
Default Values
Default values for fields may be optionally specified by using an equals sign after the field name followed by a JSON expression indicating the default value. This JSON is interpreted as described in the spec.
Complex Types
Arrays
Array types are written in a manner that will seem familiar to C++ or Java programmers. An array of any type t is denoted array<t>. For example, an array of strings is denoted array<string>, and a multidimensional array of Foo records would be array<array<Foo>>.
Maps
Map types are written similarly to array types. An array that contains values of type t is written map<t>. As in the JSON schema format, all maps contain string-type keys.
Unions
Union types are denoted as union { typeA, typeB, typeC, … }. For example, this record contains a string field that is optional (unioned with null):
record RecordWithUnion {
union { null, string } optionalString;
}
Note that the same restrictions apply to Avro IDL unions as apply to unions defined in the JSON format; namely, a record may not contain multiple elements of the same type.
Defining RPC Messages
The syntax to define an RPC message within a Avro IDL protocol is similar to the syntax for a method declaration within a C header file or a Java interface. To define an RPC message add which takes two arguments named foo and bar, returning an int, simply include the following definition within the protocol:
int add(int foo, int bar = 0);
Message arguments, like record fields, may specify default values.
To define a message with no response, you may use the alias void, equivalent to the Avro null type:
void logMessage(string message);
If you have previously defined an error type within the same protocol, you may declare that a message can throw this error using the syntax:
void goKaboom() throws Kaboom;
To define a one-way message, use the keyword oneway after the parameter list, for example:
void fireAndForget(string message) oneway;
Other Language Features
All Java-style comments are supported within a Avro IDL file. Any text following // on a line is ignored, as is any text between /* and */, possibly spanning multiple lines.
Comments that begin with /** are used as the documentation string for the type or field definition that follows the comment.
Escaping Identifiers
Occasionally, one will need to use a reserved language keyword as an identifier. In order to do so, backticks (`) may be used to escape the identifier. For example, to define a message with the literal name error, you may write:
This syntax is allowed anywhere an identifier is expected.
Annotations for Ordering and Namespaces
Java-style annotations may be used to add additional properties to types and fields throughout Avro IDL.
For example, to specify the sort order of a field within a record, one may use the @order annotation before the field name as follows:
record MyRecord {
string @order("ascending") myAscendingSortField;
string @order("descending") myDescendingField;
string @order("ignore") myIgnoredField;
}
A field’s type may also be preceded by annotations, e.g.:
record MyRecord {
@java-class("java.util.ArrayList") array<string> myStrings;
}
Similarly, a @namespace annotation may be used to modify the namespace when defining a named schema. For example:
@namespace("org.apache.avro.firstNamespace")
protocol MyProto {
@namespace("org.apache.avro.someOtherNamespace")
record Foo {}
record Bar {}
}
will define a protocol in the firstNamespace namespace. The record Foo will be defined in someOtherNamespace and Bar will be defined in firstNamespace as it inherits its default from its container.
Type and field aliases are specified with the @aliases annotation as follows:
@aliases(["org.old.OldRecord", "org.ancient.AncientRecord"])
record MyRecord {
string @aliases(["oldField", "ancientField"]) myNewField;
}
Some annotations like those listed above are handled specially. All other annotations are added as properties to the protocol, message, schema or field.
Complete Example
The following is a complete example of a Avro IDL file that shows most of the above features:
/**
* An example protocol in Avro IDL
*/
@namespace("org.apache.avro.test")
protocol Simple {
@aliases(["org.foo.KindOf"])
enum Kind {
FOO,
BAR, // the bar enum value
BAZ
}
fixed MD5(16);
record TestRecord {
@order("ignore")
string name;
@order("descending")
Kind kind;
MD5 hash;
union { MD5, null} @aliases(["hash"]) nullableHash;
array<long> arrayOfLongs;
}
error TestError {
string message;
}
string hello(string greeting);
TestRecord echo(TestRecord `record`);
int add(int arg1, int arg2);
bytes echoBytes(bytes data);
void `error`() throws TestError;
void ping() oneway;
}
Additional examples may be found in the Avro source tree under the src/test/idl/input directory.
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
18.3 - SASL profile
Introduction
SASL (RFC 2222) provides a framework for authentication and security of network protocols. Each protocol that uses SASL is meant to define a SASL profile. This document provides a SASL profile for connection-based Avro RPC.
Overview
SASL negotiation proceeds as a series of message interactions over a connection between a client and server using a selected SASL mechanism. The client starts this negotiation by sending its chosen mechanism name with an initial (possibly empty) message. Negotiation proceeds with the exchange of messages until either side indicates success or failure. The content of the messages is mechanism-specific. If the negotiation succeeds, then the session can proceed over the connection, otherwise it must be abandoned.
Some mechanisms continue to process session data after negotiation (e.g., encrypting it), while some specify that further session data is transmitted unmodified.
Negotiation
Commands
Avro SASL negotiation uses four one-byte commands.
- 0: START Used in a client’s initial message.
- 1: CONTINUE Used while negotiation is ongoing.
- 2: FAIL Terminates negotiation unsuccessfully.
- 3: COMPLETE Terminates negotiation successfully.
The format of a START message is:
| 0 | 4-byte mechanism name length | mechanism name | 4-byte payload length | payload data |
The format of a CONTINUE message is:
| 1 | 4-byte payload length | payload data |
The format of a FAIL message is:
| 2 | 4-byte message length | UTF-8 message |
The format of a COMPLETE message is:
| 3 | 4-byte payload length | payload data |
Process
Negotiation is initiated by a client sending a START command containing the client’s chosen mechanism name and any mechanism-specific payload data.
The server and client then interchange some number (possibly zero) of CONTINUE messages. Each message contains payload data that is processed by the security mechanism to generate the next message.
Once either the client or server send a FAIL message then negotiation has failed. UTF-8-encoded text is included in the failure message. Once either a FAIL message has been sent or received, or any other error occurs in the negotiation, further communication on this connection must cease.
Once either the client or server send a COMPLETE message then negotiation has completed successfully. Session data may now be transmitted over the connection until it is closed by either side.
Session Data
If no SASL QOP (quality of protection) is negotiated, then all subsequent writes to/reads over this connection are written/read unmodified. In particular, messages use Avro framing, and are of the form:
| 4-byte frame length | frame data | ... | 4 zero bytes |
If a SASL QOP is negotiated, then it must be used by the connection for all subsequent messages. This is done by wrapping each non-empty frame written using the security mechanism and unwrapping each non-empty frame read. The length written in each non-empty frame is the length of the wrapped data. Complete frames must be passed to the security mechanism for unwrapping. Unwrapped data is then passed to the application as the content of the frame.
If at any point processing fails due to wrapping, unwrapping or framing errors, then all further communication on this connection must cease.
Anonymous Mechanism
The SASL anonymous mechanism (RFC 2245) is quite simple to implement. In particular, an initial anonymous request may be prefixed by the following static sequence:
| 0 | 0009 | ANONYMOUS | 0000 |
If a server uses the anonymous mechanism, it should check that the mechanism name in the start message prefixing the first request received is ‘ANONYMOUS’, then simply prefix its initial response with a COMPLETE message of:
| 3 | 0000 |
If an anonymous server recieves some other mechanism name, then it may respond with a FAIL message as simple as:
| 2 | 0000 |
Note that the anonymous mechanism need add no additional round-trip messages between client and server. The START message can be piggybacked on the initial request and the COMPLETE or FAIL message can be piggybacked on the initial response.
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
18.4 - Wiki
The Wiki page can be found here.
18.5 - FAQ
What is Avro?
Avro is a data serialization system.
Avro provides:
- Rich data structures.
- A compact, fast, binary data format.
- A container file, to store persistent data.
- Remote procedure call (RPC).
- Simple integration with dynamic languages. Code generation is not required to read or write data files nor to use or implement RPC protocols. Code generation as an optional optimization, only worth implementing for statically typed languages.
How can I get started quickly with Avro?
Check out the Quick Start Guide.
How do I statically compile a schema or protocol into generated code?
In Java
- Add the avro jar, the jackson-mapper-asl.jar and jackson-core-asl.jar to your CLASSPATH.
- Run java org.apache.avro.specific.SpecificCompiler .
This appears to be out of date, the SpecificCompiler requires two arguments, presumably an input and and output file, but it isn’t clear that this does.
Or use the Schema or Protocol Ant tasks. Avro’s build.xml provides examples of how these are used.
Lastly, you can also use the “avro-tools” jar which ships with an Avro release. Just use the “compile (schema|protocol)” command.
How are Strings represented in Java?
They use org.apache.avro.util.Utf8, not java.lang.String.
More generally, how do Avro types map to Java types?
The mappings are documented in the package javadoc for generic, specific and reflect API.
From Doug Cutting:
HDFS splits files into blocks, and mapreduce runs a map task for each block. When the task starts, it needs to be able to seek into the file to the start of the block process through the block’s end. If the file were, e.g., a gzip file, this would not be possible, since gzip files must be decompressed from the start. One cannot seek into the middle of a gzip file and start decompressing. So Hadoop’s SequenceFile places a marker periodically (~64k) in the file at record and compression boundaries, where processing can be sensibly started. Then, when a map task starts processing an HDFS block, it finds the first marker after the block’s start and continues through the first marker in the next block of the file. This requires a bit of non-local access (~0.1%). Avro’s data file uses the same method as SequenceFile.
Why isn’t every value in Avro nullable?
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 of every value stored and
corresponding computational costs to create this bit on write and
interpret it on read. These costs are wasted when values may not in
fact be null, as is the case in many datasets. In Avro such costs are
only paid when values may actually be null.
Also, allowing values to be null is a well-known source of errors. In
Avro, a value declared as non-null will always be non-null and programs
need not test for null values when processing it nor will they ever fail
for lack of such tests.
Tony Hoare calls his invention of null references his “Billion Dollar
Mistake”.
http://qconlondon.com/london-2009/presentation/Null+References:+The+Billion+Dollar+Mistake
Also note that in some programming languages not all values are permitted to be null. For example, in Java, values of type boolean, byte, short, char, int, float, long, and double may not be null.
How can I serialize directly to/from a byte array?
As pointed out in the specification, Avro data should always be stored with its schema. The Avro provided classes DataFileWriter, DataFileReader, and DataFileStream all ensure this by serializing the Schema in a container header. In some special cases, such as when implementing a new storage system or writing unit tests, you may need to write and read directly with the bare Avro serialized values. The following examples use code generated for Java from an Avro Schema, specifically the User example from the Quickstart guide.
Serializing to a byte array
This example takes a User object and returns a newly allocated byte array with the Avro serialization of that user.
ByteArrayOutputStream out = new ByteArrayOutputStream();
BinaryEncoder encoder = EncoderFactory.get().binaryEncoder(out, null);
DatumWriter<User> writer = new SpecificDatumWriter<User>(User.getClassSchema());
writer.write(user, encoder);
encoder.flush();
out.close();
byte[] serializedBytes = out.toByteArray();
Deserializing from a byte array
This example takes a byte array containing the Avro serialization of a user and returns a User object.SpecificDatumReader reader = new SpecificDatumReader(User.getClassSchema());
Decoder decoder = DecoderFactory.get().binaryDecoder(bytes, null);
User user = reader.read(null, decoder);
Note: If you are serializing or deserializing in a loop or as a method, you should be reusing objects, readers and/or writers. Check the JavaDoc to see how to reuse the objects.
19 - Apache Avro™ 1.7.0 Documentation
Introduction
Apache Avro™ is a data serialization system.
Avro provides:
- Rich data structures.
- A compact, fast, binary data format.
- A container file, to store persistent data.
- Remote procedure call (RPC).
- Simple integration with dynamic languages. Code generation is not required to read or write data files nor to use or implement RPC protocols. Code generation as an optional optimization, only worth implementing for statically typed languages.
Schemas
Avro relies on schemas. When Avro data is read, the schema used when writing it is always present. This permits each datum to be written with no per-value overheads, making serialization both fast and small. This also facilitates use with dynamic, scripting languages, since data, together with its schema, is fully self-describing.
When Avro data is stored in a file, its schema is stored with it, so that files may be processed later by any program. If the program reading the data expects a different schema this can be easily resolved, since both schemas are present.
When Avro is used in RPC, the client and server exchange schemas in the connection handshake. (This can be optimized so that, for most calls, no schemas are actually transmitted.) Since both client and server both have the other’s full schema, correspondence between same named fields, missing fields, extra fields, etc. can all be easily resolved.
Avro schemas are defined with JSON . This facilitates implementation in languages that already have JSON libraries.
Comparison with other systems
Avro provides functionality similar to systems such as Thrift, Protocol Buffers, etc. Avro differs from these systems in the following fundamental aspects.
- Dynamic typing: Avro does not require that code be generated. Data is always accompanied by a schema that permits full processing of that data without code generation, static datatypes, etc. This facilitates construction of generic data-processing systems and languages.
- Untagged data: Since the schema is present when data is read, considerably less type information need be encoded with data, resulting in smaller serialization size.
- No manually-assigned field IDs: When a schema changes, both the old and new schema are always present when processing data, so differences may be resolved symbolically, using field names.
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
19.1 - Specification
Introduction
This document defines Apache Avro. It is intended to be the authoritative specification. Implementations of Avro must adhere to this document.
Schema Declaration
A Schema is represented in JSON by one of:
- A JSON string, naming a defined type.
- A JSON object, of the form:
{"type": "typeName" ...attributes...}
where typeName is either a primitive or derived type name, as defined below. Attributes not defined in this document are permitted as metadata, but must not affect the format of serialized data.
- A JSON array, representing a union of embedded types.
Primitive Types
The set of primitive type names is:
- null: no value
- boolean: a binary value
- int: 32-bit signed integer
- long: 64-bit signed integer
- float: single precision (32-bit) IEEE 754 floating-point number
- double: double precision (64-bit) IEEE 754 floating-point number
- bytes: sequence of 8-bit unsigned bytes
- string: unicode character sequence
Primitive types have no specified attributes.
Primitive type names are also defined type names. Thus, for example, the schema “string” is equivalent to:
Complex Types
Avro supports six kinds of complex types: records, enums, arrays, maps, unions and fixed.
Records
Records use the type name “record” and support three attributes:
- name: a JSON string providing the name of the record (required).
- namespace, a JSON string that qualifies the name;
- doc: a JSON string providing documentation to the user of this schema (optional).
- aliases: a JSON array of strings, providing alternate names for this record (optional).
- fields: a JSON array, listing fields (required). Each field is a JSON object with the following attributes:
- name: a JSON string providing the name of the field (required), and
- doc: a JSON string describing this field for users (optional).
- type: A JSON object defining a schema, or a JSON string naming a record definition (required).
- default: A default value for this field, used when reading instances that lack this field (optional). 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. Default values for bytes and fixed fields are JSON strings, where Unicode code points 0-255 are mapped to unsigned 8-bit byte values 0-255.
field default values
| avro type |
json type |
example |
| null |
null |
null |
| boolean |
boolean |
true |
| int,long |
integer |
1 |
| float,double |
number |
1.1 |
| bytes |
string |
"\u00FF" |
| string |
string |
"foo" |
| record |
object |
{"a": 1} |
| enum |
string |
"FOO" |
| array |
array |
[1] |
| map |
object |
{"a": 1} |
| fixed |
string |
"\u00ff" |
- order: specifies how this field impacts sort ordering of this record (optional). Valid values are “ascending” (the default), “descending”, or “ignore”. For more details on how this is used, see the sort order section below.
- aliases: a JSON array of strings, providing alternate names for this field (optional).
For example, a linked-list of 64-bit values may be defined with:
{
"type": "record",
"name": "LongList",
"aliases": ["LinkedLongs"], // old name for this
"fields" : [
{"name": "value", "type": "long"}, // each element has a long
{"name": "next", "type": ["LongList", "null"]} // optional next element
]
}
Enums
Enums use the type name “enum” and support the following attributes:
- name: a JSON string providing the name of the enum (required).
- namespace, a JSON string that qualifies the name ;
- aliases: a JSON array of strings, providing alternate names for this enum (optional).
- doc: a JSON string providing documentation to the user of this schema (optional).
- symbols: a JSON array, listing symbols, as JSON strings (required). All symbols in an enum must be unique; duplicates are prohibited.
For example, playing card suits might be defined with:
{
"type": "enum",
"name": "Suit",
"symbols" : ["SPADES", "HEARTS", "DIAMONDS", "CLUBS"]
}
Arrays
Arrays use the type name “array” and support a single attribute:
- items: the schema of the array’s items.
For example, an array of strings is declared with:
{
"type": "array",
"items" : "string"
}
Maps
Maps use the type name “map” and support one attribute:
- values: the schema of the map’s values.
Map keys are assumed to be strings.
For example, a map from string to long is declared with:
{
"type": "map",
"values" : "long"
}
Unions
Unions, as mentioned above, are represented using JSON arrays. For example, ["string", "null"] declares a schema which may be either a string or null.
Unions may not contain more than one schema with the same type, except for the named types record, fixed and enum. For example, unions containing two array types or two map types are not permitted, but two types with different names are permitted. (Names permit efficient resolution when reading and writing unions.)
Unions may not immediately contain other unions.
Fixed
Fixed uses the type name “fixed” and supports two attributes:
- name: a string naming this fixed (required).
- namespace, a string that qualifies the name;
- aliases: a JSON array of strings, providing alternate names for this enum (optional).
- size: an integer, specifying the number of bytes per value (required).
For example, 16-byte quantity may be declared with:
{"type": "fixed", "size": 16, "name": "md5"}
Names
Record, enums and fixed are named types. Each has a fullname that is composed of two parts; a name and a namespace. Equality of names is defined on the fullname.
The name portion of a fullname, record field names, and enum symbols must:
- start with [A-Za-z_]
- subsequently contain only [A-Za-z0-9_]
A namespace is a dot-separated sequence of such names. Equality of names (including field names and enum symbols) as well as fullnames is case-sensitive.
In record, enum and fixed definitions, the fullname is determined in one of the following ways:
A name and namespace are both specified. For example, one might use “name”: “X”, “namespace”: “org.foo” to indicate the fullname org.foo.X.
-
A fullname is specified. If the name specified contains a dot, then it is assumed to be a fullname, and any namespace also specified is ignored. For example, use “name”: “org.foo.X” to indicate the fullname org.foo.X.
-
A name only is specified, i.e., a name that contains no dots. In this case the namespace is taken from the most tightly enclosing schema or protocol. For example, if “name”: “X” is specified, and this occurs within a field of the record definition of org.foo.Y, then the fullname is org.foo.X.
References to previously defined names are as in the latter two cases above: if they contain a dot they are a fullname, if they do not contain a dot, the namespace is the namespace of the enclosing definition.
Primitive type names have no namespace and their names may not be defined in any namespace.
A schema or protocol may not contain multiple definitions of a fullname. Further, a name must be defined before it is used (“before” in the depth-first, left-to-right traversal of the JSON parse tree, where the types attribute of a protocol is always deemed to come “before” the messages attribute.)
Aliases
Named types and fields may have aliases. An implementation may optionally use aliases to map a writer’s schema to the reader’s. This facilitates both schema evolution as well as processing disparate datasets.
Aliases function by re-writing the writer’s schema using aliases from the reader’s schema. For example, if the writer’s schema was named “Foo” and the reader’s schema is named “Bar” and has an alias of “Foo”, then the implementation would act as though “Foo” were named “Bar” when reading. Similarly, if data was written as a record with a field named “x” and is read as a record with a field named “y” with alias “x”, then the implementation would act as though “x” were named “y” when reading.
A type alias may be specified either as a fully namespace-qualified, or relative to the namespace of the name it is an alias for. For example, if a type named “a.b” has aliases of “c” and “x.y”, then the fully qualified names of its aliases are “a.c” and “x.y”.
Data Serialization
Avro data is always serialized with its schema. Files that store Avro data should always also include the schema for that data in the same file. Avro-based remote procedure call (RPC) systems must also guarantee that remote recipients of data have a copy of the schema used to write that data.
Because the schema used to write data is always available when the data is read, Avro data itself is not tagged with type information. The schema is required to parse data.
In general, both serialization and deserialization proceed as a depth-first, left-to-right traversal of the schema, serializing primitive types as they are encountered.
Encodings
Avro specifies two serialization encodings: binary and JSON. Most applications will use the binary encoding, as it is smaller and faster. But, for debugging and web-based applications, the JSON encoding may sometimes be appropriate.
Binary Encoding
Primitive Types
Primitive types are encoded in binary as follows:
- null is written as zero bytes.
- a boolean is written as a single byte whose value is either 0 (false) or 1 (true).
- int and long values are written using variable-length zig-zag coding. Some examples:
| value |
hex |
| 0 |
00 |
| -1 |
01 |
| 1 |
02 |
| -2 |
03 |
| 2 |
04 |
| … |
|
| -64 |
7f |
| 64 |
80 01 |
| … |
|
- a float is written as 4 bytes. The float is converted into a 32-bit integer using a method equivalent to Java’s floatToIntBits and then encoded in little-endian format.
- a double is written as 8 bytes. The double is converted into a 64-bit integer using a method equivalent to Java’s doubleToLongBits and then encoded in little-endian format.
- bytes are encoded as a long followed by that many bytes of data.
- a string is encoded as a long followed by that many bytes of UTF-8 encoded character data.
For example, the three-character string “foo” would be encoded as the long value 3 (encoded as hex 06) followed by the UTF-8 encoding of ‘f’, ‘o’, and ‘o’ (the hex bytes 66 6f 6f):
06 66 6f 6f
Complex Types
Complex types are encoded in binary as follows:
Records
A record is encoded by encoding the values of its fields in the order that they are declared. In other words, a record is encoded as just the concatenation of the encodings of its fields. Field values are encoded per their schema.
For example, the record schema
{
"type": "record",
"name": "test",
"fields" : [
{"name": "a", "type": "long"},
{"name": "b", "type": "string"}
]
}
An instance of this record whose a field has value 27 (encoded as hex 36) and whose b field has value “foo” (encoded as hex bytes 06 66 6f 6f), would be encoded simply as the concatenation of these, namely the hex byte sequence:
36 06 66 6f 6f
Enums
An enum is encoded by a int, representing the zero-based position of the symbol in the schema.
For example, consider the enum:
{"type": "enum", "name": "Foo", "symbols": ["A", "B", "C", "D"] }
This would be encoded by an int between zero and three, with zero indicating “A”, and 3 indicating “D”.
Arrays
Arrays are encoded as a series of blocks. Each block consists of a long count value, followed by that many array items. A block with count zero indicates the end of the array. Each item is encoded per the array’s item schema.
If a block’s count is negative, its absolute value is used, and the count is followed immediately by a long block size indicating the number of bytes in the block. This block size permits fast skipping through data, e.g., when projecting a record to a subset of its fields.
For example, the array schema
{"type": "array", "items": "long"}
an array containing the items 3 and 27 could be encoded as the long value 2 (encoded as hex 04) followed by long values 3 and 27 (encoded as hex 06 36) terminated by zero:
04 06 36 00
The blocked representation permits one to read and write arrays larger than can be buffered in memory, since one can start writing items without knowing the full length of the array.
Maps
Maps are encoded as a series of blocks. Each block consists of a long count value, followed by that many key/value pairs. A block with count zero indicates the end of the map. Each item is encoded per the map’s value schema.
If a block’s count is negative, its absolute value is used, and the count is followed immediately by a long block size indicating the number of bytes in the block. This block size permits fast skipping through data, e.g., when projecting a record to a subset of its fields.
The blocked representation permits one to read and write maps larger than can be buffered in memory, since one can start writing items without knowing the full length of the map.
Unions
A union is encoded by first writing a long value indicating the zero-based position within the union of the schema of its value. The value is then encoded per the indicated schema within the union.
For example, the union schema ["string","null"] would encode:
-
null as the integer 1 (the index of “null” in the union, encoded as hex 02):
02
-
the string “a” as zero (the index of “string” in the union), followed by the serialized string:
02 02 61
Fixed
Fixed instances are encoded using the number of bytes declared in the schema.
JSON Encoding
Except for unions, the JSON encoding is the same as is used to encode field default values.
The value of a union is encoded in JSON as follows:
- if its type is null, then it is encoded as a JSON null;
- otherwise it is encoded as a JSON object with one name/value pair whose name is the type’s name and whose value is the recursively encoded value. For Avro’s named types (record, fixed or enum) the user-specified name is used, for other types the type name is used.
For example, the union schema ["null","string","Foo"], where Foo is a record name, would encode:
- null as null;
- the string “a” as
{"string": "a"} and
- a Foo instance as
{"Foo": {...}}, where {...} indicates the JSON encoding of a Foo instance.
Note that a schema is still required to correctly process JSON-encoded data. For example, the JSON encoding does not distinguish between int and long, float and double, records and maps, enums and strings, etc.
Sort Order
Avro defines a standard sort order for data. This permits data written by one system to be efficiently sorted by another system. This can be an important optimization, as sort order comparisons are sometimes the most frequent per-object operation. Note also that Avro binary-encoded data can be efficiently ordered without deserializing it to objects.
Data items may only be compared if they have identical schemas. Pairwise comparisons are implemented recursively with a depth-first, left-to-right traversal of the schema. The first mismatch encountered determines the order of the items.
Two items with the same schema are compared according to the following rules.
- null data is always equal.
- boolean data is ordered with false before true.
- int, long, float and double data is ordered by ascending numeric value.
- bytes and fixed data are compared lexicographically by unsigned 8-bit values.
- string data is compared lexicographically by Unicode code point. Note that since UTF-8 is used as the binary encoding for strings, sorting of bytes and string binary data is identical.
- array data is compared lexicographically by element.
- enum data is ordered by the symbol’s position in the enum schema. For example, an enum whose symbols are
["z", "a"] would sort “z” values before “a” values.
- union data is first ordered by the branch within the union, and, within that, by the type of the branch. For example, an
["int", "string"] union would order all int values before all string values, with the ints and strings themselves ordered as defined above.
- record data is ordered lexicographically by field. If a field specifies that its order is:
- “ascending”, then the order of its values is unaltered.
- “descending”, then the order of its values is reversed.
- “ignore”, then its values are ignored when sorting.
- map data may not be compared. It is an error to attempt to compare data containing maps unless those maps are in an
"order":"ignore" record field.
Object Container Files
Avro includes a simple object container file format. A file has a schema, and all objects stored in the file must be written according to that schema, using binary encoding. Objects are stored in blocks that may be compressed. Syncronization markers are used between blocks to permit efficient splitting of files for MapReduce processing.
Files may include arbitrary user-specified metadata.
A file consists of:
- A file header, followed by
- one or more file data blocks.
A file header consists of:
- Four bytes, ASCII ‘O’, ‘b’, ‘j’, followed by 1.
- file metadata, including the schema.
- The 16-byte, randomly-generated sync marker for this file.
File metadata consists of:
A long indicating the number of metadata key/value pairs.
For each pair, a string key and bytes value.
All metadata properties that start with “avro.” are reserved. The following file metadata properties are currently used:
- avro.schema contains the schema of objects stored in the file, as JSON data (required).
- avro.codec the name of the compression codec used to compress blocks, as a string. Implementations are required to support the following codecs: “null” and “deflate”. If codec is absent, it is assumed to be “null”. The codecs are described with more detail below.
A file header is thus described by the following schema:
{"type": "record", "name": "org.apache.avro.file.Header",
"fields" : [
{"name": "magic", "type": {"type": "fixed", "name": "Magic", "size": 4}},
{"name": "meta", "type": {"type": "map", "values": "bytes"}},
{"name": "sync", "type": {"type": "fixed", "name": "Sync", "size": 16}}
]
}
A file data block consists of:
- A long indicating the count of objects in this block.
- A long indicating the size in bytes of the serialized objects in the current block, after any codec is applied
- The serialized objects. If a codec is specified, this is compressed by that codec.
- The file’s 16-byte sync marker.
Thus, each block’s binary data can be efficiently extracted or skipped without deserializing the contents. The combination of block size, object counts, and sync markers enable detection of corrupt blocks and help ensure data integrity.
Required Codecs
null
The “null” codec simply passes through data uncompressed.
deflate
The “deflate” codec writes the data block using the deflate algorithm as specified in RFC 1951, and typically implemented using the zlib library. Note that this format (unlike the “zlib format” in RFC 1950) does not have a checksum.
Optional Codecs
snappy
The “snappy” codec uses Google’s Snappy compression library. Each compressed block is followed by the 4-byte, big-endian CRC32 checksum of the uncompressed data in the block.
Protocol Declaration
Avro protocols describe RPC interfaces. Like schemas, they are defined with JSON text.
A protocol is a JSON object with the following attributes:
- protocol, a string, the name of the protocol (required);
- namespace, an optional string that qualifies the name;
- doc, an optional string describing this protocol;
- types, an optional list of definitions of named types (records, enums, fixed and errors). An error definition is just like a record definition except it uses “error” instead of “record”. Note that forward references to named types are not permitted.
- messages, an optional JSON object whose keys are message names and whose values are objects whose attributes are described below. No two messages may have the same name.
The name and namespace qualification rules defined for schema objects apply to protocols as well.
Messages
A message has attributes:
- a doc, an optional description of the message,
- a request, a list of named, typed parameter schemas (this has the same form as the fields of a record declaration);
- a response schema;
- an optional union of declared error schemas. The effective union has “string” prepended to the declared union, to permit transmission of undeclared “system” errors. For example, if the declared error union is
["AccessError"], then the effective union is ["string", "AccessError"]. When no errors are declared, the effective error union is ["string"]. Errors are serialized using the effective union; however, a protocol’s JSON declaration contains only the declared union.
- an optional one-way boolean parameter.
A request parameter list is processed equivalently to an anonymous record. Since record field lists may vary between reader and writer, request parameters may also differ between the caller and responder, and such differences are resolved in the same manner as record field differences.
The one-way parameter may only be true when the response type is "null" and no errors are listed.
Sample Protocol
For example, one may define a simple HelloWorld protocol with:
{
"namespace": "com.acme",
"protocol": "HelloWorld",
"doc": "Protocol Greetings",
"types": [
{"name": "Greeting", "type": "record", "fields": [
{"name": "message", "type": "string"}]},
{"name": "Curse", "type": "error", "fields": [
{"name": "message", "type": "string"}]}
],
"messages": {
"hello": {
"doc": "Say hello.",
"request": [{"name": "greeting", "type": "Greeting" }],
"response": "Greeting",
"errors": ["Curse"]
}
}
}
Message Transport
Messages may be transmitted via different transport mechanisms.
To the transport, a message is an opaque byte sequence.
A transport is a system that supports:
- transmission of request messages
- receipt of corresponding response messages
Servers may send a response message back to the client corresponding to a request message. The mechanism of correspondance is transport-specific. For example, in HTTP it is implicit, since HTTP directly supports requests and responses. But a transport that multiplexes many client threads over a single socket would need to tag messages with unique identifiers.
Transports may be either stateless or stateful. In a stateless transport, messaging assumes no established connection state, while stateful transports establish connections that may be used for multiple messages. This distinction is discussed further in the handshake section below.
HTTP as Transport
When HTTP is used as a transport, each Avro message exchange is an HTTP request/response pair. All messages of an Avro protocol should share a single URL at an HTTP server. Other protocols may also use that URL. Both normal and error Avro response messages should use the 200 (OK) response code. The chunked encoding may be used for requests and responses, but, regardless the Avro request and response are the entire content of an HTTP request and response. The HTTP Content-Type of requests and responses should be specified as “avro/binary”. Requests should be made using the POST method.
HTTP is used by Avro as a stateless transport.
Message Framing
Avro messages are framed as a list of buffers.
Framing is a layer between messages and the transport. It exists to optimize certain operations.
The format of framed message data is:
- a series of buffers, where each buffer consists of:
- a four-byte, big-endian buffer length, followed by
- that many bytes of buffer data.
- A message is always terminated by a zero-lenghted buffer.
Framing is transparent to request and response message formats (described below). Any message may be presented as a single or multiple buffers.
Framing can permit readers to more efficiently get different buffers from different sources and for writers to more efficiently store different buffers to different destinations. In particular, it can reduce the number of times large binary objects are copied. For example, if an RPC parameter consists of a megabyte of file data, that data can be copied directly to a socket from a file descriptor, and, on the other end, it could be written directly to a file descriptor, never entering user space.
A simple, recommended, framing policy is for writers to create a new segment whenever a single binary object is written that is larger than a normal output buffer. Small objects are then appended in buffers, while larger objects are written as their own buffers. When a reader then tries to read a large object the runtime can hand it an entire buffer directly, without having to copy it.
Handshake
The purpose of the handshake is to ensure that the client and the server have each other’s protocol definition, so that the client can correctly deserialize responses, and the server can correctly deserialize requests. Both clients and servers should maintain a cache of recently seen protocols, so that, in most cases, a handshake will be completed without extra round-trip network exchanges or the transmission of full protocol text.
RPC requests and responses may not be processed until a handshake has been completed. With a stateless transport, all requests and responses are prefixed by handshakes. With a stateful transport, handshakes are only attached to requests and responses until a successful handshake response has been returned over a connection. After this, request and response payloads are sent without handshakes for the lifetime of that connection.
The handshake process uses the following record schemas:
{
"type": "record",
"name": "HandshakeRequest", "namespace":"org.apache.avro.ipc",
"fields": [
{"name": "clientHash",
"type": {"type": "fixed", "name": "MD5", "size": 16}},
{"name": "clientProtocol", "type": ["null", "string"]},
{"name": "serverHash", "type": "MD5"},
{"name": "meta", "type": ["null", {"type": "map", "values": "bytes"}]}
]
}
{
"type": "record",
"name": "HandshakeResponse", "namespace": "org.apache.avro.ipc",
"fields": [
{"name": "match",
"type": {"type": "enum", "name": "HandshakeMatch",
"symbols": ["BOTH", "CLIENT", "NONE"]}},
{"name": "serverProtocol",
"type": ["null", "string"]},
{"name": "serverHash",
"type": ["null", {"type": "fixed", "name": "MD5", "size": 16}]},
{"name": "meta",
"type": ["null", {"type": "map", "values": "bytes"}]}
]
}
- A client first prefixes each request with a
HandshakeRequest containing just the hash of its protocol and of the server’s protocol (clientHash!=null, clientProtocol=null, serverHash!=null), where the hashes are 128-bit MD5 hashes of the JSON protocol text. If a client has never connected to a given server, it sends its hash as a guess of the server’s hash, otherwise it sends the hash that it previously obtained from this server.
The server responds with a HandshakeResponse containing one of:
-
match=BOTH, serverProtocol=null, serverHash=null if the client sent the valid hash of the server’s protocol and the server knows what protocol corresponds to the client’s hash. In this case, the request is complete and the response data immediately follows the HandshakeResponse.
-
match=CLIENT, serverProtocol!=null, serverHash!=null if the server has previously seen the client’s protocol, but the client sent an incorrect hash of the server’s protocol. The request is complete and the response data immediately follows the HandshakeResponse. The client must use the returned protocol to process the response and should also cache that protocol and its hash for future interactions with this server.
-
match=NONE if the server has not previously seen the client’s protocol. The serverHash and serverProtocol may also be non-null if the server’s protocol hash was incorrect.
In this case the client must then re-submit its request with its protocol text (clientHash!=null, clientProtocol!=null, serverHash!=null) and the server should respond with a successful match (match=BOTH, serverProtocol=null, serverHash=null) as above.
The meta field is reserved for future handshake enhancements.
A call consists of a request message paired with its resulting response or error message. Requests and responses contain extensible metadata, and both kinds of messages are framed as described above.
The format of a call request is:
- request metadata, a map with values of type bytes
- the message name, an Avro string, followed by
- the message parameters. Parameters are serialized according to the message’s request declaration.
When a message is declared one-way and a stateful connection has been established by a successful handshake response, no response data is sent. Otherwise the format of the call response is:
- response metadata, a map with values of type bytes
- a one-byte error flag boolean, followed by either:
- if the error flag is false, the message response, serialized per the message’s response schema.
- if the error flag is true, the error, serialized per the message’s effective error union schema.
Schema Resolution
A reader of Avro data, whether from an RPC or a file, can always parse that data because its schema is provided. But that schema may not be exactly the schema that was expected. For example, if the data was written with a different version of the software than it is read, then records may have had fields added or removed. This section specifies how such schema differences should be resolved.
We refer to the schema used to write the data as the writer’s schema, and the schema that the application expects the reader’s schema. Differences between these should be resolved as follows:
- It is an error if the two schemas do not match.
To match, one of the following must hold:
-
both schemas are arrays whose item types match
-
both schemas are maps whose value types match
-
both schemas are enums whose names match
-
both schemas are fixed whose sizes and names match
-
both schemas are records with the same name
-
either schema is a union
-
both schemas have same primitive type
-
the writer’s schema may be promoted to the reader’s as follows:
- int is promotable to long, float, or double
- long is promotable to float or double
- float is promotable to double
- string is promotable to bytes
- bytes is promotable to string
-
if both are records:
- the ordering of fields may be different: fields are matched by name.
- schemas for fields with the same name in both records are resolved recursively.
- if the writer’s record contains a field with a name not present in the reader’s record, the writer’s value for that field is ignored.
- if the reader’s record schema has a field that contains a default value, and writer’s schema does not have a field with the same name, then the reader should use the default value from its field.
- if the reader’s record schema has a field with no default value, and writer’s schema does not have a field with the same name, an error is signalled.
-
if both are enums:
if the writer’s symbol is not present in the reader’s enum, then an error is signalled.
This resolution algorithm is applied recursively to the reader’s and writer’s array item schemas.
This resolution algorithm is applied recursively to the reader’s and writer’s value schemas.
The first schema in the reader’s union that matches the selected writer’s union schema is recursively resolved against it. if none match, an error is signalled.
- if reader’s is a union, but writer’s is not
The first schema in the reader’s union that matches the writer’s schema is recursively resolved against it. If none match, an error is signalled.
- if writer’s is a union, but reader’s is not
If the reader’s schema matches the selected writer’s schema, it is recursively resolved against it. If they do not match, an error is signalled.
A schema’s doc fields are ignored for the purposes of schema resolution. Hence, the doc portion of a schema may be dropped at serialization.
One of the defining characteristics of Avro is that a reader is assumed to have the “same” schema used by the writer of the data the reader is reading. This assumption leads to a data format that’s compact and also amenable to many forms of schema evolution. However, the specification so far has not defined what it means for the reader to have the “same” schema as the writer. Does the schema need to be textually identical? Well, clearly adding or removing some whitespace to a JSON expression does not change its meaning. At the same time, reordering the fields of records clearly does change the meaning. So what does it mean for a reader to have “the same” schema as a writer?
Parsing Canonical Form is a transformation of a writer’s schema that let’s us define what it means for two schemas to be “the same” for the purpose of reading data written against the schema. It is called Parsing Canonical Form because the transformations strip away parts of the schema, like “doc” attributes, that are irrelevant to readers trying to parse incoming data. It is called Canonical Form because the transformations normalize the JSON text (such as the order of attributes) in a way that eliminates unimportant differences between schemas. If the Parsing Canonical Forms of two different schemas are textually equal, then those schemas are “the same” as far as any reader is concerned, i.e., there is no serialized data that would allow a reader to distinguish data generated by a writer using one of the original schemas from data generated by a writing using the other original schema. (We sketch a proof of this property in a companion document.)
The next subsection specifies the transformations that define Parsing Canonical Form. But with a well-defined canonical form, it can be convenient to go one step further, transforming these canonical forms into simple integers (“fingerprints”) that can be used to uniquely identify schemas. The subsection after next recommends some standard practices for generating such fingerprints.
Assuming an input schema (in JSON form) that’s already UTF-8 text for a valid Avro schema (including all quotes as required by JSON), the following transformations will produce its Parsing Canonical Form:
- [PRIMITIVES] Convert primitive schemas to their simple form (e.g., int instead of
{"type":"int"}).
- [FULLNAMES] Replace short names with fullnames, using applicable namespaces to do so. Then eliminate namespace attributes, which are now redundant.
- [STRIP] Keep only attributes that are relevant to parsing data, which are: type, name, fields, symbols, items, values, size. Strip all others (e.g., doc and aliases).
- [ORDER] Order the appearance of fields of JSON objects as follows: name, type, fields, symbols, items, values, size. For example, if an object has type, name, and size fields, then the name field should appear first, followed by the type and then the size fields.
- [STRINGS] For all JSON string literals in the schema text, replace any escaped characters (e.g., \uXXXX escapes) with their UTF-8 equivalents.
- [INTEGERS] Eliminate quotes around and any leading zeros in front of JSON integer literals (which appear in the size attributes of fixed schemas).
- [WHITESPACE] Eliminate all whitespace in JSON outside of string literals.
Schema Fingerprints
“[A] fingerprinting algorithm is a procedure that maps an arbitrarily large data item (such as a computer file) to a much shorter bit string, its fingerprint, that uniquely identifies the original data for all practical purposes” (quoted from Wikipedia). In the Avro context, fingerprints of Parsing Canonical Form can be useful in a number of applications; for example, to cache encoder and decoder objects, to tag data items with a short substitute for the writer’s full schema, and to quickly negotiate common-case schemas between readers and writers.
In designing fingerprinting algorithms, there is a fundamental trade-off between the length of the fingerprint and the probability of collisions. To help application designers find appropriate points within this trade-off space, while encouraging interoperability and ease of implementation, we recommend using one of the following three algorithms when fingerprinting Avro schemas:
- When applications can tolerate longer fingerprints, we recommend using the SHA-256 digest algorithm to generate 256-bit fingerprints of Parsing Canonical Forms. Most languages today have SHA-256 implementations in their libraries.
- At the opposite extreme, the smallest fingerprint we recommend is a 64-bit Rabin fingerprint. Below, we provide pseudo-code for this algorithm that can be easily translated into any programming language. 64-bit fingerprints should guarantee uniqueness for schema caches of up to a million entries (for such a cache, the chance of a collision is 3E-8). We don’t recommend shorter fingerprints, as the chances of collisions is too great (for example, with 32-bit fingerprints, a cache with as few as 100,000 schemas has a 50% chance of having a collision).
- Between these two extremes, we recommend using the MD5 message digest to generate 128-bit fingerprints. These make sense only where very large numbers of schemas are being manipulated (tens of millions); otherwise, 64-bit fingerprints should be sufficient. As with SHA-256, MD5 implementations are found in most libraries today.
These fingerprints are not meant to provide any security guarantees, even the longer SHA-256-based ones. Most Avro applications should be surrounded by security measures that prevent attackers from writing random data and otherwise interfering with the consumers of schemas. We recommend that these surrounding mechanisms be used to prevent collision and pre-image attacks (i.e., “forgery”) on schema fingerprints, rather than relying on the security properties of the fingerprints themselves.
Rabin fingerprints are cyclic redundancy checks computed using irreducible polynomials. In the style of the Appendix of RFC 1952 (pg 10), which defines the CRC-32 algorithm, here’s our definition of the 64-bit AVRO fingerprinting algorithm:
long fingerprint64(byte[] buf) {
if (FP_TABLE == null) initFPTable();
long fp = EMPTY;
for (int i = 0; i < buf.length; i++)
fp = (fp >>> 8) ^ FP_TABLE[(int)(fp ^ buf[i]) & 0xff];
return fp;
}
static long EMPTY = 0xc15d213aa4d7a795L;
static long[] FP_TABLE = null;
void initFPTable() {
FP_TABLE = new long[256];
for (int i = 0; i < 256; i++) {
long fp = i;
for (int j = 0; j < 8; j++)
fp = (fp >>> 1) ^ (EMPTY & -(fp & 1L));
FP_TABLE[i] = fp;
}
}
Readers interested in the mathematics behind this algorithm may want to read this book chapter. (Unlike RFC-1952 and the book chapter, we prepend a single one bit to messages. We do this because CRCs ignore leading zero bits, which can be problematic. Our code prepends a one-bit by initializing fingerprints using EMPTY, rather than initializing using zero as in RFC-1952 and the book chapter.)
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
19.2 - IDL Language
Introduction
This document defines Avro IDL, a higher-level language for authoring Avro schemata. Before reading this document, you should have familiarity with the concepts of schemata and protocols, as well as the various primitive and complex types available in Avro.
Overview
Purpose
The aim of the Avro IDL language is to enable developers to author schemata in a way that feels more similar to common programming languages like Java, C++, or Python. Additionally, the Avro IDL language may feel more familiar for those users who have previously used the interface description languages (IDLs) in other frameworks like Thrift, Protocol Buffers, or CORBA.
Usage
Each Avro IDL file defines a single Avro Protocol, and thus generates as its output a JSON-format Avro Protocol file with extension .avpr.
To convert a .avdl file into a .avpr file, it may be processed by the idl tool. For example:
$ java -jar avroj-tools.jar idl src/test/idl/input/namespaces.avdl /tmp/namespaces.avpr
$ head /tmp/namespaces.avpr
{
"protocol" : "TestNamespace",
"namespace" : "avro.test.protocol",
The idl tool can also process input to and from stdin and stdout. See idl --help for full usage information.
A Maven plugin is also provided to compile .avdl files. To use it, add something like the following to your pom.xml:
<build>
<plugins>
<plugin>
<groupId>org.apache.avro</groupId>
<artifactId>avro-maven-plugin</artifactId>
<executions>
<execution>
<goals>
<goal>idl-protocol</goal>
</goals>
</execution>
</executions>
</plugin>
</plugins>
</build>
Defining a Protocol in Avro IDL
An Avro IDL file consists of exactly one protocol definition. The minimal protocol is defined by the following code:
This is equivalent to (and generates) the following JSON protocol definition:
{
"protocol" : "MyProtocol",
"types" : [ ],
"messages" : {
}
}
The namespace of the protocol may be changed using the @namespace annotation:
@namespace("mynamespace")
protocol MyProtocol {
}
This notation is used throughout Avro IDL as a way of specifying properties for the annotated element, as will be described later in this document.
Protocols in Avro IDL can contain the following items:
- Imports of external protocol and schema files.
- Definitions of named schemata, including records, errors, enums, and fixeds.
- Definitions of RPC messages
Imports
Files may be imported in one of three formats:
-
An IDL file may be imported with a statement like:
import idl "foo.avdl";
-
A JSON protocol file may be imported with a statement like:
import protocol "foo.avpr";
-
A JSON schema file may be imported with a statement like:
import schema "foo.avsc";
Messages and types in the imported file are added to this file’s protocol.
Imported file names are resolved relative to the current IDL file.
Defining an Enumeration
Enums are defined in Avro IDL using a syntax similar to C or Java:
enum Suit {
SPADES, DIAMONDS, CLUBS, HEARTS
}
Note that, unlike the JSON format, anonymous enums cannot be defined.
Defining a Fixed Length Field
Fixed fields are defined using the following syntax:
fixed MD5(16);
This example defines a fixed-length type called MD5 which contains 16 bytes.
Defining Records and Errors
Records are defined in Avro IDL using a syntax similar to a struct definition in C:
record Employee {
string name;
boolean active = true;
long salary;
}
The above example defines a record with the name “Employee” with three fields.
To define an error, simply use the keyword error instead of record. For example:
error Kaboom {
string explanation;
int result_code = -1;
}
Each field in a record or error consists of a type and a name, optional property annotations and an optional default value.
A type reference in Avro IDL must be one of:
- A primitive type
- A logical type
- A named schema defined prior to this usage in the same Protocol
- A complex type (array, map, or union)
Primitive Types
The primitive types supported by Avro IDL are the same as those supported by Avro’s JSON format. This list includes int, long, string, boolean, float, double, null, and bytes.
References to Named Schemata
If a named schema has already been defined in the same Avro IDL file, it may be referenced by name as if it were a primitive type:
record Card {
Suit suit; // refers to the enum Card defined above
int number;
}
Default Values
Default values for fields may be optionally specified by using an equals sign after the field name followed by a JSON expression indicating the default value. This JSON is interpreted as described in the spec.
Complex Types
Arrays
Array types are written in a manner that will seem familiar to C++ or Java programmers. An array of any type t is denoted array<t>. For example, an array of strings is denoted array<string>, and a multidimensional array of Foo records would be array<array<Foo>>.
Maps
Map types are written similarly to array types. An array that contains values of type t is written map<t>. As in the JSON schema format, all maps contain string-type keys.
Unions
Union types are denoted as union { typeA, typeB, typeC, … }. For example, this record contains a string field that is optional (unioned with null):
record RecordWithUnion {
union { null, string } optionalString;
}
Note that the same restrictions apply to Avro IDL unions as apply to unions defined in the JSON format; namely, a record may not contain multiple elements of the same type.
Defining RPC Messages
The syntax to define an RPC message within a Avro IDL protocol is similar to the syntax for a method declaration within a C header file or a Java interface. To define an RPC message add which takes two arguments named foo and bar, returning an int, simply include the following definition within the protocol:
int add(int foo, int bar = 0);
Message arguments, like record fields, may specify default values.
To define a message with no response, you may use the alias void, equivalent to the Avro null type:
void logMessage(string message);
If you have previously defined an error type within the same protocol, you may declare that a message can throw this error using the syntax:
void goKaboom() throws Kaboom;
To define a one-way message, use the keyword oneway after the parameter list, for example:
void fireAndForget(string message) oneway;
Other Language Features
All Java-style comments are supported within a Avro IDL file. Any text following // on a line is ignored, as is any text between /* and */, possibly spanning multiple lines.
Comments that begin with /** are used as the documentation string for the type or field definition that follows the comment.
Escaping Identifiers
Occasionally, one will need to use a reserved language keyword as an identifier. In order to do so, backticks (`) may be used to escape the identifier. For example, to define a message with the literal name error, you may write:
This syntax is allowed anywhere an identifier is expected.
Annotations for Ordering and Namespaces
Java-style annotations may be used to add additional properties to types and fields throughout Avro IDL.
For example, to specify the sort order of a field within a record, one may use the @order annotation before the field name as follows:
record MyRecord {
string @order("ascending") myAscendingSortField;
string @order("descending") myDescendingField;
string @order("ignore") myIgnoredField;
}
A field’s type may also be preceded by annotations, e.g.:
record MyRecord {
@java-class("java.util.ArrayList") array<string> myStrings;
}
Similarly, a @namespace annotation may be used to modify the namespace when defining a named schema. For example:
@namespace("org.apache.avro.firstNamespace")
protocol MyProto {
@namespace("org.apache.avro.someOtherNamespace")
record Foo {}
record Bar {}
}
will define a protocol in the firstNamespace namespace. The record Foo will be defined in someOtherNamespace and Bar will be defined in firstNamespace as it inherits its default from its container.
Type and field aliases are specified with the @aliases annotation as follows:
@aliases(["org.old.OldRecord", "org.ancient.AncientRecord"])
record MyRecord {
string @aliases(["oldField", "ancientField"]) myNewField;
}
Some annotations like those listed above are handled specially. All other annotations are added as properties to the protocol, message, schema or field.
Complete Example
The following is a complete example of a Avro IDL file that shows most of the above features:
/**
* An example protocol in Avro IDL
*/
@namespace("org.apache.avro.test")
protocol Simple {
@aliases(["org.foo.KindOf"])
enum Kind {
FOO,
BAR, // the bar enum value
BAZ
}
fixed MD5(16);
record TestRecord {
@order("ignore")
string name;
@order("descending")
Kind kind;
MD5 hash;
union { MD5, null} @aliases(["hash"]) nullableHash;
array<long> arrayOfLongs;
}
error TestError {
string message;
}
string hello(string greeting);
TestRecord echo(TestRecord `record`);
int add(int arg1, int arg2);
bytes echoBytes(bytes data);
void `error`() throws TestError;
void ping() oneway;
}
Additional examples may be found in the Avro source tree under the src/test/idl/input directory.
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
19.3 - SASL profile
Introduction
SASL (RFC 2222) provides a framework for authentication and security of network protocols. Each protocol that uses SASL is meant to define a SASL profile. This document provides a SASL profile for connection-based Avro RPC.
Overview
SASL negotiation proceeds as a series of message interactions over a connection between a client and server using a selected SASL mechanism. The client starts this negotiation by sending its chosen mechanism name with an initial (possibly empty) message. Negotiation proceeds with the exchange of messages until either side indicates success or failure. The content of the messages is mechanism-specific. If the negotiation succeeds, then the session can proceed over the connection, otherwise it must be abandoned.
Some mechanisms continue to process session data after negotiation (e.g., encrypting it), while some specify that further session data is transmitted unmodified.
Negotiation
Commands
Avro SASL negotiation uses four one-byte commands.
- 0: START Used in a client’s initial message.
- 1: CONTINUE Used while negotiation is ongoing.
- 2: FAIL Terminates negotiation unsuccessfully.
- 3: COMPLETE Terminates negotiation successfully.
The format of a START message is:
| 0 | 4-byte mechanism name length | mechanism name | 4-byte payload length | payload data |
The format of a CONTINUE message is:
| 1 | 4-byte payload length | payload data |
The format of a FAIL message is:
| 2 | 4-byte message length | UTF-8 message |
The format of a COMPLETE message is:
| 3 | 4-byte payload length | payload data |
Process
Negotiation is initiated by a client sending a START command containing the client’s chosen mechanism name and any mechanism-specific payload data.
The server and client then interchange some number (possibly zero) of CONTINUE messages. Each message contains payload data that is processed by the security mechanism to generate the next message.
Once either the client or server send a FAIL message then negotiation has failed. UTF-8-encoded text is included in the failure message. Once either a FAIL message has been sent or received, or any other error occurs in the negotiation, further communication on this connection must cease.
Once either the client or server send a COMPLETE message then negotiation has completed successfully. Session data may now be transmitted over the connection until it is closed by either side.
Session Data
If no SASL QOP (quality of protection) is negotiated, then all subsequent writes to/reads over this connection are written/read unmodified. In particular, messages use Avro framing, and are of the form:
| 4-byte frame length | frame data | ... | 4 zero bytes |
If a SASL QOP is negotiated, then it must be used by the connection for all subsequent messages. This is done by wrapping each non-empty frame written using the security mechanism and unwrapping each non-empty frame read. The length written in each non-empty frame is the length of the wrapped data. Complete frames must be passed to the security mechanism for unwrapping. Unwrapped data is then passed to the application as the content of the frame.
If at any point processing fails due to wrapping, unwrapping or framing errors, then all further communication on this connection must cease.
Anonymous Mechanism
The SASL anonymous mechanism (RFC 2245) is quite simple to implement. In particular, an initial anonymous request may be prefixed by the following static sequence:
| 0 | 0009 | ANONYMOUS | 0000 |
If a server uses the anonymous mechanism, it should check that the mechanism name in the start message prefixing the first request received is ‘ANONYMOUS’, then simply prefix its initial response with a COMPLETE message of:
| 3 | 0000 |
If an anonymous server recieves some other mechanism name, then it may respond with a FAIL message as simple as:
| 2 | 0000 |
Note that the anonymous mechanism need add no additional round-trip messages between client and server. The START message can be piggybacked on the initial request and the COMPLETE or FAIL message can be piggybacked on the initial response.
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
19.4 - Wiki
The Wiki page can be found here.
19.5 - FAQ
What is Avro?
Avro is a data serialization system.
Avro provides:
- Rich data structures.
- A compact, fast, binary data format.
- A container file, to store persistent data.
- Remote procedure call (RPC).
- Simple integration with dynamic languages. Code generation is not required to read or write data files nor to use or implement RPC protocols. Code generation as an optional optimization, only worth implementing for statically typed languages.
How can I get started quickly with Avro?
Check out the Quick Start Guide.
How do I statically compile a schema or protocol into generated code?
In Java
- Add the avro jar, the jackson-mapper-asl.jar and jackson-core-asl.jar to your CLASSPATH.
- Run java org.apache.avro.specific.SpecificCompiler .
This appears to be out of date, the SpecificCompiler requires two arguments, presumably an input and and output file, but it isn’t clear that this does.
Or use the Schema or Protocol Ant tasks. Avro’s build.xml provides examples of how these are used.
Lastly, you can also use the “avro-tools” jar which ships with an Avro release. Just use the “compile (schema|protocol)” command.
How are Strings represented in Java?
They use org.apache.avro.util.Utf8, not java.lang.String.
More generally, how do Avro types map to Java types?
The mappings are documented in the package javadoc for generic, specific and reflect API.
From Doug Cutting:
HDFS splits files into blocks, and mapreduce runs a map task for each block. When the task starts, it needs to be able to seek into the file to the start of the block process through the block’s end. If the file were, e.g., a gzip file, this would not be possible, since gzip files must be decompressed from the start. One cannot seek into the middle of a gzip file and start decompressing. So Hadoop’s SequenceFile places a marker periodically (~64k) in the file at record and compression boundaries, where processing can be sensibly started. Then, when a map task starts processing an HDFS block, it finds the first marker after the block’s start and continues through the first marker in the next block of the file. This requires a bit of non-local access (~0.1%). Avro’s data file uses the same method as SequenceFile.
Why isn’t every value in Avro nullable?
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 of every value stored and
corresponding computational costs to create this bit on write and
interpret it on read. These costs are wasted when values may not in
fact be null, as is the case in many datasets. In Avro such costs are
only paid when values may actually be null.
Also, allowing values to be null is a well-known source of errors. In
Avro, a value declared as non-null will always be non-null and programs
need not test for null values when processing it nor will they ever fail
for lack of such tests.
Tony Hoare calls his invention of null references his “Billion Dollar
Mistake”.
http://qconlondon.com/london-2009/presentation/Null+References:+The+Billion+Dollar+Mistake
Also note that in some programming languages not all values are permitted to be null. For example, in Java, values of type boolean, byte, short, char, int, float, long, and double may not be null.
How can I serialize directly to/from a byte array?
As pointed out in the specification, Avro data should always be stored with its schema. The Avro provided classes DataFileWriter, DataFileReader, and DataFileStream all ensure this by serializing the Schema in a container header. In some special cases, such as when implementing a new storage system or writing unit tests, you may need to write and read directly with the bare Avro serialized values. The following examples use code generated for Java from an Avro Schema, specifically the User example from the Quickstart guide.
Serializing to a byte array
This example takes a User object and returns a newly allocated byte array with the Avro serialization of that user.
ByteArrayOutputStream out = new ByteArrayOutputStream();
BinaryEncoder encoder = EncoderFactory.get().binaryEncoder(out, null);
DatumWriter<User> writer = new SpecificDatumWriter<User>(User.getClassSchema());
writer.write(user, encoder);
encoder.flush();
out.close();
byte[] serializedBytes = out.toByteArray();
Deserializing from a byte array
This example takes a byte array containing the Avro serialization of a user and returns a User object.SpecificDatumReader reader = new SpecificDatumReader(User.getClassSchema());
Decoder decoder = DecoderFactory.get().binaryDecoder(bytes, null);
User user = reader.read(null, decoder);
Note: If you are serializing or deserializing in a loop or as a method, you should be reusing objects, readers and/or writers. Check the JavaDoc to see how to reuse the objects.
20 - Apache Avro™ 1.6.3 Documentation
Introduction
Apache Avro™ is a data serialization system.
Avro provides:
- Rich data structures.
- A compact, fast, binary data format.
- A container file, to store persistent data.
- Remote procedure call (RPC).
- Simple integration with dynamic languages. Code generation is not required to read or write data files nor to use or implement RPC protocols. Code generation as an optional optimization, only worth implementing for statically typed languages.
Schemas
Avro relies on schemas. When Avro data is read, the schema used when writing it is always present. This permits each datum to be written with no per-value overheads, making serialization both fast and small. This also facilitates use with dynamic, scripting languages, since data, together with its schema, is fully self-describing.
When Avro data is stored in a file, its schema is stored with it, so that files may be processed later by any program. If the program reading the data expects a different schema this can be easily resolved, since both schemas are present.
When Avro is used in RPC, the client and server exchange schemas in the connection handshake. (This can be optimized so that, for most calls, no schemas are actually transmitted.) Since both client and server both have the other’s full schema, correspondence between same named fields, missing fields, extra fields, etc. can all be easily resolved.
Avro schemas are defined with JSON . This facilitates implementation in languages that already have JSON libraries.
Comparison with other systems
Avro provides functionality similar to systems such as Thrift, Protocol Buffers, etc. Avro differs from these systems in the following fundamental aspects.
- Dynamic typing: Avro does not require that code be generated. Data is always accompanied by a schema that permits full processing of that data without code generation, static datatypes, etc. This facilitates construction of generic data-processing systems and languages.
- Untagged data: Since the schema is present when data is read, considerably less type information need be encoded with data, resulting in smaller serialization size.
- No manually-assigned field IDs: When a schema changes, both the old and new schema are always present when processing data, so differences may be resolved symbolically, using field names.
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
20.1 - Specification
Introduction
This document defines Apache Avro. It is intended to be the authoritative specification. Implementations of Avro must adhere to this document.
Schema Declaration
A Schema is represented in JSON by one of:
- A JSON string, naming a defined type.
- A JSON object, of the form:
{"type": "typeName" ...attributes...}
where typeName is either a primitive or derived type name, as defined below. Attributes not defined in this document are permitted as metadata, but must not affect the format of serialized data.
- A JSON array, representing a union of embedded types.
Primitive Types
The set of primitive type names is:
- null: no value
- boolean: a binary value
- int: 32-bit signed integer
- long: 64-bit signed integer
- float: single precision (32-bit) IEEE 754 floating-point number
- double: double precision (64-bit) IEEE 754 floating-point number
- bytes: sequence of 8-bit unsigned bytes
- string: unicode character sequence
Primitive types have no specified attributes.
Primitive type names are also defined type names. Thus, for example, the schema “string” is equivalent to:
Complex Types
Avro supports six kinds of complex types: records, enums, arrays, maps, unions and fixed.
Records
Records use the type name “record” and support three attributes:
- name: a JSON string providing the name of the record (required).
- namespace, a JSON string that qualifies the name;
- doc: a JSON string providing documentation to the user of this schema (optional).
- aliases: a JSON array of strings, providing alternate names for this record (optional).
- fields: a JSON array, listing fields (required). Each field is a JSON object with the following attributes:
- name: a JSON string providing the name of the field (required), and
- doc: a JSON string describing this field for users (optional).
- type: A JSON object defining a schema, or a JSON string naming a record definition (required).
- default: A default value for this field, used when reading instances that lack this field (optional). 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. Default values for bytes and fixed fields are JSON strings, where Unicode code points 0-255 are mapped to unsigned 8-bit byte values 0-255.
field default values
| avro type |
json type |
example |
| null |
null |
null |
| boolean |
boolean |
true |
| int,long |
integer |
1 |
| float,double |
number |
1.1 |
| bytes |
string |
"\u00FF" |
| string |
string |
"foo" |
| record |
object |
{"a": 1} |
| enum |
string |
"FOO" |
| array |
array |
[1] |
| map |
object |
{"a": 1} |
| fixed |
string |
"\u00ff" |
- order: specifies how this field impacts sort ordering of this record (optional). Valid values are “ascending” (the default), “descending”, or “ignore”. For more details on how this is used, see the sort order section below.
- aliases: a JSON array of strings, providing alternate names for this field (optional).
For example, a linked-list of 64-bit values may be defined with:
{
"type": "record",
"name": "LongList",
"aliases": ["LinkedLongs"], // old name for this
"fields" : [
{"name": "value", "type": "long"}, // each element has a long
{"name": "next", "type": ["LongList", "null"]} // optional next element
]
}
Enums
Enums use the type name “enum” and support the following attributes:
- name: a JSON string providing the name of the enum (required).
- namespace, a JSON string that qualifies the name ;
- aliases: a JSON array of strings, providing alternate names for this enum (optional).
- doc: a JSON string providing documentation to the user of this schema (optional).
- symbols: a JSON array, listing symbols, as JSON strings (required). All symbols in an enum must be unique; duplicates are prohibited.
For example, playing card suits might be defined with:
{
"type": "enum",
"name": "Suit",
"symbols" : ["SPADES", "HEARTS", "DIAMONDS", "CLUBS"]
}
Arrays
Arrays use the type name “array” and support a single attribute:
- items: the schema of the array’s items.
For example, an array of strings is declared with:
{
"type": "array",
"items" : "string"
}
Maps
Maps use the type name “map” and support one attribute:
- values: the schema of the map’s values.
Map keys are assumed to be strings.
For example, a map from string to long is declared with:
{
"type": "map",
"values" : "long"
}
Unions
Unions, as mentioned above, are represented using JSON arrays. For example, ["string", "null"] declares a schema which may be either a string or null.
Unions may not contain more than one schema with the same type, except for the named types record, fixed and enum. For example, unions containing two array types or two map types are not permitted, but two types with different names are permitted. (Names permit efficient resolution when reading and writing unions.)
Unions may not immediately contain other unions.
Fixed
Fixed uses the type name “fixed” and supports two attributes:
- name: a string naming this fixed (required).
- namespace, a string that qualifies the name;
- aliases: a JSON array of strings, providing alternate names for this enum (optional).
- size: an integer, specifying the number of bytes per value (required).
For example, 16-byte quantity may be declared with:
{"type": "fixed", "size": 16, "name": "md5"}
Names
Record, enums and fixed are named types. Each has a fullname that is composed of two parts; a name and a namespace. Equality of names is defined on the fullname.
The name portion of a fullname, record field names, and enum symbols must:
- start with [A-Za-z_]
- subsequently contain only [A-Za-z0-9_]
A namespace is a dot-separated sequence of such names. Equality of names (including field names and enum symbols) as well as fullnames is case-sensitive.
In record, enum and fixed definitions, the fullname is determined in one of the following ways:
A name and namespace are both specified. For example, one might use “name”: “X”, “namespace”: “org.foo” to indicate the fullname org.foo.X.
-
A fullname is specified. If the name specified contains a dot, then it is assumed to be a fullname, and any namespace also specified is ignored. For example, use “name”: “org.foo.X” to indicate the fullname org.foo.X.
-
A name only is specified, i.e., a name that contains no dots. In this case the namespace is taken from the most tightly enclosing schema or protocol. For example, if “name”: “X” is specified, and this occurs within a field of the record definition of org.foo.Y, then the fullname is org.foo.X.
References to previously defined names are as in the latter two cases above: if they contain a dot they are a fullname, if they do not contain a dot, the namespace is the namespace of the enclosing definition.
Primitive type names have no namespace and their names may not be defined in any namespace.
A schema or protocol may not contain multiple definitions of a fullname. Further, a name must be defined before it is used (“before” in the depth-first, left-to-right traversal of the JSON parse tree, where the types attribute of a protocol is always deemed to come “before” the messages attribute.)
Aliases
Named types and fields may have aliases. An implementation may optionally use aliases to map a writer’s schema to the reader’s. This facilitates both schema evolution as well as processing disparate datasets.
Aliases function by re-writing the writer’s schema using aliases from the reader’s schema. For example, if the writer’s schema was named “Foo” and the reader’s schema is named “Bar” and has an alias of “Foo”, then the implementation would act as though “Foo” were named “Bar” when reading. Similarly, if data was written as a record with a field named “x” and is read as a record with a field named “y” with alias “x”, then the implementation would act as though “x” were named “y” when reading.
A type alias may be specified either as a fully namespace-qualified, or relative to the namespace of the name it is an alias for. For example, if a type named “a.b” has aliases of “c” and “x.y”, then the fully qualified names of its aliases are “a.c” and “x.y”.
Data Serialization
Avro data is always serialized with its schema. Files that store Avro data should always also include the schema for that data in the same file. Avro-based remote procedure call (RPC) systems must also guarantee that remote recipients of data have a copy of the schema used to write that data.
Because the schema used to write data is always available when the data is read, Avro data itself is not tagged with type information. The schema is required to parse data.
In general, both serialization and deserialization proceed as a depth-first, left-to-right traversal of the schema, serializing primitive types as they are encountered.
Encodings
Avro specifies two serialization encodings: binary and JSON. Most applications will use the binary encoding, as it is smaller and faster. But, for debugging and web-based applications, the JSON encoding may sometimes be appropriate.
Binary Encoding
Primitive Types
Primitive types are encoded in binary as follows:
- null is written as zero bytes.
- a boolean is written as a single byte whose value is either 0 (false) or 1 (true).
- int and long values are written using variable-length zig-zag coding. Some examples:
| value |
hex |
| 0 |
00 |
| -1 |
01 |
| 1 |
02 |
| -2 |
03 |
| 2 |
04 |
| … |
|
| -64 |
7f |
| 64 |
80 01 |
| … |
|
- a float is written as 4 bytes. The float is converted into a 32-bit integer using a method equivalent to Java’s floatToIntBits and then encoded in little-endian format.
- a double is written as 8 bytes. The double is converted into a 64-bit integer using a method equivalent to Java’s doubleToLongBits and then encoded in little-endian format.
- bytes are encoded as a long followed by that many bytes of data.
- a string is encoded as a long followed by that many bytes of UTF-8 encoded character data.
For example, the three-character string “foo” would be encoded as the long value 3 (encoded as hex 06) followed by the UTF-8 encoding of ‘f’, ‘o’, and ‘o’ (the hex bytes 66 6f 6f):
06 66 6f 6f
Complex Types
Complex types are encoded in binary as follows:
Records
A record is encoded by encoding the values of its fields in the order that they are declared. In other words, a record is encoded as just the concatenation of the encodings of its fields. Field values are encoded per their schema.
For example, the record schema
{
"type": "record",
"name": "test",
"fields" : [
{"name": "a", "type": "long"},
{"name": "b", "type": "string"}
]
}
An instance of this record whose a field has value 27 (encoded as hex 36) and whose b field has value “foo” (encoded as hex bytes 06 66 6f 6f), would be encoded simply as the concatenation of these, namely the hex byte sequence:
36 06 66 6f 6f
Enums
An enum is encoded by a int, representing the zero-based position of the symbol in the schema.
For example, consider the enum:
{"type": "enum", "name": "Foo", "symbols": ["A", "B", "C", "D"] }
This would be encoded by an int between zero and three, with zero indicating “A”, and 3 indicating “D”.
Arrays
Arrays are encoded as a series of blocks. Each block consists of a long count value, followed by that many array items. A block with count zero indicates the end of the array. Each item is encoded per the array’s item schema.
If a block’s count is negative, its absolute value is used, and the count is followed immediately by a long block size indicating the number of bytes in the block. This block size permits fast skipping through data, e.g., when projecting a record to a subset of its fields.
For example, the array schema
{"type": "array", "items": "long"}
an array containing the items 3 and 27 could be encoded as the long value 2 (encoded as hex 04) followed by long values 3 and 27 (encoded as hex 06 36) terminated by zero:
04 06 36 00
The blocked representation permits one to read and write arrays larger than can be buffered in memory, since one can start writing items without knowing the full length of the array.
Maps
Maps are encoded as a series of blocks. Each block consists of a long count value, followed by that many key/value pairs. A block with count zero indicates the end of the map. Each item is encoded per the map’s value schema.
If a block’s count is negative, its absolute value is used, and the count is followed immediately by a long block size indicating the number of bytes in the block. This block size permits fast skipping through data, e.g., when projecting a record to a subset of its fields.
The blocked representation permits one to read and write maps larger than can be buffered in memory, since one can start writing items without knowing the full length of the map.
Unions
A union is encoded by first writing a long value indicating the zero-based position within the union of the schema of its value. The value is then encoded per the indicated schema within the union.
For example, the union schema ["string","null"] would encode:
-
null as the integer 1 (the index of “null” in the union, encoded as hex 02):
02
-
the string “a” as zero (the index of “string” in the union), followed by the serialized string:
02 02 61
Fixed
Fixed instances are encoded using the number of bytes declared in the schema.
JSON Encoding
Except for unions, the JSON encoding is the same as is used to encode field default values.
The value of a union is encoded in JSON as follows:
- if its type is null, then it is encoded as a JSON null;
- otherwise it is encoded as a JSON object with one name/value pair whose name is the type’s name and whose value is the recursively encoded value. For Avro’s named types (record, fixed or enum) the user-specified name is used, for other types the type name is used.
For example, the union schema ["null","string","Foo"], where Foo is a record name, would encode:
- null as null;
- the string “a” as
{"string": "a"} and
- a Foo instance as
{"Foo": {...}}, where {...} indicates the JSON encoding of a Foo instance.
Note that a schema is still required to correctly process JSON-encoded data. For example, the JSON encoding does not distinguish between int and long, float and double, records and maps, enums and strings, etc.
Sort Order
Avro defines a standard sort order for data. This permits data written by one system to be efficiently sorted by another system. This can be an important optimization, as sort order comparisons are sometimes the most frequent per-object operation. Note also that Avro binary-encoded data can be efficiently ordered without deserializing it to objects.
Data items may only be compared if they have identical schemas. Pairwise comparisons are implemented recursively with a depth-first, left-to-right traversal of the schema. The first mismatch encountered determines the order of the items.
Two items with the same schema are compared according to the following rules.
- null data is always equal.
- boolean data is ordered with false before true.
- int, long, float and double data is ordered by ascending numeric value.
- bytes and fixed data are compared lexicographically by unsigned 8-bit values.
- string data is compared lexicographically by Unicode code point. Note that since UTF-8 is used as the binary encoding for strings, sorting of bytes and string binary data is identical.
- array data is compared lexicographically by element.
- enum data is ordered by the symbol’s position in the enum schema. For example, an enum whose symbols are
["z", "a"] would sort “z” values before “a” values.
- union data is first ordered by the branch within the union, and, within that, by the type of the branch. For example, an
["int", "string"] union would order all int values before all string values, with the ints and strings themselves ordered as defined above.
- record data is ordered lexicographically by field. If a field specifies that its order is:
- “ascending”, then the order of its values is unaltered.
- “descending”, then the order of its values is reversed.
- “ignore”, then its values are ignored when sorting.
- map data may not be compared. It is an error to attempt to compare data containing maps unless those maps are in an
"order":"ignore" record field.
Object Container Files
Avro includes a simple object container file format. A file has a schema, and all objects stored in the file must be written according to that schema, using binary encoding. Objects are stored in blocks that may be compressed. Syncronization markers are used between blocks to permit efficient splitting of files for MapReduce processing.
Files may include arbitrary user-specified metadata.
A file consists of:
- A file header, followed by
- one or more file data blocks.
A file header consists of:
- Four bytes, ASCII ‘O’, ‘b’, ‘j’, followed by 1.
- file metadata, including the schema.
- The 16-byte, randomly-generated sync marker for this file.
File metadata consists of:
A long indicating the number of metadata key/value pairs.
For each pair, a string key and bytes value.
All metadata properties that start with “avro.” are reserved. The following file metadata properties are currently used:
- avro.schema contains the schema of objects stored in the file, as JSON data (required).
- avro.codec the name of the compression codec used to compress blocks, as a string. Implementations are required to support the following codecs: “null” and “deflate”. If codec is absent, it is assumed to be “null”. The codecs are described with more detail below.
A file header is thus described by the following schema:
{"type": "record", "name": "org.apache.avro.file.Header",
"fields" : [
{"name": "magic", "type": {"type": "fixed", "name": "Magic", "size": 4}},
{"name": "meta", "type": {"type": "map", "values": "bytes"}},
{"name": "sync", "type": {"type": "fixed", "name": "Sync", "size": 16}}
]
}
A file data block consists of:
- A long indicating the count of objects in this block.
- A long indicating the size in bytes of the serialized objects in the current block, after any codec is applied
- The serialized objects. If a codec is specified, this is compressed by that codec.
- The file’s 16-byte sync marker.
Thus, each block’s binary data can be efficiently extracted or skipped without deserializing the contents. The combination of block size, object counts, and sync markers enable detection of corrupt blocks and help ensure data integrity.
Required Codecs
null
The “null” codec simply passes through data uncompressed.
deflate
The “deflate” codec writes the data block using the deflate algorithm as specified in RFC 1951, and typically implemented using the zlib library. Note that this format (unlike the “zlib format” in RFC 1950) does not have a checksum.
Optional Codecs
snappy
The “snappy” codec uses Google’s Snappy compression library. Each compressed block is followed by the 4-byte, big-endian CRC32 checksum of the uncompressed data in the block.
Protocol Declaration
Avro protocols describe RPC interfaces. Like schemas, they are defined with JSON text.
A protocol is a JSON object with the following attributes:
- protocol, a string, the name of the protocol (required);
- namespace, an optional string that qualifies the name;
- doc, an optional string describing this protocol;
- types, an optional list of definitions of named types (records, enums, fixed and errors). An error definition is just like a record definition except it uses “error” instead of “record”. Note that forward references to named types are not permitted.
- messages, an optional JSON object whose keys are message names and whose values are objects whose attributes are described below. No two messages may have the same name.
The name and namespace qualification rules defined for schema objects apply to protocols as well.
Messages
A message has attributes:
- a doc, an optional description of the message,
- a request, a list of named, typed parameter schemas (this has the same form as the fields of a record declaration);
- a response schema;
- an optional union of declared error schemas. The effective union has “string” prepended to the declared union, to permit transmission of undeclared “system” errors. For example, if the declared error union is
["AccessError"], then the effective union is ["string", "AccessError"]. When no errors are declared, the effective error union is ["string"]. Errors are serialized using the effective union; however, a protocol’s JSON declaration contains only the declared union.
- an optional one-way boolean parameter.
A request parameter list is processed equivalently to an anonymous record. Since record field lists may vary between reader and writer, request parameters may also differ between the caller and responder, and such differences are resolved in the same manner as record field differences.
The one-way parameter may only be true when the response type is "null" and no errors are listed.
Sample Protocol
For example, one may define a simple HelloWorld protocol with:
{
"namespace": "com.acme",
"protocol": "HelloWorld",
"doc": "Protocol Greetings",
"types": [
{"name": "Greeting", "type": "record", "fields": [
{"name": "message", "type": "string"}]},
{"name": "Curse", "type": "error", "fields": [
{"name": "message", "type": "string"}]}
],
"messages": {
"hello": {
"doc": "Say hello.",
"request": [{"name": "greeting", "type": "Greeting" }],
"response": "Greeting",
"errors": ["Curse"]
}
}
}
Message Transport
Messages may be transmitted via different transport mechanisms.
To the transport, a message is an opaque byte sequence.
A transport is a system that supports:
- transmission of request messages
- receipt of corresponding response messages
Servers may send a response message back to the client corresponding to a request message. The mechanism of correspondance is transport-specific. For example, in HTTP it is implicit, since HTTP directly supports requests and responses. But a transport that multiplexes many client threads over a single socket would need to tag messages with unique identifiers.
Transports may be either stateless or stateful. In a stateless transport, messaging assumes no established connection state, while stateful transports establish connections that may be used for multiple messages. This distinction is discussed further in the handshake section below.
HTTP as Transport
When HTTP is used as a transport, each Avro message exchange is an HTTP request/response pair. All messages of an Avro protocol should share a single URL at an HTTP server. Other protocols may also use that URL. Both normal and error Avro response messages should use the 200 (OK) response code. The chunked encoding may be used for requests and responses, but, regardless the Avro request and response are the entire content of an HTTP request and response. The HTTP Content-Type of requests and responses should be specified as “avro/binary”. Requests should be made using the POST method.
HTTP is used by Avro as a stateless transport.
Message Framing
Avro messages are framed as a list of buffers.
Framing is a layer between messages and the transport. It exists to optimize certain operations.
The format of framed message data is:
- a series of buffers, where each buffer consists of:
- a four-byte, big-endian buffer length, followed by
- that many bytes of buffer data.
- A message is always terminated by a zero-lenghted buffer.
Framing is transparent to request and response message formats (described below). Any message may be presented as a single or multiple buffers.
Framing can permit readers to more efficiently get different buffers from different sources and for writers to more efficiently store different buffers to different destinations. In particular, it can reduce the number of times large binary objects are copied. For example, if an RPC parameter consists of a megabyte of file data, that data can be copied directly to a socket from a file descriptor, and, on the other end, it could be written directly to a file descriptor, never entering user space.
A simple, recommended, framing policy is for writers to create a new segment whenever a single binary object is written that is larger than a normal output buffer. Small objects are then appended in buffers, while larger objects are written as their own buffers. When a reader then tries to read a large object the runtime can hand it an entire buffer directly, without having to copy it.
Handshake
The purpose of the handshake is to ensure that the client and the server have each other’s protocol definition, so that the client can correctly deserialize responses, and the server can correctly deserialize requests. Both clients and servers should maintain a cache of recently seen protocols, so that, in most cases, a handshake will be completed without extra round-trip network exchanges or the transmission of full protocol text.
RPC requests and responses may not be processed until a handshake has been completed. With a stateless transport, all requests and responses are prefixed by handshakes. With a stateful transport, handshakes are only attached to requests and responses until a successful handshake response has been returned over a connection. After this, request and response payloads are sent without handshakes for the lifetime of that connection.
The handshake process uses the following record schemas:
{
"type": "record",
"name": "HandshakeRequest", "namespace":"org.apache.avro.ipc",
"fields": [
{"name": "clientHash",
"type": {"type": "fixed", "name": "MD5", "size": 16}},
{"name": "clientProtocol", "type": ["null", "string"]},
{"name": "serverHash", "type": "MD5"},
{"name": "meta", "type": ["null", {"type": "map", "values": "bytes"}]}
]
}
{
"type": "record",
"name": "HandshakeResponse", "namespace": "org.apache.avro.ipc",
"fields": [
{"name": "match",
"type": {"type": "enum", "name": "HandshakeMatch",
"symbols": ["BOTH", "CLIENT", "NONE"]}},
{"name": "serverProtocol",
"type": ["null", "string"]},
{"name": "serverHash",
"type": ["null", {"type": "fixed", "name": "MD5", "size": 16}]},
{"name": "meta",
"type": ["null", {"type": "map", "values": "bytes"}]}
]
}
- A client first prefixes each request with a
HandshakeRequest containing just the hash of its protocol and of the server’s protocol (clientHash!=null, clientProtocol=null, serverHash!=null), where the hashes are 128-bit MD5 hashes of the JSON protocol text. If a client has never connected to a given server, it sends its hash as a guess of the server’s hash, otherwise it sends the hash that it previously obtained from this server.
The server responds with a HandshakeResponse containing one of:
-
match=BOTH, serverProtocol=null, serverHash=null if the client sent the valid hash of the server’s protocol and the server knows what protocol corresponds to the client’s hash. In this case, the request is complete and the response data immediately follows the HandshakeResponse.
-
match=CLIENT, serverProtocol!=null, serverHash!=null if the server has previously seen the client’s protocol, but the client sent an incorrect hash of the server’s protocol. The request is complete and the response data immediately follows the HandshakeResponse. The client must use the returned protocol to process the response and should also cache that protocol and its hash for future interactions with this server.
-
match=NONE if the server has not previously seen the client’s protocol. The serverHash and serverProtocol may also be non-null if the server’s protocol hash was incorrect.
In this case the client must then re-submit its request with its protocol text (clientHash!=null, clientProtocol!=null, serverHash!=null) and the server should respond with a successful match (match=BOTH, serverProtocol=null, serverHash=null) as above.
The meta field is reserved for future handshake enhancements.
A call consists of a request message paired with its resulting response or error message. Requests and responses contain extensible metadata, and both kinds of messages are framed as described above.
The format of a call request is:
- request metadata, a map with values of type bytes
- the message name, an Avro string, followed by
- the message parameters. Parameters are serialized according to the message’s request declaration.
When a message is declared one-way and a stateful connection has been established by a successful handshake response, no response data is sent. Otherwise the format of the call response is:
- response metadata, a map with values of type bytes
- a one-byte error flag boolean, followed by either:
- if the error flag is false, the message response, serialized per the message’s response schema.
- if the error flag is true, the error, serialized per the message’s effective error union schema.
Schema Resolution
A reader of Avro data, whether from an RPC or a file, can always parse that data because its schema is provided. But that schema may not be exactly the schema that was expected. For example, if the data was written with a different version of the software than it is read, then records may have had fields added or removed. This section specifies how such schema differences should be resolved.
We refer to the schema used to write the data as the writer’s schema, and the schema that the application expects the reader’s schema. Differences between these should be resolved as follows:
- It is an error if the two schemas do not match.
To match, one of the following must hold:
-
both schemas are arrays whose item types match
-
both schemas are maps whose value types match
-
both schemas are enums whose names match
-
both schemas are fixed whose sizes and names match
-
both schemas are records with the same name
-
either schema is a union
-
both schemas have same primitive type
-
the writer’s schema may be promoted to the reader’s as follows:
- int is promotable to long, float, or double
- long is promotable to float or double
- float is promotable to double
- string is promotable to bytes
- bytes is promotable to string
-
if both are records:
- the ordering of fields may be different: fields are matched by name.
- schemas for fields with the same name in both records are resolved recursively.
- if the writer’s record contains a field with a name not present in the reader’s record, the writer’s value for that field is ignored.
- if the reader’s record schema has a field that contains a default value, and writer’s schema does not have a field with the same name, then the reader should use the default value from its field.
- if the reader’s record schema has a field with no default value, and writer’s schema does not have a field with the same name, an error is signalled.
-
if both are enums:
if the writer’s symbol is not present in the reader’s enum, then an error is signalled.
This resolution algorithm is applied recursively to the reader’s and writer’s array item schemas.
This resolution algorithm is applied recursively to the reader’s and writer’s value schemas.
The first schema in the reader’s union that matches the selected writer’s union schema is recursively resolved against it. if none match, an error is signalled.
- if reader’s is a union, but writer’s is not:
The first schema in the reader’s union that matches the writer’s schema is recursively resolved against it. If none match, an error is signalled.
- if writer’s is a union, but reader’s is not:
If the reader’s schema matches the selected writer’s schema, it is recursively resolved against it. If they do not match, an error is signalled.
A schema’s “doc” fields are ignored for the purposes of schema resolution. Hence, the “doc” portion of a schema may be dropped at serialization.
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
20.2 - IDL Language
Introduction
This document defines Avro IDL, a higher-level language for authoring Avro schemata. Before reading this document, you should have familiarity with the concepts of schemata and protocols, as well as the various primitive and complex types available in Avro.
Overview
Purpose
The aim of the Avro IDL language is to enable developers to author schemata in a way that feels more similar to common programming languages like Java, C++, or Python. Additionally, the Avro IDL language may feel more familiar for those users who have previously used the interface description languages (IDLs) in other frameworks like Thrift, Protocol Buffers, or CORBA.
Usage
Each Avro IDL file defines a single Avro Protocol, and thus generates as its output a JSON-format Avro Protocol file with extension .avpr.
To convert a .avdl file into a .avpr file, it may be processed by the idl tool. For example:
$ java -jar avroj-tools.jar idl src/test/idl/input/namespaces.avdl /tmp/namespaces.avpr
$ head /tmp/namespaces.avpr
{
"protocol" : "TestNamespace",
"namespace" : "avro.test.protocol",
The idl tool can also process input to and from stdin and stdout. See idl --help for full usage information.
A Maven plugin is also provided to compile .avdl files. To use it, add something like the following to your pom.xml:
<build>
<plugins>
<plugin>
<groupId>org.apache.avro</groupId>
<artifactId>avro-maven-plugin</artifactId>
<executions>
<execution>
<goals>
<goal>idl-protocol</goal>
</goals>
</execution>
</executions>
</plugin>
</plugins>
</build>
Defining a Protocol in Avro IDL
An Avro IDL file consists of exactly one protocol definition. The minimal protocol is defined by the following code:
This is equivalent to (and generates) the following JSON protocol definition:
{
"protocol" : "MyProtocol",
"types" : [ ],
"messages" : {
}
}
The namespace of the protocol may be changed using the @namespace annotation:
@namespace("mynamespace")
protocol MyProtocol {
}
This notation is used throughout Avro IDL as a way of specifying properties for the annotated element, as will be described later in this document.
Protocols in Avro IDL can contain the following items:
- Imports of external protocol and schema files.
- Definitions of named schemata, including records, errors, enums, and fixeds.
- Definitions of RPC messages
Imports
Files may be imported in one of three formats:
-
An IDL file may be imported with a statement like:
import idl "foo.avdl";
-
A JSON protocol file may be imported with a statement like:
import protocol "foo.avpr";
-
A JSON schema file may be imported with a statement like:
import schema "foo.avsc";
Messages and types in the imported file are added to this file’s protocol.
Imported file names are resolved relative to the current IDL file.
Defining an Enumeration
Enums are defined in Avro IDL using a syntax similar to C or Java:
enum Suit {
SPADES, DIAMONDS, CLUBS, HEARTS
}
Note that, unlike the JSON format, anonymous enums cannot be defined.
Defining a Fixed Length Field
Fixed fields are defined using the following syntax:
fixed MD5(16);
This example defines a fixed-length type called MD5 which contains 16 bytes.
Defining Records and Errors
Records are defined in Avro IDL using a syntax similar to a struct definition in C:
record Employee {
string name;
boolean active = true;
long salary;
}
The above example defines a record with the name “Employee” with three fields.
To define an error, simply use the keyword error instead of record. For example:
error Kaboom {
string explanation;
int result_code = -1;
}
Each field in a record or error consists of a type and a name, optional property annotations and an optional default value.
A type reference in Avro IDL must be one of:
- A primitive type
- A logical type
- A named schema defined prior to this usage in the same Protocol
- A complex type (array, map, or union)
Primitive Types
The primitive types supported by Avro IDL are the same as those supported by Avro’s JSON format. This list includes int, long, string, boolean, float, double, null, and bytes.
References to Named Schemata
If a named schema has already been defined in the same Avro IDL file, it may be referenced by name as if it were a primitive type:
record Card {
Suit suit; // refers to the enum Card defined above
int number;
}
Default Values
Default values for fields may be optionally specified by using an equals sign after the field name followed by a JSON expression indicating the default value. This JSON is interpreted as described in the spec.
Complex Types
Arrays
Array types are written in a manner that will seem familiar to C++ or Java programmers. An array of any type t is denoted array<t>. For example, an array of strings is denoted array<string>, and a multidimensional array of Foo records would be array<array<Foo>>.
Maps
Map types are written similarly to array types. An array that contains values of type t is written map<t>. As in the JSON schema format, all maps contain string-type keys.
Unions
Union types are denoted as union { typeA, typeB, typeC, … }. For example, this record contains a string field that is optional (unioned with null):
record RecordWithUnion {
union { null, string } optionalString;
}
Note that the same restrictions apply to Avro IDL unions as apply to unions defined in the JSON format; namely, a record may not contain multiple elements of the same type.
Defining RPC Messages
The syntax to define an RPC message within a Avro IDL protocol is similar to the syntax for a method declaration within a C header file or a Java interface. To define an RPC message add which takes two arguments named foo and bar, returning an int, simply include the following definition within the protocol:
int add(int foo, int bar = 0);
Message arguments, like record fields, may specify default values.
To define a message with no response, you may use the alias void, equivalent to the Avro null type:
void logMessage(string message);
If you have previously defined an error type within the same protocol, you may declare that a message can throw this error using the syntax:
void goKaboom() throws Kaboom;
To define a one-way message, use the keyword oneway after the parameter list, for example:
void fireAndForget(string message) oneway;
Other Language Features
All Java-style comments are supported within a Avro IDL file. Any text following // on a line is ignored, as is any text between /* and */, possibly spanning multiple lines.
Comments that begin with /** are used as the documentation string for the type or field definition that follows the comment.
Escaping Identifiers
Occasionally, one will need to use a reserved language keyword as an identifier. In order to do so, backticks (`) may be used to escape the identifier. For example, to define a message with the literal name error, you may write:
This syntax is allowed anywhere an identifier is expected.
Annotations for Ordering and Namespaces
Java-style annotations may be used to add additional properties to types and fields throughout Avro IDL.
For example, to specify the sort order of a field within a record, one may use the @order annotation before the field name as follows:
record MyRecord {
string @order("ascending") myAscendingSortField;
string @order("descending") myDescendingField;
string @order("ignore") myIgnoredField;
}
A field’s type may also be preceded by annotations, e.g.:
record MyRecord {
@java-class("java.util.ArrayList") array<string> myStrings;
}
Similarly, a @namespace annotation may be used to modify the namespace when defining a named schema. For example:
@namespace("org.apache.avro.firstNamespace")
protocol MyProto {
@namespace("org.apache.avro.someOtherNamespace")
record Foo {}
record Bar {}
}
will define a protocol in the firstNamespace namespace. The record Foo will be defined in someOtherNamespace and Bar will be defined in firstNamespace as it inherits its default from its container.
Type and field aliases are specified with the @aliases annotation as follows:
@aliases(["org.old.OldRecord", "org.ancient.AncientRecord"])
record MyRecord {
string @aliases(["oldField", "ancientField"]) myNewField;
}
Some annotations like those listed above are handled specially. All other annotations are added as properties to the protocol, message, schema or field.
Complete Example
The following is a complete example of a Avro IDL file that shows most of the above features:
/**
* An example protocol in Avro IDL
*/
@namespace("org.apache.avro.test")
protocol Simple {
@aliases(["org.foo.KindOf"])
enum Kind {
FOO,
BAR, // the bar enum value
BAZ
}
fixed MD5(16);
record TestRecord {
@order("ignore")
string name;
@order("descending")
Kind kind;
MD5 hash;
union { MD5, null} @aliases(["hash"]) nullableHash;
array<long> arrayOfLongs;
}
error TestError {
string message;
}
string hello(string greeting);
TestRecord echo(TestRecord `record`);
int add(int arg1, int arg2);
bytes echoBytes(bytes data);
void `error`() throws TestError;
void ping() oneway;
}
Additional examples may be found in the Avro source tree under the src/test/idl/input directory.
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
20.3 - SASL profile
Introduction
SASL (RFC 2222) provides a framework for authentication and security of network protocols. Each protocol that uses SASL is meant to define a SASL profile. This document provides a SASL profile for connection-based Avro RPC.
Overview
SASL negotiation proceeds as a series of message interactions over a connection between a client and server using a selected SASL mechanism. The client starts this negotiation by sending its chosen mechanism name with an initial (possibly empty) message. Negotiation proceeds with the exchange of messages until either side indicates success or failure. The content of the messages is mechanism-specific. If the negotiation succeeds, then the session can proceed over the connection, otherwise it must be abandoned.
Some mechanisms continue to process session data after negotiation (e.g., encrypting it), while some specify that further session data is transmitted unmodified.
Negotiation
Commands
Avro SASL negotiation uses four one-byte commands.
- 0: START Used in a client’s initial message.
- 1: CONTINUE Used while negotiation is ongoing.
- 2: FAIL Terminates negotiation unsuccessfully.
- 3: COMPLETE Terminates negotiation successfully.
The format of a START message is:
| 0 | 4-byte mechanism name length | mechanism name | 4-byte payload length | payload data |
The format of a CONTINUE message is:
| 1 | 4-byte payload length | payload data |
The format of a FAIL message is:
| 2 | 4-byte message length | UTF-8 message |
The format of a COMPLETE message is:
| 3 | 4-byte payload length | payload data |
Process
Negotiation is initiated by a client sending a START command containing the client’s chosen mechanism name and any mechanism-specific payload data.
The server and client then interchange some number (possibly zero) of CONTINUE messages. Each message contains payload data that is processed by the security mechanism to generate the next message.
Once either the client or server send a FAIL message then negotiation has failed. UTF-8-encoded text is included in the failure message. Once either a FAIL message has been sent or received, or any other error occurs in the negotiation, further communication on this connection must cease.
Once either the client or server send a COMPLETE message then negotiation has completed successfully. Session data may now be transmitted over the connection until it is closed by either side.
Session Data
If no SASL QOP (quality of protection) is negotiated, then all subsequent writes to/reads over this connection are written/read unmodified. In particular, messages use Avro framing, and are of the form:
| 4-byte frame length | frame data | ... | 4 zero bytes |
If a SASL QOP is negotiated, then it must be used by the connection for all subsequent messages. This is done by wrapping each non-empty frame written using the security mechanism and unwrapping each non-empty frame read. The length written in each non-empty frame is the length of the wrapped data. Complete frames must be passed to the security mechanism for unwrapping. Unwrapped data is then passed to the application as the content of the frame.
If at any point processing fails due to wrapping, unwrapping or framing errors, then all further communication on this connection must cease.
Anonymous Mechanism
The SASL anonymous mechanism (RFC 2245) is quite simple to implement. In particular, an initial anonymous request may be prefixed by the following static sequence:
| 0 | 0009 | ANONYMOUS | 0000 |
If a server uses the anonymous mechanism, it should check that the mechanism name in the start message prefixing the first request received is ‘ANONYMOUS’, then simply prefix its initial response with a COMPLETE message of:
| 3 | 0000 |
If an anonymous server recieves some other mechanism name, then it may respond with a FAIL message as simple as:
| 2 | 0000 |
Note that the anonymous mechanism need add no additional round-trip messages between client and server. The START message can be piggybacked on the initial request and the COMPLETE or FAIL message can be piggybacked on the initial response.
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
20.4 - Wiki
The Wiki page can be found here.
20.5 - FAQ
What is Avro?
Avro is a data serialization system.
Avro provides:
- Rich data structures.
- A compact, fast, binary data format.
- A container file, to store persistent data.
- Remote procedure call (RPC).
- Simple integration with dynamic languages. Code generation is not required to read or write data files nor to use or implement RPC protocols. Code generation as an optional optimization, only worth implementing for statically typed languages.
How can I get started quickly with Avro?
Check out the Quick Start Guide.
How do I statically compile a schema or protocol into generated code?
In Java
- Add the avro jar, the jackson-mapper-asl.jar and jackson-core-asl.jar to your CLASSPATH.
- Run java org.apache.avro.specific.SpecificCompiler .
This appears to be out of date, the SpecificCompiler requires two arguments, presumably an input and and output file, but it isn’t clear that this does.
Or use the Schema or Protocol Ant tasks. Avro’s build.xml provides examples of how these are used.
Lastly, you can also use the “avro-tools” jar which ships with an Avro release. Just use the “compile (schema|protocol)” command.
How are Strings represented in Java?
They use org.apache.avro.util.Utf8, not java.lang.String.
More generally, how do Avro types map to Java types?
The mappings are documented in the package javadoc for generic, specific and reflect API.
From Doug Cutting:
HDFS splits files into blocks, and mapreduce runs a map task for each block. When the task starts, it needs to be able to seek into the file to the start of the block process through the block’s end. If the file were, e.g., a gzip file, this would not be possible, since gzip files must be decompressed from the start. One cannot seek into the middle of a gzip file and start decompressing. So Hadoop’s SequenceFile places a marker periodically (~64k) in the file at record and compression boundaries, where processing can be sensibly started. Then, when a map task starts processing an HDFS block, it finds the first marker after the block’s start and continues through the first marker in the next block of the file. This requires a bit of non-local access (~0.1%). Avro’s data file uses the same method as SequenceFile.
Why isn’t every value in Avro nullable?
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 of every value stored and
corresponding computational costs to create this bit on write and
interpret it on read. These costs are wasted when values may not in
fact be null, as is the case in many datasets. In Avro such costs are
only paid when values may actually be null.
Also, allowing values to be null is a well-known source of errors. In
Avro, a value declared as non-null will always be non-null and programs
need not test for null values when processing it nor will they ever fail
for lack of such tests.
Tony Hoare calls his invention of null references his “Billion Dollar
Mistake”.
http://qconlondon.com/london-2009/presentation/Null+References:+The+Billion+Dollar+Mistake
Also note that in some programming languages not all values are permitted to be null. For example, in Java, values of type boolean, byte, short, char, int, float, long, and double may not be null.
How can I serialize directly to/from a byte array?
As pointed out in the specification, Avro data should always be stored with its schema. The Avro provided classes DataFileWriter, DataFileReader, and DataFileStream all ensure this by serializing the Schema in a container header. In some special cases, such as when implementing a new storage system or writing unit tests, you may need to write and read directly with the bare Avro serialized values. The following examples use code generated for Java from an Avro Schema, specifically the User example from the Quickstart guide.
Serializing to a byte array
This example takes a User object and returns a newly allocated byte array with the Avro serialization of that user.
ByteArrayOutputStream out = new ByteArrayOutputStream();
BinaryEncoder encoder = EncoderFactory.get().binaryEncoder(out, null);
DatumWriter<User> writer = new SpecificDatumWriter<User>(User.getClassSchema());
writer.write(user, encoder);
encoder.flush();
out.close();
byte[] serializedBytes = out.toByteArray();
Deserializing from a byte array
This example takes a byte array containing the Avro serialization of a user and returns a User object.SpecificDatumReader reader = new SpecificDatumReader(User.getClassSchema());
Decoder decoder = DecoderFactory.get().binaryDecoder(bytes, null);
User user = reader.read(null, decoder);
Note: If you are serializing or deserializing in a loop or as a method, you should be reusing objects, readers and/or writers. Check the JavaDoc to see how to reuse the objects.
21 - Apache Avro™ 1.6.2 Documentation
Introduction
Apache Avro™ is a data serialization system.
Avro provides:
- Rich data structures.
- A compact, fast, binary data format.
- A container file, to store persistent data.
- Remote procedure call (RPC).
- Simple integration with dynamic languages. Code generation is not required to read or write data files nor to use or implement RPC protocols. Code generation as an optional optimization, only worth implementing for statically typed languages.
Schemas
Avro relies on schemas. When Avro data is read, the schema used when writing it is always present. This permits each datum to be written with no per-value overheads, making serialization both fast and small. This also facilitates use with dynamic, scripting languages, since data, together with its schema, is fully self-describing.
When Avro data is stored in a file, its schema is stored with it, so that files may be processed later by any program. If the program reading the data expects a different schema this can be easily resolved, since both schemas are present.
When Avro is used in RPC, the client and server exchange schemas in the connection handshake. (This can be optimized so that, for most calls, no schemas are actually transmitted.) Since both client and server both have the other’s full schema, correspondence between same named fields, missing fields, extra fields, etc. can all be easily resolved.
Avro schemas are defined with JSON . This facilitates implementation in languages that already have JSON libraries.
Comparison with other systems
Avro provides functionality similar to systems such as Thrift, Protocol Buffers, etc. Avro differs from these systems in the following fundamental aspects.
- Dynamic typing: Avro does not require that code be generated. Data is always accompanied by a schema that permits full processing of that data without code generation, static datatypes, etc. This facilitates construction of generic data-processing systems and languages.
- Untagged data: Since the schema is present when data is read, considerably less type information need be encoded with data, resulting in smaller serialization size.
- No manually-assigned field IDs: When a schema changes, both the old and new schema are always present when processing data, so differences may be resolved symbolically, using field names.
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
21.1 - Specification
Introduction
This document defines Apache Avro. It is intended to be the authoritative specification. Implementations of Avro must adhere to this document.
Schema Declaration
A Schema is represented in JSON by one of:
- A JSON string, naming a defined type.
- A JSON object, of the form:
{"type": "typeName" ...attributes...}
where typeName is either a primitive or derived type name, as defined below. Attributes not defined in this document are permitted as metadata, but must not affect the format of serialized data.
- A JSON array, representing a union of embedded types.
Primitive Types
The set of primitive type names is:
- null: no value
- boolean: a binary value
- int: 32-bit signed integer
- long: 64-bit signed integer
- float: single precision (32-bit) IEEE 754 floating-point number
- double: double precision (64-bit) IEEE 754 floating-point number
- bytes: sequence of 8-bit unsigned bytes
- string: unicode character sequence
Primitive types have no specified attributes.
Primitive type names are also defined type names. Thus, for example, the schema “string” is equivalent to:
Complex Types
Avro supports six kinds of complex types: records, enums, arrays, maps, unions and fixed.
Records
Records use the type name “record” and support three attributes:
- name: a JSON string providing the name of the record (required).
- namespace, a JSON string that qualifies the name;
- doc: a JSON string providing documentation to the user of this schema (optional).
- aliases: a JSON array of strings, providing alternate names for this record (optional).
- fields: a JSON array, listing fields (required). Each field is a JSON object with the following attributes:
- name: a JSON string providing the name of the field (required), and
- doc: a JSON string describing this field for users (optional).
- type: A JSON object defining a schema, or a JSON string naming a record definition (required).
- default: A default value for this field, used when reading instances that lack this field (optional). 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. Default values for bytes and fixed fields are JSON strings, where Unicode code points 0-255 are mapped to unsigned 8-bit byte values 0-255.
field default values
| avro type |
json type |
example |
| null |
null |
null |
| boolean |
boolean |
true |
| int,long |
integer |
1 |
| float,double |
number |
1.1 |
| bytes |
string |
"\u00FF" |
| string |
string |
"foo" |
| record |
object |
{"a": 1} |
| enum |
string |
"FOO" |
| array |
array |
[1] |
| map |
object |
{"a": 1} |
| fixed |
string |
"\u00ff" |
- order: specifies how this field impacts sort ordering of this record (optional). Valid values are “ascending” (the default), “descending”, or “ignore”. For more details on how this is used, see the sort order section below.
- aliases: a JSON array of strings, providing alternate names for this field (optional).
For example, a linked-list of 64-bit values may be defined with:
{
"type": "record",
"name": "LongList",
"aliases": ["LinkedLongs"], // old name for this
"fields" : [
{"name": "value", "type": "long"}, // each element has a long
{"name": "next", "type": ["LongList", "null"]} // optional next element
]
}
Enums
Enums use the type name “enum” and support the following attributes:
- name: a JSON string providing the name of the enum (required).
- namespace, a JSON string that qualifies the name ;
- aliases: a JSON array of strings, providing alternate names for this enum (optional).
- doc: a JSON string providing documentation to the user of this schema (optional).
- symbols: a JSON array, listing symbols, as JSON strings (required). All symbols in an enum must be unique; duplicates are prohibited.
For example, playing card suits might be defined with:
{
"type": "enum",
"name": "Suit",
"symbols" : ["SPADES", "HEARTS", "DIAMONDS", "CLUBS"]
}
Arrays
Arrays use the type name “array” and support a single attribute:
- items: the schema of the array’s items.
For example, an array of strings is declared with:
{
"type": "array",
"items" : "string"
}
Maps
Maps use the type name “map” and support one attribute:
- values: the schema of the map’s values.
Map keys are assumed to be strings.
For example, a map from string to long is declared with:
{
"type": "map",
"values" : "long"
}
Unions
Unions, as mentioned above, are represented using JSON arrays. For example, ["string", "null"] declares a schema which may be either a string or null.
Unions may not contain more than one schema with the same type, except for the named types record, fixed and enum. For example, unions containing two array types or two map types are not permitted, but two types with different names are permitted. (Names permit efficient resolution when reading and writing unions.)
Unions may not immediately contain other unions.
Fixed
Fixed uses the type name “fixed” and supports two attributes:
- name: a string naming this fixed (required).
- namespace, a string that qualifies the name;
- aliases: a JSON array of strings, providing alternate names for this enum (optional).
- size: an integer, specifying the number of bytes per value (required).
For example, 16-byte quantity may be declared with:
{"type": "fixed", "size": 16, "name": "md5"}
Names
Record, enums and fixed are named types. Each has a fullname that is composed of two parts; a name and a namespace. Equality of names is defined on the fullname.
The name portion of a fullname, record field names, and enum symbols must:
- start with [A-Za-z_]
- subsequently contain only [A-Za-z0-9_]
A namespace is a dot-separated sequence of such names. Equality of names (including field names and enum symbols) as well as fullnames is case-sensitive.
In record, enum and fixed definitions, the fullname is determined in one of the following ways:
A name and namespace are both specified. For example, one might use “name”: “X”, “namespace”: “org.foo” to indicate the fullname org.foo.X.
-
A fullname is specified. If the name specified contains a dot, then it is assumed to be a fullname, and any namespace also specified is ignored. For example, use “name”: “org.foo.X” to indicate the fullname org.foo.X.
-
A name only is specified, i.e., a name that contains no dots. In this case the namespace is taken from the most tightly enclosing schema or protocol. For example, if “name”: “X” is specified, and this occurs within a field of the record definition of org.foo.Y, then the fullname is org.foo.X.
References to previously defined names are as in the latter two cases above: if they contain a dot they are a fullname, if they do not contain a dot, the namespace is the namespace of the enclosing definition.
Primitive type names have no namespace and their names may not be defined in any namespace.
A schema or protocol may not contain multiple definitions of a fullname. Further, a name must be defined before it is used (“before” in the depth-first, left-to-right traversal of the JSON parse tree, where the types attribute of a protocol is always deemed to come “before” the messages attribute.)
Aliases
Named types and fields may have aliases. An implementation may optionally use aliases to map a writer’s schema to the reader’s. This facilitates both schema evolution as well as processing disparate datasets.
Aliases function by re-writing the writer’s schema using aliases from the reader’s schema. For example, if the writer’s schema was named “Foo” and the reader’s schema is named “Bar” and has an alias of “Foo”, then the implementation would act as though “Foo” were named “Bar” when reading. Similarly, if data was written as a record with a field named “x” and is read as a record with a field named “y” with alias “x”, then the implementation would act as though “x” were named “y” when reading.
A type alias may be specified either as a fully namespace-qualified, or relative to the namespace of the name it is an alias for. For example, if a type named “a.b” has aliases of “c” and “x.y”, then the fully qualified names of its aliases are “a.c” and “x.y”.
Data Serialization
Avro data is always serialized with its schema. Files that store Avro data should always also include the schema for that data in the same file. Avro-based remote procedure call (RPC) systems must also guarantee that remote recipients of data have a copy of the schema used to write that data.
Because the schema used to write data is always available when the data is read, Avro data itself is not tagged with type information. The schema is required to parse data.
In general, both serialization and deserialization proceed as a depth-first, left-to-right traversal of the schema, serializing primitive types as they are encountered.
Encodings
Avro specifies two serialization encodings: binary and JSON. Most applications will use the binary encoding, as it is smaller and faster. But, for debugging and web-based applications, the JSON encoding may sometimes be appropriate.
Binary Encoding
Primitive Types
Primitive types are encoded in binary as follows:
- null is written as zero bytes.
- a boolean is written as a single byte whose value is either 0 (false) or 1 (true).
- int and long values are written using variable-length zig-zag coding. Some examples:
| value |
hex |
| 0 |
00 |
| -1 |
01 |
| 1 |
02 |
| -2 |
03 |
| 2 |
04 |
| … |
|
| -64 |
7f |
| 64 |
80 01 |
| … |
|
- a float is written as 4 bytes. The float is converted into a 32-bit integer using a method equivalent to Java’s floatToIntBits and then encoded in little-endian format.
- a double is written as 8 bytes. The double is converted into a 64-bit integer using a method equivalent to Java’s doubleToLongBits and then encoded in little-endian format.
- bytes are encoded as a long followed by that many bytes of data.
- a string is encoded as a long followed by that many bytes of UTF-8 encoded character data.
For example, the three-character string “foo” would be encoded as the long value 3 (encoded as hex 06) followed by the UTF-8 encoding of ‘f’, ‘o’, and ‘o’ (the hex bytes 66 6f 6f):
06 66 6f 6f
Complex Types
Complex types are encoded in binary as follows:
Records
A record is encoded by encoding the values of its fields in the order that they are declared. In other words, a record is encoded as just the concatenation of the encodings of its fields. Field values are encoded per their schema.
For example, the record schema
{
"type": "record",
"name": "test",
"fields" : [
{"name": "a", "type": "long"},
{"name": "b", "type": "string"}
]
}
An instance of this record whose a field has value 27 (encoded as hex 36) and whose b field has value “foo” (encoded as hex bytes 06 66 6f 6f), would be encoded simply as the concatenation of these, namely the hex byte sequence:
36 06 66 6f 6f
Enums
An enum is encoded by a int, representing the zero-based position of the symbol in the schema.
For example, consider the enum:
{"type": "enum", "name": "Foo", "symbols": ["A", "B", "C", "D"] }
This would be encoded by an int between zero and three, with zero indicating “A”, and 3 indicating “D”.
Arrays
Arrays are encoded as a series of blocks. Each block consists of a long count value, followed by that many array items. A block with count zero indicates the end of the array. Each item is encoded per the array’s item schema.
If a block’s count is negative, its absolute value is used, and the count is followed immediately by a long block size indicating the number of bytes in the block. This block size permits fast skipping through data, e.g., when projecting a record to a subset of its fields.
For example, the array schema
{"type": "array", "items": "long"}
an array containing the items 3 and 27 could be encoded as the long value 2 (encoded as hex 04) followed by long values 3 and 27 (encoded as hex 06 36) terminated by zero:
04 06 36 00
The blocked representation permits one to read and write arrays larger than can be buffered in memory, since one can start writing items without knowing the full length of the array.
Maps
Maps are encoded as a series of blocks. Each block consists of a long count value, followed by that many key/value pairs. A block with count zero indicates the end of the map. Each item is encoded per the map’s value schema.
If a block’s count is negative, its absolute value is used, and the count is followed immediately by a long block size indicating the number of bytes in the block. This block size permits fast skipping through data, e.g., when projecting a record to a subset of its fields.
The blocked representation permits one to read and write maps larger than can be buffered in memory, since one can start writing items without knowing the full length of the map.
Unions
A union is encoded by first writing a long value indicating the zero-based position within the union of the schema of its value. The value is then encoded per the indicated schema within the union.
For example, the union schema ["string","null"] would encode:
-
null as the integer 1 (the index of “null” in the union, encoded as hex 02):
02
-
the string “a” as zero (the index of “string” in the union), followed by the serialized string:
02 02 61
Fixed
Fixed instances are encoded using the number of bytes declared in the schema.
JSON Encoding
Except for unions, the JSON encoding is the same as is used to encode field default values.
The value of a union is encoded in JSON as follows:
- if its type is null, then it is encoded as a JSON null;
- otherwise it is encoded as a JSON object with one name/value pair whose name is the type’s name and whose value is the recursively encoded value. For Avro’s named types (record, fixed or enum) the user-specified name is used, for other types the type name is used.
For example, the union schema ["null","string","Foo"], where Foo is a record name, would encode:
- null as null;
- the string “a” as
{"string": "a"} and
- a Foo instance as
{"Foo": {...}}, where {...} indicates the JSON encoding of a Foo instance.
Note that a schema is still required to correctly process JSON-encoded data. For example, the JSON encoding does not distinguish between int and long, float and double, records and maps, enums and strings, etc.
Sort Order
Avro defines a standard sort order for data. This permits data written by one system to be efficiently sorted by another system. This can be an important optimization, as sort order comparisons are sometimes the most frequent per-object operation. Note also that Avro binary-encoded data can be efficiently ordered without deserializing it to objects.
Data items may only be compared if they have identical schemas. Pairwise comparisons are implemented recursively with a depth-first, left-to-right traversal of the schema. The first mismatch encountered determines the order of the items.
Two items with the same schema are compared according to the following rules.
- null data is always equal.
- boolean data is ordered with false before true.
- int, long, float and double data is ordered by ascending numeric value.
- bytes and fixed data are compared lexicographically by unsigned 8-bit values.
- string data is compared lexicographically by Unicode code point. Note that since UTF-8 is used as the binary encoding for strings, sorting of bytes and string binary data is identical.
- array data is compared lexicographically by element.
- enum data is ordered by the symbol’s position in the enum schema. For example, an enum whose symbols are
["z", "a"] would sort “z” values before “a” values.
- union data is first ordered by the branch within the union, and, within that, by the type of the branch. For example, an
["int", "string"] union would order all int values before all string values, with the ints and strings themselves ordered as defined above.
- record data is ordered lexicographically by field. If a field specifies that its order is:
- “ascending”, then the order of its values is unaltered.
- “descending”, then the order of its values is reversed.
- “ignore”, then its values are ignored when sorting.
- map data may not be compared. It is an error to attempt to compare data containing maps unless those maps are in an
"order":"ignore" record field.
Object Container Files
Avro includes a simple object container file format. A file has a schema, and all objects stored in the file must be written according to that schema, using binary encoding. Objects are stored in blocks that may be compressed. Syncronization markers are used between blocks to permit efficient splitting of files for MapReduce processing.
Files may include arbitrary user-specified metadata.
A file consists of:
- A file header, followed by
- one or more file data blocks.
A file header consists of:
- Four bytes, ASCII ‘O’, ‘b’, ‘j’, followed by 1.
- file metadata, including the schema.
- The 16-byte, randomly-generated sync marker for this file.
File metadata consists of:
A long indicating the number of metadata key/value pairs.
For each pair, a string key and bytes value.
All metadata properties that start with “avro.” are reserved. The following file metadata properties are currently used:
- avro.schema contains the schema of objects stored in the file, as JSON data (required).
- avro.codec the name of the compression codec used to compress blocks, as a string. Implementations are required to support the following codecs: “null” and “deflate”. If codec is absent, it is assumed to be “null”. The codecs are described with more detail below.
A file header is thus described by the following schema:
{"type": "record", "name": "org.apache.avro.file.Header",
"fields" : [
{"name": "magic", "type": {"type": "fixed", "name": "Magic", "size": 4}},
{"name": "meta", "type": {"type": "map", "values": "bytes"}},
{"name": "sync", "type": {"type": "fixed", "name": "Sync", "size": 16}}
]
}
A file data block consists of:
- A long indicating the count of objects in this block.
- A long indicating the size in bytes of the serialized objects in the current block, after any codec is applied
- The serialized objects. If a codec is specified, this is compressed by that codec.
- The file’s 16-byte sync marker.
Thus, each block’s binary data can be efficiently extracted or skipped without deserializing the contents. The combination of block size, object counts, and sync markers enable detection of corrupt blocks and help ensure data integrity.
Required Codecs
null
The “null” codec simply passes through data uncompressed.
deflate
The “deflate” codec writes the data block using the deflate algorithm as specified in RFC 1951, and typically implemented using the zlib library. Note that this format (unlike the “zlib format” in RFC 1950) does not have a checksum.
Optional Codecs
snappy
The “snappy” codec uses Google’s Snappy compression library. Each compressed block is followed by the 4-byte, big-endian CRC32 checksum of the uncompressed data in the block.
Protocol Declaration
Avro protocols describe RPC interfaces. Like schemas, they are defined with JSON text.
A protocol is a JSON object with the following attributes:
- protocol, a string, the name of the protocol (required);
- namespace, an optional string that qualifies the name;
- doc, an optional string describing this protocol;
- types, an optional list of definitions of named types (records, enums, fixed and errors). An error definition is just like a record definition except it uses “error” instead of “record”. Note that forward references to named types are not permitted.
- messages, an optional JSON object whose keys are message names and whose values are objects whose attributes are described below. No two messages may have the same name.
The name and namespace qualification rules defined for schema objects apply to protocols as well.
Messages
A message has attributes:
- a doc, an optional description of the message,
- a request, a list of named, typed parameter schemas (this has the same form as the fields of a record declaration);
- a response schema;
- an optional union of declared error schemas. The effective union has “string” prepended to the declared union, to permit transmission of undeclared “system” errors. For example, if the declared error union is
["AccessError"], then the effective union is ["string", "AccessError"]. When no errors are declared, the effective error union is ["string"]. Errors are serialized using the effective union; however, a protocol’s JSON declaration contains only the declared union.
- an optional one-way boolean parameter.
A request parameter list is processed equivalently to an anonymous record. Since record field lists may vary between reader and writer, request parameters may also differ between the caller and responder, and such differences are resolved in the same manner as record field differences.
The one-way parameter may only be true when the response type is "null" and no errors are listed.
Sample Protocol
For example, one may define a simple HelloWorld protocol with:
{
"namespace": "com.acme",
"protocol": "HelloWorld",
"doc": "Protocol Greetings",
"types": [
{"name": "Greeting", "type": "record", "fields": [
{"name": "message", "type": "string"}]},
{"name": "Curse", "type": "error", "fields": [
{"name": "message", "type": "string"}]}
],
"messages": {
"hello": {
"doc": "Say hello.",
"request": [{"name": "greeting", "type": "Greeting" }],
"response": "Greeting",
"errors": ["Curse"]
}
}
}
Message Transport
Messages may be transmitted via different transport mechanisms.
To the transport, a message is an opaque byte sequence.
A transport is a system that supports:
- transmission of request messages
- receipt of corresponding response messages
Servers may send a response message back to the client corresponding to a request message. The mechanism of correspondance is transport-specific. For example, in HTTP it is implicit, since HTTP directly supports requests and responses. But a transport that multiplexes many client threads over a single socket would need to tag messages with unique identifiers.
Transports may be either stateless or stateful. In a stateless transport, messaging assumes no established connection state, while stateful transports establish connections that may be used for multiple messages. This distinction is discussed further in the handshake section below.
HTTP as Transport
When HTTP is used as a transport, each Avro message exchange is an HTTP request/response pair. All messages of an Avro protocol should share a single URL at an HTTP server. Other protocols may also use that URL. Both normal and error Avro response messages should use the 200 (OK) response code. The chunked encoding may be used for requests and responses, but, regardless the Avro request and response are the entire content of an HTTP request and response. The HTTP Content-Type of requests and responses should be specified as “avro/binary”. Requests should be made using the POST method.
HTTP is used by Avro as a stateless transport.
Message Framing
Avro messages are framed as a list of buffers.
Framing is a layer between messages and the transport. It exists to optimize certain operations.
The format of framed message data is:
- a series of buffers, where each buffer consists of:
- a four-byte, big-endian buffer length, followed by
- that many bytes of buffer data.
- A message is always terminated by a zero-lenghted buffer.
Framing is transparent to request and response message formats (described below). Any message may be presented as a single or multiple buffers.
Framing can permit readers to more efficiently get different buffers from different sources and for writers to more efficiently store different buffers to different destinations. In particular, it can reduce the number of times large binary objects are copied. For example, if an RPC parameter consists of a megabyte of file data, that data can be copied directly to a socket from a file descriptor, and, on the other end, it could be written directly to a file descriptor, never entering user space.
A simple, recommended, framing policy is for writers to create a new segment whenever a single binary object is written that is larger than a normal output buffer. Small objects are then appended in buffers, while larger objects are written as their own buffers. When a reader then tries to read a large object the runtime can hand it an entire buffer directly, without having to copy it.
Handshake
The purpose of the handshake is to ensure that the client and the server have each other’s protocol definition, so that the client can correctly deserialize responses, and the server can correctly deserialize requests. Both clients and servers should maintain a cache of recently seen protocols, so that, in most cases, a handshake will be completed without extra round-trip network exchanges or the transmission of full protocol text.
RPC requests and responses may not be processed until a handshake has been completed. With a stateless transport, all requests and responses are prefixed by handshakes. With a stateful transport, handshakes are only attached to requests and responses until a successful handshake response has been returned over a connection. After this, request and response payloads are sent without handshakes for the lifetime of that connection.
The handshake process uses the following record schemas:
{
"type": "record",
"name": "HandshakeRequest", "namespace":"org.apache.avro.ipc",
"fields": [
{"name": "clientHash",
"type": {"type": "fixed", "name": "MD5", "size": 16}},
{"name": "clientProtocol", "type": ["null", "string"]},
{"name": "serverHash", "type": "MD5"},
{"name": "meta", "type": ["null", {"type": "map", "values": "bytes"}]}
]
}
{
"type": "record",
"name": "HandshakeResponse", "namespace": "org.apache.avro.ipc",
"fields": [
{"name": "match",
"type": {"type": "enum", "name": "HandshakeMatch",
"symbols": ["BOTH", "CLIENT", "NONE"]}},
{"name": "serverProtocol",
"type": ["null", "string"]},
{"name": "serverHash",
"type": ["null", {"type": "fixed", "name": "MD5", "size": 16}]},
{"name": "meta",
"type": ["null", {"type": "map", "values": "bytes"}]}
]
}
- A client first prefixes each request with a
HandshakeRequest containing just the hash of its protocol and of the server’s protocol (clientHash!=null, clientProtocol=null, serverHash!=null), where the hashes are 128-bit MD5 hashes of the JSON protocol text. If a client has never connected to a given server, it sends its hash as a guess of the server’s hash, otherwise it sends the hash that it previously obtained from this server.
The server responds with a HandshakeResponse containing one of:
-
match=BOTH, serverProtocol=null, serverHash=null if the client sent the valid hash of the server’s protocol and the server knows what protocol corresponds to the client’s hash. In this case, the request is complete and the response data immediately follows the HandshakeResponse.
-
match=CLIENT, serverProtocol!=null, serverHash!=null if the server has previously seen the client’s protocol, but the client sent an incorrect hash of the server’s protocol. The request is complete and the response data immediately follows the HandshakeResponse. The client must use the returned protocol to process the response and should also cache that protocol and its hash for future interactions with this server.
-
match=NONE if the server has not previously seen the client’s protocol. The serverHash and serverProtocol may also be non-null if the server’s protocol hash was incorrect.
In this case the client must then re-submit its request with its protocol text (clientHash!=null, clientProtocol!=null, serverHash!=null) and the server should respond with a successful match (match=BOTH, serverProtocol=null, serverHash=null) as above.
The meta field is reserved for future handshake enhancements.
A call consists of a request message paired with its resulting response or error message. Requests and responses contain extensible metadata, and both kinds of messages are framed as described above.
The format of a call request is:
- request metadata, a map with values of type bytes
- the message name, an Avro string, followed by
- the message parameters. Parameters are serialized according to the message’s request declaration.
When a message is declared one-way and a stateful connection has been established by a successful handshake response, no response data is sent. Otherwise the format of the call response is:
- response metadata, a map with values of type bytes
- a one-byte error flag boolean, followed by either:
- if the error flag is false, the message response, serialized per the message’s response schema.
- if the error flag is true, the error, serialized per the message’s effective error union schema.
Schema Resolution
A reader of Avro data, whether from an RPC or a file, can always parse that data because its schema is provided. But that schema may not be exactly the schema that was expected. For example, if the data was written with a different version of the software than it is read, then records may have had fields added or removed. This section specifies how such schema differences should be resolved.
We refer to the schema used to write the data as the writer’s schema, and the schema that the application expects the reader’s schema. Differences between these should be resolved as follows:
- It is an error if the two schemas do not match.
To match, one of the following must hold:
-
both schemas are arrays whose item types match
-
both schemas are maps whose value types match
-
both schemas are enums whose names match
-
both schemas are fixed whose sizes and names match
-
both schemas are records with the same name
-
either schema is a union
-
both schemas have same primitive type
-
the writer’s schema may be promoted to the reader’s as follows:
- int is promotable to long, float, or double
- long is promotable to float or double
- float is promotable to double
- string is promotable to bytes
- bytes is promotable to string
-
if both are records:
- the ordering of fields may be different: fields are matched by name.
- schemas for fields with the same name in both records are resolved recursively.
- if the writer’s record contains a field with a name not present in the reader’s record, the writer’s value for that field is ignored.
- if the reader’s record schema has a field that contains a default value, and writer’s schema does not have a field with the same name, then the reader should use the default value from its field.
- if the reader’s record schema has a field with no default value, and writer’s schema does not have a field with the same name, an error is signalled.
-
if both are enums:
if the writer’s symbol is not present in the reader’s enum, then an error is signalled.
This resolution algorithm is applied recursively to the reader’s and writer’s array item schemas.
This resolution algorithm is applied recursively to the reader’s and writer’s value schemas.
The first schema in the reader’s union that matches the selected writer’s union schema is recursively resolved against it. if none match, an error is signalled.
- if reader’s is a union, but writer’s is not:
The first schema in the reader’s union that matches the writer’s schema is recursively resolved against it. If none match, an error is signalled.
- if writer’s is a union, but reader’s is not:
If the reader’s schema matches the selected writer’s schema, it is recursively resolved against it. If they do not match, an error is signalled.
A schema’s “doc” fields are ignored for the purposes of schema resolution. Hence, the “doc” portion of a schema may be dropped at serialization.
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
21.2 - IDL Language
Introduction
This document defines Avro IDL, a higher-level language for authoring Avro schemata. Before reading this document, you should have familiarity with the concepts of schemata and protocols, as well as the various primitive and complex types available in Avro.
Overview
Purpose
The aim of the Avro IDL language is to enable developers to author schemata in a way that feels more similar to common programming languages like Java, C++, or Python. Additionally, the Avro IDL language may feel more familiar for those users who have previously used the interface description languages (IDLs) in other frameworks like Thrift, Protocol Buffers, or CORBA.
Usage
Each Avro IDL file defines a single Avro Protocol, and thus generates as its output a JSON-format Avro Protocol file with extension .avpr.
To convert a .avdl file into a .avpr file, it may be processed by the idl tool. For example:
$ java -jar avroj-tools.jar idl src/test/idl/input/namespaces.avdl /tmp/namespaces.avpr
$ head /tmp/namespaces.avpr
{
"protocol" : "TestNamespace",
"namespace" : "avro.test.protocol",
The idl tool can also process input to and from stdin and stdout. See idl --help for full usage information.
A Maven plugin is also provided to compile .avdl files. To use it, add something like the following to your pom.xml:
<build>
<plugins>
<plugin>
<groupId>org.apache.avro</groupId>
<artifactId>avro-maven-plugin</artifactId>
<executions>
<execution>
<goals>
<goal>idl-protocol</goal>
</goals>
</execution>
</executions>
</plugin>
</plugins>
</build>
Defining a Protocol in Avro IDL
An Avro IDL file consists of exactly one protocol definition. The minimal protocol is defined by the following code:
This is equivalent to (and generates) the following JSON protocol definition:
{
"protocol" : "MyProtocol",
"types" : [ ],
"messages" : {
}
}
The namespace of the protocol may be changed using the @namespace annotation:
@namespace("mynamespace")
protocol MyProtocol {
}
This notation is used throughout Avro IDL as a way of specifying properties for the annotated element, as will be described later in this document.
Protocols in Avro IDL can contain the following items:
- Imports of external protocol and schema files.
- Definitions of named schemata, including records, errors, enums, and fixeds.
- Definitions of RPC messages
Imports
Files may be imported in one of three formats:
-
An IDL file may be imported with a statement like:
import idl "foo.avdl";
-
A JSON protocol file may be imported with a statement like:
import protocol "foo.avpr";
-
A JSON schema file may be imported with a statement like:
import schema "foo.avsc";
Messages and types in the imported file are added to this file’s protocol.
Imported file names are resolved relative to the current IDL file.
Defining an Enumeration
Enums are defined in Avro IDL using a syntax similar to C or Java:
enum Suit {
SPADES, DIAMONDS, CLUBS, HEARTS
}
Note that, unlike the JSON format, anonymous enums cannot be defined.
Defining a Fixed Length Field
Fixed fields are defined using the following syntax:
fixed MD5(16);
This example defines a fixed-length type called MD5 which contains 16 bytes.
Defining Records and Errors
Records are defined in Avro IDL using a syntax similar to a struct definition in C:
record Employee {
string name;
boolean active = true;
long salary;
}
The above example defines a record with the name “Employee” with three fields.
To define an error, simply use the keyword error instead of record. For example:
error Kaboom {
string explanation;
int result_code = -1;
}
Each field in a record or error consists of a type and a name, optional property annotations and an optional default value.
A type reference in Avro IDL must be one of:
- A primitive type
- A logical type
- A named schema defined prior to this usage in the same Protocol
- A complex type (array, map, or union)
Primitive Types
The primitive types supported by Avro IDL are the same as those supported by Avro’s JSON format. This list includes int, long, string, boolean, float, double, null, and bytes.
References to Named Schemata
If a named schema has already been defined in the same Avro IDL file, it may be referenced by name as if it were a primitive type:
record Card {
Suit suit; // refers to the enum Card defined above
int number;
}
Default Values
Default values for fields may be optionally specified by using an equals sign after the field name followed by a JSON expression indicating the default value. This JSON is interpreted as described in the spec.
Complex Types
Arrays
Array types are written in a manner that will seem familiar to C++ or Java programmers. An array of any type t is denoted array<t>. For example, an array of strings is denoted array<string>, and a multidimensional array of Foo records would be array<array<Foo>>.
Maps
Map types are written similarly to array types. An array that contains values of type t is written map<t>. As in the JSON schema format, all maps contain string-type keys.
Unions
Union types are denoted as union { typeA, typeB, typeC, … }. For example, this record contains a string field that is optional (unioned with null):
record RecordWithUnion {
union { null, string } optionalString;
}
Note that the same restrictions apply to Avro IDL unions as apply to unions defined in the JSON format; namely, a record may not contain multiple elements of the same type.
Defining RPC Messages
The syntax to define an RPC message within a Avro IDL protocol is similar to the syntax for a method declaration within a C header file or a Java interface. To define an RPC message add which takes two arguments named foo and bar, returning an int, simply include the following definition within the protocol:
int add(int foo, int bar = 0);
Message arguments, like record fields, may specify default values.
To define a message with no response, you may use the alias void, equivalent to the Avro null type:
void logMessage(string message);
If you have previously defined an error type within the same protocol, you may declare that a message can throw this error using the syntax:
void goKaboom() throws Kaboom;
To define a one-way message, use the keyword oneway after the parameter list, for example:
void fireAndForget(string message) oneway;
Other Language Features
All Java-style comments are supported within a Avro IDL file. Any text following // on a line is ignored, as is any text between /* and */, possibly spanning multiple lines.
Comments that begin with /** are used as the documentation string for the type or field definition that follows the comment.
Escaping Identifiers
Occasionally, one will need to use a reserved language keyword as an identifier. In order to do so, backticks (`) may be used to escape the identifier. For example, to define a message with the literal name error, you may write:
This syntax is allowed anywhere an identifier is expected.
Annotations for Ordering and Namespaces
Java-style annotations may be used to add additional properties to types and fields throughout Avro IDL.
For example, to specify the sort order of a field within a record, one may use the @order annotation before the field name as follows:
record MyRecord {
string @order("ascending") myAscendingSortField;
string @order("descending") myDescendingField;
string @order("ignore") myIgnoredField;
}
A field’s type may also be preceded by annotations, e.g.:
record MyRecord {
@java-class("java.util.ArrayList") array<string> myStrings;
}
Similarly, a @namespace annotation may be used to modify the namespace when defining a named schema. For example:
@namespace("org.apache.avro.firstNamespace")
protocol MyProto {
@namespace("org.apache.avro.someOtherNamespace")
record Foo {}
record Bar {}
}
will define a protocol in the firstNamespace namespace. The record Foo will be defined in someOtherNamespace and Bar will be defined in firstNamespace as it inherits its default from its container.
Type and field aliases are specified with the @aliases annotation as follows:
@aliases(["org.old.OldRecord", "org.ancient.AncientRecord"])
record MyRecord {
string @aliases(["oldField", "ancientField"]) myNewField;
}
Some annotations like those listed above are handled specially. All other annotations are added as properties to the protocol, message, schema or field.
Complete Example
The following is a complete example of a Avro IDL file that shows most of the above features:
/**
* An example protocol in Avro IDL
*/
@namespace("org.apache.avro.test")
protocol Simple {
@aliases(["org.foo.KindOf"])
enum Kind {
FOO,
BAR, // the bar enum value
BAZ
}
fixed MD5(16);
record TestRecord {
@order("ignore")
string name;
@order("descending")
Kind kind;
MD5 hash;
union { MD5, null} @aliases(["hash"]) nullableHash;
array<long> arrayOfLongs;
}
error TestError {
string message;
}
string hello(string greeting);
TestRecord echo(TestRecord `record`);
int add(int arg1, int arg2);
bytes echoBytes(bytes data);
void `error`() throws TestError;
void ping() oneway;
}
Additional examples may be found in the Avro source tree under the src/test/idl/input directory.
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
21.3 - SASL profile
Introduction
SASL (RFC 2222) provides a framework for authentication and security of network protocols. Each protocol that uses SASL is meant to define a SASL profile. This document provides a SASL profile for connection-based Avro RPC.
Overview
SASL negotiation proceeds as a series of message interactions over a connection between a client and server using a selected SASL mechanism. The client starts this negotiation by sending its chosen mechanism name with an initial (possibly empty) message. Negotiation proceeds with the exchange of messages until either side indicates success or failure. The content of the messages is mechanism-specific. If the negotiation succeeds, then the session can proceed over the connection, otherwise it must be abandoned.
Some mechanisms continue to process session data after negotiation (e.g., encrypting it), while some specify that further session data is transmitted unmodified.
Negotiation
Commands
Avro SASL negotiation uses four one-byte commands.
- 0: START Used in a client’s initial message.
- 1: CONTINUE Used while negotiation is ongoing.
- 2: FAIL Terminates negotiation unsuccessfully.
- 3: COMPLETE Terminates negotiation successfully.
The format of a START message is:
| 0 | 4-byte mechanism name length | mechanism name | 4-byte payload length | payload data |
The format of a CONTINUE message is:
| 1 | 4-byte payload length | payload data |
The format of a FAIL message is:
| 2 | 4-byte message length | UTF-8 message |
The format of a COMPLETE message is:
| 3 | 4-byte payload length | payload data |
Process
Negotiation is initiated by a client sending a START command containing the client’s chosen mechanism name and any mechanism-specific payload data.
The server and client then interchange some number (possibly zero) of CONTINUE messages. Each message contains payload data that is processed by the security mechanism to generate the next message.
Once either the client or server send a FAIL message then negotiation has failed. UTF-8-encoded text is included in the failure message. Once either a FAIL message has been sent or received, or any other error occurs in the negotiation, further communication on this connection must cease.
Once either the client or server send a COMPLETE message then negotiation has completed successfully. Session data may now be transmitted over the connection until it is closed by either side.
Session Data
If no SASL QOP (quality of protection) is negotiated, then all subsequent writes to/reads over this connection are written/read unmodified. In particular, messages use Avro framing, and are of the form:
| 4-byte frame length | frame data | ... | 4 zero bytes |
If a SASL QOP is negotiated, then it must be used by the connection for all subsequent messages. This is done by wrapping each non-empty frame written using the security mechanism and unwrapping each non-empty frame read. The length written in each non-empty frame is the length of the wrapped data. Complete frames must be passed to the security mechanism for unwrapping. Unwrapped data is then passed to the application as the content of the frame.
If at any point processing fails due to wrapping, unwrapping or framing errors, then all further communication on this connection must cease.
Anonymous Mechanism
The SASL anonymous mechanism (RFC 2245) is quite simple to implement. In particular, an initial anonymous request may be prefixed by the following static sequence:
| 0 | 0009 | ANONYMOUS | 0000 |
If a server uses the anonymous mechanism, it should check that the mechanism name in the start message prefixing the first request received is ‘ANONYMOUS’, then simply prefix its initial response with a COMPLETE message of:
| 3 | 0000 |
If an anonymous server recieves some other mechanism name, then it may respond with a FAIL message as simple as:
| 2 | 0000 |
Note that the anonymous mechanism need add no additional round-trip messages between client and server. The START message can be piggybacked on the initial request and the COMPLETE or FAIL message can be piggybacked on the initial response.
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
21.4 - Wiki
The Wiki page can be found here.
21.5 - FAQ
What is Avro?
Avro is a data serialization system.
Avro provides:
- Rich data structures.
- A compact, fast, binary data format.
- A container file, to store persistent data.
- Remote procedure call (RPC).
- Simple integration with dynamic languages. Code generation is not required to read or write data files nor to use or implement RPC protocols. Code generation as an optional optimization, only worth implementing for statically typed languages.
How can I get started quickly with Avro?
Check out the Quick Start Guide.
How do I statically compile a schema or protocol into generated code?
In Java
- Add the avro jar, the jackson-mapper-asl.jar and jackson-core-asl.jar to your CLASSPATH.
- Run java org.apache.avro.specific.SpecificCompiler .
This appears to be out of date, the SpecificCompiler requires two arguments, presumably an input and and output file, but it isn’t clear that this does.
Or use the Schema or Protocol Ant tasks. Avro’s build.xml provides examples of how these are used.
Lastly, you can also use the “avro-tools” jar which ships with an Avro release. Just use the “compile (schema|protocol)” command.
How are Strings represented in Java?
They use org.apache.avro.util.Utf8, not java.lang.String.
More generally, how do Avro types map to Java types?
The mappings are documented in the package javadoc for generic, specific and reflect API.
From Doug Cutting:
HDFS splits files into blocks, and mapreduce runs a map task for each block. When the task starts, it needs to be able to seek into the file to the start of the block process through the block’s end. If the file were, e.g., a gzip file, this would not be possible, since gzip files must be decompressed from the start. One cannot seek into the middle of a gzip file and start decompressing. So Hadoop’s SequenceFile places a marker periodically (~64k) in the file at record and compression boundaries, where processing can be sensibly started. Then, when a map task starts processing an HDFS block, it finds the first marker after the block’s start and continues through the first marker in the next block of the file. This requires a bit of non-local access (~0.1%). Avro’s data file uses the same method as SequenceFile.
Why isn’t every value in Avro nullable?
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 of every value stored and
corresponding computational costs to create this bit on write and
interpret it on read. These costs are wasted when values may not in
fact be null, as is the case in many datasets. In Avro such costs are
only paid when values may actually be null.
Also, allowing values to be null is a well-known source of errors. In
Avro, a value declared as non-null will always be non-null and programs
need not test for null values when processing it nor will they ever fail
for lack of such tests.
Tony Hoare calls his invention of null references his “Billion Dollar
Mistake”.
http://qconlondon.com/london-2009/presentation/Null+References:+The+Billion+Dollar+Mistake
Also note that in some programming languages not all values are permitted to be null. For example, in Java, values of type boolean, byte, short, char, int, float, long, and double may not be null.
How can I serialize directly to/from a byte array?
As pointed out in the specification, Avro data should always be stored with its schema. The Avro provided classes DataFileWriter, DataFileReader, and DataFileStream all ensure this by serializing the Schema in a container header. In some special cases, such as when implementing a new storage system or writing unit tests, you may need to write and read directly with the bare Avro serialized values. The following examples use code generated for Java from an Avro Schema, specifically the User example from the Quickstart guide.
Serializing to a byte array
This example takes a User object and returns a newly allocated byte array with the Avro serialization of that user.
ByteArrayOutputStream out = new ByteArrayOutputStream();
BinaryEncoder encoder = EncoderFactory.get().binaryEncoder(out, null);
DatumWriter<User> writer = new SpecificDatumWriter<User>(User.getClassSchema());
writer.write(user, encoder);
encoder.flush();
out.close();
byte[] serializedBytes = out.toByteArray();
Deserializing from a byte array
This example takes a byte array containing the Avro serialization of a user and returns a User object.SpecificDatumReader reader = new SpecificDatumReader(User.getClassSchema());
Decoder decoder = DecoderFactory.get().binaryDecoder(bytes, null);
User user = reader.read(null, decoder);
Note: If you are serializing or deserializing in a loop or as a method, you should be reusing objects, readers and/or writers. Check the JavaDoc to see how to reuse the objects.
22 - Apache Avro™ 1.6.1 Documentation
Introduction
Apache Avro™ is a data serialization system.
Avro provides:
- Rich data structures.
- A compact, fast, binary data format.
- A container file, to store persistent data.
- Remote procedure call (RPC).
- Simple integration with dynamic languages. Code generation is not required to read or write data files nor to use or implement RPC protocols. Code generation as an optional optimization, only worth implementing for statically typed languages.
Schemas
Avro relies on schemas. When Avro data is read, the schema used when writing it is always present. This permits each datum to be written with no per-value overheads, making serialization both fast and small. This also facilitates use with dynamic, scripting languages, since data, together with its schema, is fully self-describing.
When Avro data is stored in a file, its schema is stored with it, so that files may be processed later by any program. If the program reading the data expects a different schema this can be easily resolved, since both schemas are present.
When Avro is used in RPC, the client and server exchange schemas in the connection handshake. (This can be optimized so that, for most calls, no schemas are actually transmitted.) Since both client and server both have the other’s full schema, correspondence between same named fields, missing fields, extra fields, etc. can all be easily resolved.
Avro schemas are defined with JSON . This facilitates implementation in languages that already have JSON libraries.
Comparison with other systems
Avro provides functionality similar to systems such as Thrift, Protocol Buffers, etc. Avro differs from these systems in the following fundamental aspects.
- Dynamic typing: Avro does not require that code be generated. Data is always accompanied by a schema that permits full processing of that data without code generation, static datatypes, etc. This facilitates construction of generic data-processing systems and languages.
- Untagged data: Since the schema is present when data is read, considerably less type information need be encoded with data, resulting in smaller serialization size.
- No manually-assigned field IDs: When a schema changes, both the old and new schema are always present when processing data, so differences may be resolved symbolically, using field names.
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
22.1 - Specification
Introduction
This document defines Apache Avro. It is intended to be the authoritative specification. Implementations of Avro must adhere to this document.
Schema Declaration
A Schema is represented in JSON by one of:
- A JSON string, naming a defined type.
- A JSON object, of the form:
{"type": "typeName" ...attributes...}
where typeName is either a primitive or derived type name, as defined below. Attributes not defined in this document are permitted as metadata, but must not affect the format of serialized data.
- A JSON array, representing a union of embedded types.
Primitive Types
The set of primitive type names is:
- null: no value
- boolean: a binary value
- int: 32-bit signed integer
- long: 64-bit signed integer
- float: single precision (32-bit) IEEE 754 floating-point number
- double: double precision (64-bit) IEEE 754 floating-point number
- bytes: sequence of 8-bit unsigned bytes
- string: unicode character sequence
Primitive types have no specified attributes.
Primitive type names are also defined type names. Thus, for example, the schema “string” is equivalent to:
Complex Types
Avro supports six kinds of complex types: records, enums, arrays, maps, unions and fixed.
Records
Records use the type name “record” and support three attributes:
- name: a JSON string providing the name of the record (required).
- namespace, a JSON string that qualifies the name;
- doc: a JSON string providing documentation to the user of this schema (optional).
- aliases: a JSON array of strings, providing alternate names for this record (optional).
- fields: a JSON array, listing fields (required). Each field is a JSON object with the following attributes:
- name: a JSON string providing the name of the field (required), and
- doc: a JSON string describing this field for users (optional).
- type: A JSON object defining a schema, or a JSON string naming a record definition (required).
- default: A default value for this field, used when reading instances that lack this field (optional). 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. Default values for bytes and fixed fields are JSON strings, where Unicode code points 0-255 are mapped to unsigned 8-bit byte values 0-255.
field default values
| avro type |
json type |
example |
| null |
null |
null |
| boolean |
boolean |
true |
| int,long |
integer |
1 |
| float,double |
number |
1.1 |
| bytes |
string |
"\u00FF" |
| string |
string |
"foo" |
| record |
object |
{"a": 1} |
| enum |
string |
"FOO" |
| array |
array |
[1] |
| map |
object |
{"a": 1} |
| fixed |
string |
"\u00ff" |
- order: specifies how this field impacts sort ordering of this record (optional). Valid values are “ascending” (the default), “descending”, or “ignore”. For more details on how this is used, see the sort order section below.
- aliases: a JSON array of strings, providing alternate names for this field (optional).
For example, a linked-list of 64-bit values may be defined with:
{
"type": "record",
"name": "LongList",
"aliases": ["LinkedLongs"], // old name for this
"fields" : [
{"name": "value", "type": "long"}, // each element has a long
{"name": "next", "type": ["LongList", "null"]} // optional next element
]
}
Enums
Enums use the type name “enum” and support the following attributes:
- name: a JSON string providing the name of the enum (required).
- namespace, a JSON string that qualifies the name ;
- aliases: a JSON array of strings, providing alternate names for this enum (optional).
- doc: a JSON string providing documentation to the user of this schema (optional).
- symbols: a JSON array, listing symbols, as JSON strings (required). All symbols in an enum must be unique; duplicates are prohibited.
For example, playing card suits might be defined with:
{
"type": "enum",
"name": "Suit",
"symbols" : ["SPADES", "HEARTS", "DIAMONDS", "CLUBS"]
}
Arrays
Arrays use the type name “array” and support a single attribute:
- items: the schema of the array’s items.
For example, an array of strings is declared with:
{
"type": "array",
"items" : "string"
}
Maps
Maps use the type name “map” and support one attribute:
- values: the schema of the map’s values.
Map keys are assumed to be strings.
For example, a map from string to long is declared with:
{
"type": "map",
"values" : "long"
}
Unions
Unions, as mentioned above, are represented using JSON arrays. For example, ["string", "null"] declares a schema which may be either a string or null.
Unions may not contain more than one schema with the same type, except for the named types record, fixed and enum. For example, unions containing two array types or two map types are not permitted, but two types with different names are permitted. (Names permit efficient resolution when reading and writing unions.)
Unions may not immediately contain other unions.
Fixed
Fixed uses the type name “fixed” and supports two attributes:
- name: a string naming this fixed (required).
- namespace, a string that qualifies the name;
- aliases: a JSON array of strings, providing alternate names for this enum (optional).
- size: an integer, specifying the number of bytes per value (required).
For example, 16-byte quantity may be declared with:
{"type": "fixed", "size": 16, "name": "md5"}
Names
Record, enums and fixed are named types. Each has a fullname that is composed of two parts; a name and a namespace. Equality of names is defined on the fullname.
The name portion of a fullname, record field names, and enum symbols must:
- start with [A-Za-z_]
- subsequently contain only [A-Za-z0-9_]
A namespace is a dot-separated sequence of such names.
In record, enum and fixed definitions, the fullname is determined in one of the following ways:
A name and namespace are both specified. For example, one might use “name”: “X”, “namespace”: “org.foo” to indicate the fullname org.foo.X.
-
A fullname is specified. If the name specified contains a dot, then it is assumed to be a fullname, and any namespace also specified is ignored. For example, use “name”: “org.foo.X” to indicate the fullname org.foo.X.
-
A name only is specified, i.e., a name that contains no dots. In this case the namespace is taken from the most tightly enclosing schema or protocol. For example, if “name”: “X” is specified, and this occurs within a field of the record definition of org.foo.Y, then the fullname is org.foo.X.
References to previously defined names are as in the latter two cases above: if they contain a dot they are a fullname, if they do not contain a dot, the namespace is the namespace of the enclosing definition.
Primitive type names have no namespace and their names may not be defined in any namespace. A schema may only contain multiple definitions of a fullname if the definitions are equivalent.
Aliases
Named types and fields may have aliases. An implementation may optionally use aliases to map a writer’s schema to the reader’s. This facilitates both schema evolution as well as processing disparate datasets.
Aliases function by re-writing the writer’s schema using aliases from the reader’s schema. For example, if the writer’s schema was named “Foo” and the reader’s schema is named “Bar” and has an alias of “Foo”, then the implementation would act as though “Foo” were named “Bar” when reading. Similarly, if data was written as a record with a field named “x” and is read as a record with a field named “y” with alias “x”, then the implementation would act as though “x” were named “y” when reading.
A type alias may be specified either as a fully namespace-qualified, or relative to the namespace of the name it is an alias for. For example, if a type named “a.b” has aliases of “c” and “x.y”, then the fully qualified names of its aliases are “a.c” and “x.y”.
Data Serialization
Avro data is always serialized with its schema. Files that store Avro data should always also include the schema for that data in the same file. Avro-based remote procedure call (RPC) systems must also guarantee that remote recipients of data have a copy of the schema used to write that data.
Because the schema used to write data is always available when the data is read, Avro data itself is not tagged with type information. The schema is required to parse data.
In general, both serialization and deserialization proceed as a depth-first, left-to-right traversal of the schema, serializing primitive types as they are encountered.
Encodings
Avro specifies two serialization encodings: binary and JSON. Most applications will use the binary encoding, as it is smaller and faster. But, for debugging and web-based applications, the JSON encoding may sometimes be appropriate.
Binary Encoding
Primitive Types
Primitive types are encoded in binary as follows:
- null is written as zero bytes.
- a boolean is written as a single byte whose value is either 0 (false) or 1 (true).
- int and long values are written using variable-length zig-zag coding. Some examples:
| value |
hex |
| 0 |
00 |
| -1 |
01 |
| 1 |
02 |
| -2 |
03 |
| 2 |
04 |
| … |
|
| -64 |
7f |
| 64 |
80 01 |
| … |
|
- a float is written as 4 bytes. The float is converted into a 32-bit integer using a method equivalent to Java’s floatToIntBits and then encoded in little-endian format.
- a double is written as 8 bytes. The double is converted into a 64-bit integer using a method equivalent to Java’s doubleToLongBits and then encoded in little-endian format.
- bytes are encoded as a long followed by that many bytes of data.
- a string is encoded as a long followed by that many bytes of UTF-8 encoded character data.
For example, the three-character string “foo” would be encoded as the long value 3 (encoded as hex 06) followed by the UTF-8 encoding of ‘f’, ‘o’, and ‘o’ (the hex bytes 66 6f 6f):
06 66 6f 6f
Complex Types
Complex types are encoded in binary as follows:
Records
A record is encoded by encoding the values of its fields in the order that they are declared. In other words, a record is encoded as just the concatenation of the encodings of its fields. Field values are encoded per their schema.
For example, the record schema
{
"type": "record",
"name": "test",
"fields" : [
{"name": "a", "type": "long"},
{"name": "b", "type": "string"}
]
}
An instance of this record whose a field has value 27 (encoded as hex 36) and whose b field has value “foo” (encoded as hex bytes 06 66 6f 6f), would be encoded simply as the concatenation of these, namely the hex byte sequence:
36 06 66 6f 6f
Enums
An enum is encoded by a int, representing the zero-based position of the symbol in the schema.
For example, consider the enum:
{"type": "enum", "name": "Foo", "symbols": ["A", "B", "C", "D"] }
This would be encoded by an int between zero and three, with zero indicating “A”, and 3 indicating “D”.
Arrays
Arrays are encoded as a series of blocks. Each block consists of a long count value, followed by that many array items. A block with count zero indicates the end of the array. Each item is encoded per the array’s item schema.
If a block’s count is negative, its absolute value is used, and the count is followed immediately by a long block size indicating the number of bytes in the block. This block size permits fast skipping through data, e.g., when projecting a record to a subset of its fields.
For example, the array schema
{"type": "array", "items": "long"}
an array containing the items 3 and 27 could be encoded as the long value 2 (encoded as hex 04) followed by long values 3 and 27 (encoded as hex 06 36) terminated by zero:
04 06 36 00
The blocked representation permits one to read and write arrays larger than can be buffered in memory, since one can start writing items without knowing the full length of the array.
Maps
Maps are encoded as a series of blocks. Each block consists of a long count value, followed by that many key/value pairs. A block with count zero indicates the end of the map. Each item is encoded per the map’s value schema.
If a block’s count is negative, its absolute value is used, and the count is followed immediately by a long block size indicating the number of bytes in the block. This block size permits fast skipping through data, e.g., when projecting a record to a subset of its fields.
The blocked representation permits one to read and write maps larger than can be buffered in memory, since one can start writing items without knowing the full length of the map.
Unions
A union is encoded by first writing a long value indicating the zero-based position within the union of the schema of its value. The value is then encoded per the indicated schema within the union.
For example, the union schema ["string","null"] would encode:
-
null as the integer 1 (the index of “null” in the union, encoded as hex 02):
02
-
the string “a” as zero (the index of “string” in the union), followed by the serialized string:
02 02 61
Fixed
Fixed instances are encoded using the number of bytes declared in the schema.
JSON Encoding
Except for unions, the JSON encoding is the same as is used to encode field default values.
The value of a union is encoded in JSON as follows:
- if its type is null, then it is encoded as a JSON null;
- otherwise it is encoded as a JSON object with one name/value pair whose name is the type’s name and whose value is the recursively encoded value. For Avro’s named types (record, fixed or enum) the user-specified name is used, for other types the type name is used.
For example, the union schema ["null","string","Foo"], where Foo is a record name, would encode:
- null as null;
- the string “a” as
{"string": "a"} and
- a Foo instance as
{"Foo": {...}}, where {...} indicates the JSON encoding of a Foo instance.
Note that a schema is still required to correctly process JSON-encoded data. For example, the JSON encoding does not distinguish between int and long, float and double, records and maps, enums and strings, etc.
Sort Order
Avro defines a standard sort order for data. This permits data written by one system to be efficiently sorted by another system. This can be an important optimization, as sort order comparisons are sometimes the most frequent per-object operation. Note also that Avro binary-encoded data can be efficiently ordered without deserializing it to objects.
Data items may only be compared if they have identical schemas. Pairwise comparisons are implemented recursively with a depth-first, left-to-right traversal of the schema. The first mismatch encountered determines the order of the items.
Two items with the same schema are compared according to the following rules.
- null data is always equal.
- boolean data is ordered with false before true.
- int, long, float and double data is ordered by ascending numeric value.
- bytes and fixed data are compared lexicographically by unsigned 8-bit values.
- string data is compared lexicographically by Unicode code point. Note that since UTF-8 is used as the binary encoding for strings, sorting of bytes and string binary data is identical.
- array data is compared lexicographically by element.
- enum data is ordered by the symbol’s position in the enum schema. For example, an enum whose symbols are
["z", "a"] would sort “z” values before “a” values.
- union data is first ordered by the branch within the union, and, within that, by the type of the branch. For example, an
["int", "string"] union would order all int values before all string values, with the ints and strings themselves ordered as defined above.
- record data is ordered lexicographically by field. If a field specifies that its order is:
- “ascending”, then the order of its values is unaltered.
- “descending”, then the order of its values is reversed.
- “ignore”, then its values are ignored when sorting.
- map data may not be compared. It is an error to attempt to compare data containing maps unless those maps are in an
"order":"ignore" record field.
Object Container Files
Avro includes a simple object container file format. A file has a schema, and all objects stored in the file must be written according to that schema, using binary encoding. Objects are stored in blocks that may be compressed. Syncronization markers are used between blocks to permit efficient splitting of files for MapReduce processing.
Files may include arbitrary user-specified metadata.
A file consists of:
- A file header, followed by
- one or more file data blocks.
A file header consists of:
- Four bytes, ASCII ‘O’, ‘b’, ‘j’, followed by 1.
- file metadata, including the schema.
- The 16-byte, randomly-generated sync marker for this file.
File metadata consists of:
A long indicating the number of metadata key/value pairs.
For each pair, a string key and bytes value.
All metadata properties that start with “avro.” are reserved. The following file metadata properties are currently used:
- avro.schema contains the schema of objects stored in the file, as JSON data (required).
- avro.codec the name of the compression codec used to compress blocks, as a string. Implementations are required to support the following codecs: “null” and “deflate”. If codec is absent, it is assumed to be “null”. The codecs are described with more detail below.
A file header is thus described by the following schema:
{"type": "record", "name": "org.apache.avro.file.Header",
"fields" : [
{"name": "magic", "type": {"type": "fixed", "name": "Magic", "size": 4}},
{"name": "meta", "type": {"type": "map", "values": "bytes"}},
{"name": "sync", "type": {"type": "fixed", "name": "Sync", "size": 16}}
]
}
A file data block consists of:
- A long indicating the count of objects in this block.
- A long indicating the size in bytes of the serialized objects in the current block, after any codec is applied
- The serialized objects. If a codec is specified, this is compressed by that codec.
- The file’s 16-byte sync marker.
Thus, each block’s binary data can be efficiently extracted or skipped without deserializing the contents. The combination of block size, object counts, and sync markers enable detection of corrupt blocks and help ensure data integrity.
Required Codecs
null
The “null” codec simply passes through data uncompressed.
deflate
The “deflate” codec writes the data block using the deflate algorithm as specified in RFC 1951, and typically implemented using the zlib library. Note that this format (unlike the “zlib format” in RFC 1950) does not have a checksum.
Optional Codecs
snappy
The “snappy” codec uses Google’s Snappy compression library. Each compressed block is followed by the 4-byte, big-endian CRC32 checksum of the uncompressed data in the block.
Protocol Declaration
Avro protocols describe RPC interfaces. Like schemas, they are defined with JSON text.
A protocol is a JSON object with the following attributes:
- protocol, a string, the name of the protocol (required);
- namespace, an optional string that qualifies the name;
- doc, an optional string describing this protocol;
- types, an optional list of definitions of named types (records, enums, fixed and errors). An error definition is just like a record definition except it uses “error” instead of “record”. Note that forward references to named types are not permitted.
- messages, an optional JSON object whose keys are message names and whose values are objects whose attributes are described below. No two messages may have the same name.
The name and namespace qualification rules defined for schema objects apply to protocols as well.
Messages
A message has attributes:
- a doc, an optional description of the message,
- a request, a list of named, typed parameter schemas (this has the same form as the fields of a record declaration);
- a response schema;
- an optional union of declared error schemas. The effective union has “string” prepended to the declared union, to permit transmission of undeclared “system” errors. For example, if the declared error union is
["AccessError"], then the effective union is ["string", "AccessError"]. When no errors are declared, the effective error union is ["string"]. Errors are serialized using the effective union; however, a protocol’s JSON declaration contains only the declared union.
- an optional one-way boolean parameter.
A request parameter list is processed equivalently to an anonymous record. Since record field lists may vary between reader and writer, request parameters may also differ between the caller and responder, and such differences are resolved in the same manner as record field differences.
The one-way parameter may only be true when the response type is "null" and no errors are listed.
Sample Protocol
For example, one may define a simple HelloWorld protocol with:
{
"namespace": "com.acme",
"protocol": "HelloWorld",
"doc": "Protocol Greetings",
"types": [
{"name": "Greeting", "type": "record", "fields": [
{"name": "message", "type": "string"}]},
{"name": "Curse", "type": "error", "fields": [
{"name": "message", "type": "string"}]}
],
"messages": {
"hello": {
"doc": "Say hello.",
"request": [{"name": "greeting", "type": "Greeting" }],
"response": "Greeting",
"errors": ["Curse"]
}
}
}
Message Transport
Messages may be transmitted via different transport mechanisms.
To the transport, a message is an opaque byte sequence.
A transport is a system that supports:
- transmission of request messages
- receipt of corresponding response messages
Servers may send a response message back to the client corresponding to a request message. The mechanism of correspondance is transport-specific. For example, in HTTP it is implicit, since HTTP directly supports requests and responses. But a transport that multiplexes many client threads over a single socket would need to tag messages with unique identifiers.
Transports may be either stateless or stateful. In a stateless transport, messaging assumes no established connection state, while stateful transports establish connections that may be used for multiple messages. This distinction is discussed further in the handshake section below.
HTTP as Transport
When HTTP is used as a transport, each Avro message exchange is an HTTP request/response pair. All messages of an Avro protocol should share a single URL at an HTTP server. Other protocols may also use that URL. Both normal and error Avro response messages should use the 200 (OK) response code. The chunked encoding may be used for requests and responses, but, regardless the Avro request and response are the entire content of an HTTP request and response. The HTTP Content-Type of requests and responses should be specified as “avro/binary”. Requests should be made using the POST method.
HTTP is used by Avro as a stateless transport.
Message Framing
Avro messages are framed as a list of buffers.
Framing is a layer between messages and the transport. It exists to optimize certain operations.
The format of framed message data is:
- a series of buffers, where each buffer consists of:
- a four-byte, big-endian buffer length, followed by
- that many bytes of buffer data.
- A message is always terminated by a zero-lenghted buffer.
Framing is transparent to request and response message formats (described below). Any message may be presented as a single or multiple buffers.
Framing can permit readers to more efficiently get different buffers from different sources and for writers to more efficiently store different buffers to different destinations. In particular, it can reduce the number of times large binary objects are copied. For example, if an RPC parameter consists of a megabyte of file data, that data can be copied directly to a socket from a file descriptor, and, on the other end, it could be written directly to a file descriptor, never entering user space.
A simple, recommended, framing policy is for writers to create a new segment whenever a single binary object is written that is larger than a normal output buffer. Small objects are then appended in buffers, while larger objects are written as their own buffers. When a reader then tries to read a large object the runtime can hand it an entire buffer directly, without having to copy it.
Handshake
The purpose of the handshake is to ensure that the client and the server have each other’s protocol definition, so that the client can correctly deserialize responses, and the server can correctly deserialize requests. Both clients and servers should maintain a cache of recently seen protocols, so that, in most cases, a handshake will be completed without extra round-trip network exchanges or the transmission of full protocol text.
RPC requests and responses may not be processed until a handshake has been completed. With a stateless transport, all requests and responses are prefixed by handshakes. With a stateful transport, handshakes are only attached to requests and responses until a successful handshake response has been returned over a connection. After this, request and response payloads are sent without handshakes for the lifetime of that connection.
The handshake process uses the following record schemas:
{
"type": "record",
"name": "HandshakeRequest", "namespace":"org.apache.avro.ipc",
"fields": [
{"name": "clientHash",
"type": {"type": "fixed", "name": "MD5", "size": 16}},
{"name": "clientProtocol", "type": ["null", "string"]},
{"name": "serverHash", "type": "MD5"},
{"name": "meta", "type": ["null", {"type": "map", "values": "bytes"}]}
]
}
{
"type": "record",
"name": "HandshakeResponse", "namespace": "org.apache.avro.ipc",
"fields": [
{"name": "match",
"type": {"type": "enum", "name": "HandshakeMatch",
"symbols": ["BOTH", "CLIENT", "NONE"]}},
{"name": "serverProtocol",
"type": ["null", "string"]},
{"name": "serverHash",
"type": ["null", {"type": "fixed", "name": "MD5", "size": 16}]},
{"name": "meta",
"type": ["null", {"type": "map", "values": "bytes"}]}
]
}
- A client first prefixes each request with a
HandshakeRequest containing just the hash of its protocol and of the server’s protocol (clientHash!=null, clientProtocol=null, serverHash!=null), where the hashes are 128-bit MD5 hashes of the JSON protocol text. If a client has never connected to a given server, it sends its hash as a guess of the server’s hash, otherwise it sends the hash that it previously obtained from this server.
The server responds with a HandshakeResponse containing one of:
-
match=BOTH, serverProtocol=null, serverHash=null if the client sent the valid hash of the server’s protocol and the server knows what protocol corresponds to the client’s hash. In this case, the request is complete and the response data immediately follows the HandshakeResponse.
-
match=CLIENT, serverProtocol!=null, serverHash!=null if the server has previously seen the client’s protocol, but the client sent an incorrect hash of the server’s protocol. The request is complete and the response data immediately follows the HandshakeResponse. The client must use the returned protocol to process the response and should also cache that protocol and its hash for future interactions with this server.
-
match=NONE if the server has not previously seen the client’s protocol. The serverHash and serverProtocol may also be non-null if the server’s protocol hash was incorrect.
In this case the client must then re-submit its request with its protocol text (clientHash!=null, clientProtocol!=null, serverHash!=null) and the server should respond with a successful match (match=BOTH, serverProtocol=null, serverHash=null) as above.
The meta field is reserved for future handshake enhancements.
A call consists of a request message paired with its resulting response or error message. Requests and responses contain extensible metadata, and both kinds of messages are framed as described above.
The format of a call request is:
- request metadata, a map with values of type bytes
- the message name, an Avro string, followed by
- the message parameters. Parameters are serialized according to the message’s request declaration.
When a message is declared one-way and a stateful connection has been established by a successful handshake response, no response data is sent. Otherwise the format of the call response is:
- response metadata, a map with values of type bytes
- a one-byte error flag boolean, followed by either:
- if the error flag is false, the message response, serialized per the message’s response schema.
- if the error flag is true, the error, serialized per the message’s effective error union schema.
Schema Resolution
A reader of Avro data, whether from an RPC or a file, can always parse that data because its schema is provided. But that schema may not be exactly the schema that was expected. For example, if the data was written with a different version of the software than it is read, then records may have had fields added or removed. This section specifies how such schema differences should be resolved.
We refer to the schema used to write the data as the writer’s schema, and the schema that the application expects the reader’s schema. Differences between these should be resolved as follows:
- It is an error if the two schemas do not match.
To match, one of the following must hold:
-
both schemas are arrays whose item types match
-
both schemas are maps whose value types match
-
both schemas are enums whose names match
-
both schemas are fixed whose sizes and names match
-
both schemas are records with the same name
-
either schema is a union
-
both schemas have same primitive type
-
the writer’s schema may be promoted to the reader’s as follows:
- int is promotable to long, float, or double
- long is promotable to float or double
- float is promotable to double
- string is promotable to bytes
- bytes is promotable to string
-
if both are records:
- the ordering of fields may be different: fields are matched by name.
- schemas for fields with the same name in both records are resolved recursively.
- if the writer’s record contains a field with a name not present in the reader’s record, the writer’s value for that field is ignored.
- if the reader’s record schema has a field that contains a default value, and writer’s schema does not have a field with the same name, then the reader should use the default value from its field.
- if the reader’s record schema has a field with no default value, and writer’s schema does not have a field with the same name, an error is signalled.
-
if both are enums:
if the writer’s symbol is not present in the reader’s enum, then an error is signalled.
This resolution algorithm is applied recursively to the reader’s and writer’s array item schemas.
This resolution algorithm is applied recursively to the reader’s and writer’s value schemas.
The first schema in the reader’s union that matches the selected writer’s union schema is recursively resolved against it. if none match, an error is signalled.
- if reader’s is a union, but writer’s is not:
The first schema in the reader’s union that matches the writer’s schema is recursively resolved against it. If none match, an error is signalled.
- if writer’s is a union, but reader’s is not:
If the reader’s schema matches the selected writer’s schema, it is recursively resolved against it. If they do not match, an error is signalled.
A schema’s “doc” fields are ignored for the purposes of schema resolution. Hence, the “doc” portion of a schema may be dropped at serialization.
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
22.2 - IDL Language
Introduction
This document defines Avro IDL, a higher-level language for authoring Avro schemata. Before reading this document, you should have familiarity with the concepts of schemata and protocols, as well as the various primitive and complex types available in Avro.
Overview
Purpose
The aim of the Avro IDL language is to enable developers to author schemata in a way that feels more similar to common programming languages like Java, C++, or Python. Additionally, the Avro IDL language may feel more familiar for those users who have previously used the interface description languages (IDLs) in other frameworks like Thrift, Protocol Buffers, or CORBA.
Usage
Each Avro IDL file defines a single Avro Protocol, and thus generates as its output a JSON-format Avro Protocol file with extension .avpr.
To convert a .avdl file into a .avpr file, it may be processed by the idl tool. For example:
$ java -jar avroj-tools-1.4.0.jar idl src/test/idl/input/namespaces.avdl /tmp/namespaces.avpr
$ head /tmp/namespaces.avpr
{
"protocol" : "TestNamespace",
"namespace" : "avro.test.protocol",
The idl tool can also process input to and from stdin and stdout. See idl --help for full usage information.
Defining a Protocol in Avro IDL
An Avro IDL file consists of exactly one protocol definition. The minimal protocol is defined by the following code:
This is equivalent to (and generates) the following JSON protocol definition:
{
"protocol" : "MyProtocol",
"types" : [ ],
"messages" : {
}
}
The namespace of the protocol may be changed using the @namespace annotation:
@namespace("mynamespace")
protocol MyProtocol {
}
This notation is used throughout Avro IDL as a way of specifying properties for the annotated element, as will be described later in this document.
Protocols in Avro IDL can contain the following items:
- Imports of external protocol and schema files.
- Definitions of named schemata, including records, errors, enums, and fixeds.
- Definitions of RPC messages
Imports
Files may be imported in one of three formats:
-
An IDL file may be imported with a statement like:
import idl "foo.avdl";
-
A JSON protocol file may be imported with a statement like:
import protocol "foo.avpr";
-
A JSON schema file may be imported with a statement like:
import schema "foo.avsc";
Messages and types in the imported file are added to this file’s protocol.
Imported file names are resolved relative to the current IDL file.
Defining an Enumeration
Enums are defined in Avro IDL using a syntax similar to C or Java:
enum Suit {
SPADES, DIAMONDS, CLUBS, HEARTS
}
Note that, unlike the JSON format, anonymous enums cannot be defined.
Defining a Fixed Length Field
Fixed fields are defined using the following syntax:
fixed MD5(16);
This example defines a fixed-length type called MD5 which contains 16 bytes.
Defining Records and Errors
Records are defined in Avro IDL using a syntax similar to a struct definition in C:
record Employee {
string name;
boolean active = true;
long salary;
}
The above example defines a record with the name “Employee” with three fields.
To define an error, simply use the keyword error instead of record. For example:
error Kaboom {
string explanation;
int result_code = -1;
}
Each field in a record or error consists of a type and a name, optional property annotations and an optional default value.
A type reference in Avro IDL must be one of:
- A primitive type
- A logical type
- A named schema defined prior to this usage in the same Protocol
- A complex type (array, map, or union)
Primitive Types
The primitive types supported by Avro IDL are the same as those supported by Avro’s JSON format. This list includes int, long, string, boolean, float, double, null, and bytes.
References to Named Schemata
If a named schema has already been defined in the same Avro IDL file, it may be referenced by name as if it were a primitive type:
record Card {
Suit suit; // refers to the enum Card defined above
int number;
}
Default Values
Default values for fields may be optionally specified by using an equals sign after the field name followed by a JSON expression indicating the default value. This JSON is interpreted as described in the spec.
Complex Types
Arrays
Array types are written in a manner that will seem familiar to C++ or Java programmers. An array of any type t is denoted array<t>. For example, an array of strings is denoted array<string>, and a multidimensional array of Foo records would be array<array<Foo>>.
Maps
Map types are written similarly to array types. An array that contains values of type t is written map<t>. As in the JSON schema format, all maps contain string-type keys.
Unions
Union types are denoted as union { typeA, typeB, typeC, … }. For example, this record contains a string field that is optional (unioned with null):
record RecordWithUnion {
union { null, string } optionalString;
}
Note that the same restrictions apply to Avro IDL unions as apply to unions defined in the JSON format; namely, a record may not contain multiple elements of the same type.
Defining RPC Messages
The syntax to define an RPC message within a Avro IDL protocol is similar to the syntax for a method declaration within a C header file or a Java interface. To define an RPC message add which takes two arguments named foo and bar, returning an int, simply include the following definition within the protocol:
int add(int foo, int bar = 0);
Message arguments, like record fields, may specify default values.
To define a message with no response, you may use the alias void, equivalent to the Avro null type:
void logMessage(string message);
If you have previously defined an error type within the same protocol, you may declare that a message can throw this error using the syntax:
void goKaboom() throws Kaboom;
To define a one-way message, use the keyword oneway after the parameter list, for example:
void fireAndForget(string message) oneway;
Other Language Features
All Java-style comments are supported within a Avro IDL file. Any text following // on a line is ignored, as is any text between /* and */, possibly spanning multiple lines.
Comments that begin with /** are used as the documentation string for the type or field definition that follows the comment.
Escaping Identifiers
Occasionally, one will need to use a reserved language keyword as an identifier. In order to do so, backticks (`) may be used to escape the identifier. For example, to define a message with the literal name error, you may write:
This syntax is allowed anywhere an identifier is expected.
Annotations for Ordering and Namespaces
Java-style annotations may be used to add additional properties to types and fields throughout Avro IDL.
For example, to specify the sort order of a field within a record, one may use the @order annotation before the field name as follows:
record MyRecord {
string @order("ascending") myAscendingSortField;
string @order("descending") myDescendingField;
string @order("ignore") myIgnoredField;
}
A field’s type may also be preceded by annotations, e.g.:
record MyRecord {
@java-class("java.util.ArrayList") array<string> myStrings;
}
Similarly, a @namespace annotation may be used to modify the namespace when defining a named schema. For example:
@namespace("org.apache.avro.firstNamespace")
protocol MyProto {
@namespace("org.apache.avro.someOtherNamespace")
record Foo {}
record Bar {}
}
will define a protocol in the firstNamespace namespace. The record Foo will be defined in someOtherNamespace and Bar will be defined in firstNamespace as it inherits its default from its container.
Type and field aliases are specified with the @aliases annotation as follows:
@aliases(["org.old.OldRecord", "org.ancient.AncientRecord"])
record MyRecord {
string @aliases(["oldField", "ancientField"]) myNewField;
}
Complete Example
The following is a complete example of a Avro IDL file that shows most of the above features:
/**
* An example protocol in Avro IDL
*/
@namespace("org.apache.avro.test")
protocol Simple {
@aliases(["org.foo.KindOf"])
enum Kind {
FOO,
BAR, // the bar enum value
BAZ
}
fixed MD5(16);
record TestRecord {
@order("ignore")
string name;
@order("descending")
Kind kind;
MD5 hash;
union { MD5, null} @aliases(["hash"]) nullableHash;
array<long> arrayOfLongs;
}
error TestError {
string message;
}
string hello(string greeting);
TestRecord echo(TestRecord `record`);
int add(int arg1, int arg2);
bytes echoBytes(bytes data);
void `error`() throws TestError;
void ping() oneway;
}
Additional examples may be found in the Avro source tree under the src/test/idl/input directory.
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
22.3 - SASL profile
Introduction
SASL (RFC 2222) provides a framework for authentication and security of network protocols. Each protocol that uses SASL is meant to define a SASL profile. This document provides a SASL profile for connection-based Avro RPC.
Overview
SASL negotiation proceeds as a series of message interactions over a connection between a client and server using a selected SASL mechanism. The client starts this negotiation by sending its chosen mechanism name with an initial (possibly empty) message. Negotiation proceeds with the exchange of messages until either side indicates success or failure. The content of the messages is mechanism-specific. If the negotiation succeeds, then the session can proceed over the connection, otherwise it must be abandoned.
Some mechanisms continue to process session data after negotiation (e.g., encrypting it), while some specify that further session data is transmitted unmodified.
Negotiation
Commands
Avro SASL negotiation uses four one-byte commands.
- 0: START Used in a client’s initial message.
- 1: CONTINUE Used while negotiation is ongoing.
- 2: FAIL Terminates negotiation unsuccessfully.
- 3: COMPLETE Terminates negotiation successfully.
The format of a START message is:
| 0 | 4-byte mechanism name length | mechanism name | 4-byte payload length | payload data |
The format of a CONTINUE message is:
| 1 | 4-byte payload length | payload data |
The format of a FAIL message is:
| 2 | 4-byte message length | UTF-8 message |
The format of a COMPLETE message is:
| 3 | 4-byte payload length | payload data |
Process
Negotiation is initiated by a client sending a START command containing the client’s chosen mechanism name and any mechanism-specific payload data.
The server and client then interchange some number (possibly zero) of CONTINUE messages. Each message contains payload data that is processed by the security mechanism to generate the next message.
Once either the client or server send a FAIL message then negotiation has failed. UTF-8-encoded text is included in the failure message. Once either a FAIL message has been sent or received, or any other error occurs in the negotiation, further communication on this connection must cease.
Once either the client or server send a COMPLETE message then negotiation has completed successfully. Session data may now be transmitted over the connection until it is closed by either side.
Session Data
If no SASL QOP (quality of protection) is negotiated, then all subsequent writes to/reads over this connection are written/read unmodified. In particular, messages use Avro framing, and are of the form:
| 4-byte frame length | frame data | ... | 4 zero bytes |
If a SASL QOP is negotiated, then it must be used by the connection for all subsequent messages. This is done by wrapping each non-empty frame written using the security mechanism and unwrapping each non-empty frame read. The length written in each non-empty frame is the length of the wrapped data. Complete frames must be passed to the security mechanism for unwrapping. Unwrapped data is then passed to the application as the content of the frame.
If at any point processing fails due to wrapping, unwrapping or framing errors, then all further communication on this connection must cease.
Anonymous Mechanism
The SASL anonymous mechanism (RFC 2245) is quite simple to implement. In particular, an initial anonymous request may be prefixed by the following static sequence:
| 0 | 0009 | ANONYMOUS | 0000 |
If a server uses the anonymous mechanism, it should check that the mechanism name in the start message prefixing the first request received is ‘ANONYMOUS’, then simply prefix its initial response with a COMPLETE message of:
| 3 | 0000 |
If an anonymous server recieves some other mechanism name, then it may respond with a FAIL message as simple as:
| 2 | 0000 |
Note that the anonymous mechanism need add no additional round-trip messages between client and server. The START message can be piggybacked on the initial request and the COMPLETE or FAIL message can be piggybacked on the initial response.
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
22.4 - Wiki
The Wiki page can be found here.
22.5 - FAQ
What is Avro?
Avro is a data serialization system.
Avro provides:
- Rich data structures.
- A compact, fast, binary data format.
- A container file, to store persistent data.
- Remote procedure call (RPC).
- Simple integration with dynamic languages. Code generation is not required to read or write data files nor to use or implement RPC protocols. Code generation as an optional optimization, only worth implementing for statically typed languages.
How can I get started quickly with Avro?
Check out the Quick Start Guide.
How do I statically compile a schema or protocol into generated code?
In Java
- Add the avro jar, the jackson-mapper-asl.jar and jackson-core-asl.jar to your CLASSPATH.
- Run java org.apache.avro.specific.SpecificCompiler .
This appears to be out of date, the SpecificCompiler requires two arguments, presumably an input and and output file, but it isn’t clear that this does.
Or use the Schema or Protocol Ant tasks. Avro’s build.xml provides examples of how these are used.
Lastly, you can also use the “avro-tools” jar which ships with an Avro release. Just use the “compile (schema|protocol)” command.
How are Strings represented in Java?
They use org.apache.avro.util.Utf8, not java.lang.String.
More generally, how do Avro types map to Java types?
The mappings are documented in the package javadoc for generic, specific and reflect API.
From Doug Cutting:
HDFS splits files into blocks, and mapreduce runs a map task for each block. When the task starts, it needs to be able to seek into the file to the start of the block process through the block’s end. If the file were, e.g., a gzip file, this would not be possible, since gzip files must be decompressed from the start. One cannot seek into the middle of a gzip file and start decompressing. So Hadoop’s SequenceFile places a marker periodically (~64k) in the file at record and compression boundaries, where processing can be sensibly started. Then, when a map task starts processing an HDFS block, it finds the first marker after the block’s start and continues through the first marker in the next block of the file. This requires a bit of non-local access (~0.1%). Avro’s data file uses the same method as SequenceFile.
Why isn’t every value in Avro nullable?
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 of every value stored and
corresponding computational costs to create this bit on write and
interpret it on read. These costs are wasted when values may not in
fact be null, as is the case in many datasets. In Avro such costs are
only paid when values may actually be null.
Also, allowing values to be null is a well-known source of errors. In
Avro, a value declared as non-null will always be non-null and programs
need not test for null values when processing it nor will they ever fail
for lack of such tests.
Tony Hoare calls his invention of null references his “Billion Dollar
Mistake”.
http://qconlondon.com/london-2009/presentation/Null+References:+The+Billion+Dollar+Mistake
Also note that in some programming languages not all values are permitted to be null. For example, in Java, values of type boolean, byte, short, char, int, float, long, and double may not be null.
How can I serialize directly to/from a byte array?
As pointed out in the specification, Avro data should always be stored with its schema. The Avro provided classes DataFileWriter, DataFileReader, and DataFileStream all ensure this by serializing the Schema in a container header. In some special cases, such as when implementing a new storage system or writing unit tests, you may need to write and read directly with the bare Avro serialized values. The following examples use code generated for Java from an Avro Schema, specifically the User example from the Quickstart guide.
Serializing to a byte array
This example takes a User object and returns a newly allocated byte array with the Avro serialization of that user.
ByteArrayOutputStream out = new ByteArrayOutputStream();
BinaryEncoder encoder = EncoderFactory.get().binaryEncoder(out, null);
DatumWriter<User> writer = new SpecificDatumWriter<User>(User.getClassSchema());
writer.write(user, encoder);
encoder.flush();
out.close();
byte[] serializedBytes = out.toByteArray();
Deserializing from a byte array
This example takes a byte array containing the Avro serialization of a user and returns a User object.SpecificDatumReader reader = new SpecificDatumReader(User.getClassSchema());
Decoder decoder = DecoderFactory.get().binaryDecoder(bytes, null);
User user = reader.read(null, decoder);
Note: If you are serializing or deserializing in a loop or as a method, you should be reusing objects, readers and/or writers. Check the JavaDoc to see how to reuse the objects.
23 - Apache Avro™ 1.6.0 Documentation
Introduction
Apache Avro™ is a data serialization system.
Avro provides:
- Rich data structures.
- A compact, fast, binary data format.
- A container file, to store persistent data.
- Remote procedure call (RPC).
- Simple integration with dynamic languages. Code generation is not required to read or write data files nor to use or implement RPC protocols. Code generation as an optional optimization, only worth implementing for statically typed languages.
Schemas
Avro relies on schemas. When Avro data is read, the schema used when writing it is always present. This permits each datum to be written with no per-value overheads, making serialization both fast and small. This also facilitates use with dynamic, scripting languages, since data, together with its schema, is fully self-describing.
When Avro data is stored in a file, its schema is stored with it, so that files may be processed later by any program. If the program reading the data expects a different schema this can be easily resolved, since both schemas are present.
When Avro is used in RPC, the client and server exchange schemas in the connection handshake. (This can be optimized so that, for most calls, no schemas are actually transmitted.) Since both client and server both have the other’s full schema, correspondence between same named fields, missing fields, extra fields, etc. can all be easily resolved.
Avro schemas are defined with JSON . This facilitates implementation in languages that already have JSON libraries.
Comparison with other systems
Avro provides functionality similar to systems such as Thrift, Protocol Buffers, etc. Avro differs from these systems in the following fundamental aspects.
- Dynamic typing: Avro does not require that code be generated. Data is always accompanied by a schema that permits full processing of that data without code generation, static datatypes, etc. This facilitates construction of generic data-processing systems and languages.
- Untagged data: Since the schema is present when data is read, considerably less type information need be encoded with data, resulting in smaller serialization size.
- No manually-assigned field IDs: When a schema changes, both the old and new schema are always present when processing data, so differences may be resolved symbolically, using field names.
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
23.1 - Specification
Introduction
This document defines Apache Avro. It is intended to be the authoritative specification. Implementations of Avro must adhere to this document.
Schema Declaration
A Schema is represented in JSON by one of:
- A JSON string, naming a defined type.
- A JSON object, of the form:
{"type": "typeName" ...attributes...}
where typeName is either a primitive or derived type name, as defined below. Attributes not defined in this document are permitted as metadata, but must not affect the format of serialized data.
- A JSON array, representing a union of embedded types.
Primitive Types
The set of primitive type names is:
- null: no value
- boolean: a binary value
- int: 32-bit signed integer
- long: 64-bit signed integer
- float: single precision (32-bit) IEEE 754 floating-point number
- double: double precision (64-bit) IEEE 754 floating-point number
- bytes: sequence of 8-bit unsigned bytes
- string: unicode character sequence
Primitive types have no specified attributes.
Primitive type names are also defined type names. Thus, for example, the schema “string” is equivalent to:
Complex Types
Avro supports six kinds of complex types: records, enums, arrays, maps, unions and fixed.
Records
Records use the type name “record” and support three attributes:
- name: a JSON string providing the name of the record (required).
- namespace, a JSON string that qualifies the name;
- doc: a JSON string providing documentation to the user of this schema (optional).
- aliases: a JSON array of strings, providing alternate names for this record (optional).
- fields: a JSON array, listing fields (required). Each field is a JSON object with the following attributes:
- name: a JSON string providing the name of the field (required), and
- doc: a JSON string describing this field for users (optional).
- type: A JSON object defining a schema, or a JSON string naming a record definition (required).
- default: A default value for this field, used when reading instances that lack this field (optional). 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. Default values for bytes and fixed fields are JSON strings, where Unicode code points 0-255 are mapped to unsigned 8-bit byte values 0-255.
field default values
| avro type |
json type |
example |
| null |
null |
null |
| boolean |
boolean |
true |
| int,long |
integer |
1 |
| float,double |
number |
1.1 |
| bytes |
string |
"\u00FF" |
| string |
string |
"foo" |
| record |
object |
{"a": 1} |
| enum |
string |
"FOO" |
| array |
array |
[1] |
| map |
object |
{"a": 1} |
| fixed |
string |
"\u00ff" |
- order: specifies how this field impacts sort ordering of this record (optional). Valid values are “ascending” (the default), “descending”, or “ignore”. For more details on how this is used, see the sort order section below.
- aliases: a JSON array of strings, providing alternate names for this field (optional).
For example, a linked-list of 64-bit values may be defined with:
{
"type": "record",
"name": "LongList",
"aliases": ["LinkedLongs"], // old name for this
"fields" : [
{"name": "value", "type": "long"}, // each element has a long
{"name": "next", "type": ["LongList", "null"]} // optional next element
]
}
Enums
Enums use the type name “enum” and support the following attributes:
- name: a JSON string providing the name of the enum (required).
- namespace, a JSON string that qualifies the name ;
- aliases: a JSON array of strings, providing alternate names for this enum (optional).
- doc: a JSON string providing documentation to the user of this schema (optional).
- symbols: a JSON array, listing symbols, as JSON strings (required). All symbols in an enum must be unique; duplicates are prohibited.
For example, playing card suits might be defined with:
{
"type": "enum",
"name": "Suit",
"symbols" : ["SPADES", "HEARTS", "DIAMONDS", "CLUBS"]
}
Arrays
Arrays use the type name “array” and support a single attribute:
- items: the schema of the array’s items.
For example, an array of strings is declared with:
{
"type": "array",
"items" : "string"
}
Maps
Maps use the type name “map” and support one attribute:
- values: the schema of the map’s values.
Map keys are assumed to be strings.
For example, a map from string to long is declared with:
{
"type": "map",
"values" : "long"
}
Unions
Unions, as mentioned above, are represented using JSON arrays. For example, ["string", "null"] declares a schema which may be either a string or null.
Unions may not contain more than one schema with the same type, except for the named types record, fixed and enum. For example, unions containing two array types or two map types are not permitted, but two types with different names are permitted. (Names permit efficient resolution when reading and writing unions.)
Unions may not immediately contain other unions.
Fixed
Fixed uses the type name “fixed” and supports two attributes:
- name: a string naming this fixed (required).
- namespace, a string that qualifies the name;
- aliases: a JSON array of strings, providing alternate names for this enum (optional).
- size: an integer, specifying the number of bytes per value (required).
For example, 16-byte quantity may be declared with:
{"type": "fixed", "size": 16, "name": "md5"}
Names
Record, enums and fixed are named types. Each has a fullname that is composed of two parts; a name and a namespace. Equality of names is defined on the fullname.
The name portion of a fullname, record field names, and enum symbols must:
- start with [A-Za-z_]
- subsequently contain only [A-Za-z0-9_]
A namespace is a dot-separated sequence of such names.
In record, enum and fixed definitions, the fullname is determined in one of the following ways:
A name and namespace are both specified. For example, one might use “name”: “X”, “namespace”: “org.foo” to indicate the fullname org.foo.X.
-
A fullname is specified. If the name specified contains a dot, then it is assumed to be a fullname, and any namespace also specified is ignored. For example, use “name”: “org.foo.X” to indicate the fullname org.foo.X.
-
A name only is specified, i.e., a name that contains no dots. In this case the namespace is taken from the most tightly enclosing schema or protocol. For example, if “name”: “X” is specified, and this occurs within a field of the record definition of org.foo.Y, then the fullname is org.foo.X.
References to previously defined names are as in the latter two cases above: if they contain a dot they are a fullname, if they do not contain a dot, the namespace is the namespace of the enclosing definition.
Primitive type names have no namespace and their names may not be defined in any namespace. A schema may only contain multiple definitions of a fullname if the definitions are equivalent.
Aliases
Named types and fields may have aliases. An implementation may optionally use aliases to map a writer’s schema to the reader’s. This facilitates both schema evolution as well as processing disparate datasets.
Aliases function by re-writing the writer’s schema using aliases from the reader’s schema. For example, if the writer’s schema was named “Foo” and the reader’s schema is named “Bar” and has an alias of “Foo”, then the implementation would act as though “Foo” were named “Bar” when reading. Similarly, if data was written as a record with a field named “x” and is read as a record with a field named “y” with alias “x”, then the implementation would act as though “x” were named “y” when reading.
A type alias may be specified either as a fully namespace-qualified, or relative to the namespace of the name it is an alias for. For example, if a type named “a.b” has aliases of “c” and “x.y”, then the fully qualified names of its aliases are “a.c” and “x.y”.
Data Serialization
Avro data is always serialized with its schema. Files that store Avro data should always also include the schema for that data in the same file. Avro-based remote procedure call (RPC) systems must also guarantee that remote recipients of data have a copy of the schema used to write that data.
Because the schema used to write data is always available when the data is read, Avro data itself is not tagged with type information. The schema is required to parse data.
In general, both serialization and deserialization proceed as a depth-first, left-to-right traversal of the schema, serializing primitive types as they are encountered.
Encodings
Avro specifies two serialization encodings: binary and JSON. Most applications will use the binary encoding, as it is smaller and faster. But, for debugging and web-based applications, the JSON encoding may sometimes be appropriate.
Binary Encoding
Primitive Types
Primitive types are encoded in binary as follows:
- null is written as zero bytes.
- a boolean is written as a single byte whose value is either 0 (false) or 1 (true).
- int and long values are written using variable-length zig-zag coding. Some examples:
| value |
hex |
| 0 |
00 |
| -1 |
01 |
| 1 |
02 |
| -2 |
03 |
| 2 |
04 |
| … |
|
| -64 |
7f |
| 64 |
80 01 |
| … |
|
- a float is written as 4 bytes. The float is converted into a 32-bit integer using a method equivalent to Java’s floatToIntBits and then encoded in little-endian format.
- a double is written as 8 bytes. The double is converted into a 64-bit integer using a method equivalent to Java’s doubleToLongBits and then encoded in little-endian format.
- bytes are encoded as a long followed by that many bytes of data.
- a string is encoded as a long followed by that many bytes of UTF-8 encoded character data.
For example, the three-character string “foo” would be encoded as the long value 3 (encoded as hex 06) followed by the UTF-8 encoding of ‘f’, ‘o’, and ‘o’ (the hex bytes 66 6f 6f):
06 66 6f 6f
Complex Types
Complex types are encoded in binary as follows:
Records
A record is encoded by encoding the values of its fields in the order that they are declared. In other words, a record is encoded as just the concatenation of the encodings of its fields. Field values are encoded per their schema.
For example, the record schema
{
"type": "record",
"name": "test",
"fields" : [
{"name": "a", "type": "long"},
{"name": "b", "type": "string"}
]
}
An instance of this record whose a field has value 27 (encoded as hex 36) and whose b field has value “foo” (encoded as hex bytes 06 66 6f 6f), would be encoded simply as the concatenation of these, namely the hex byte sequence:
36 06 66 6f 6f
Enums
An enum is encoded by a int, representing the zero-based position of the symbol in the schema.
For example, consider the enum:
{"type": "enum", "name": "Foo", "symbols": ["A", "B", "C", "D"] }
This would be encoded by an int between zero and three, with zero indicating “A”, and 3 indicating “D”.
Arrays
Arrays are encoded as a series of blocks. Each block consists of a long count value, followed by that many array items. A block with count zero indicates the end of the array. Each item is encoded per the array’s item schema.
If a block’s count is negative, its absolute value is used, and the count is followed immediately by a long block size indicating the number of bytes in the block. This block size permits fast skipping through data, e.g., when projecting a record to a subset of its fields.
For example, the array schema
{"type": "array", "items": "long"}
an array containing the items 3 and 27 could be encoded as the long value 2 (encoded as hex 04) followed by long values 3 and 27 (encoded as hex 06 36) terminated by zero:
04 06 36 00
The blocked representation permits one to read and write arrays larger than can be buffered in memory, since one can start writing items without knowing the full length of the array.
Maps
Maps are encoded as a series of blocks. Each block consists of a long count value, followed by that many key/value pairs. A block with count zero indicates the end of the map. Each item is encoded per the map’s value schema.
If a block’s count is negative, its absolute value is used, and the count is followed immediately by a long block size indicating the number of bytes in the block. This block size permits fast skipping through data, e.g., when projecting a record to a subset of its fields.
The blocked representation permits one to read and write maps larger than can be buffered in memory, since one can start writing items without knowing the full length of the map.
Unions
A union is encoded by first writing a long value indicating the zero-based position within the union of the schema of its value. The value is then encoded per the indicated schema within the union.
For example, the union schema ["string","null"] would encode:
-
null as the integer 1 (the index of “null” in the union, encoded as hex 02):
02
-
the string “a” as zero (the index of “string” in the union), followed by the serialized string:
02 02 61
Fixed
Fixed instances are encoded using the number of bytes declared in the schema.
JSON Encoding
Except for unions, the JSON encoding is the same as is used to encode field default values.
The value of a union is encoded in JSON as follows:
- if its type is null, then it is encoded as a JSON null;
- otherwise it is encoded as a JSON object with one name/value pair whose name is the type’s name and whose value is the recursively encoded value. For Avro’s named types (record, fixed or enum) the user-specified name is used, for other types the type name is used.
For example, the union schema ["null","string","Foo"], where Foo is a record name, would encode:
- null as null;
- the string “a” as
{"string": "a"} and
- a Foo instance as
{"Foo": {...}}, where {...} indicates the JSON encoding of a Foo instance.
Note that a schema is still required to correctly process JSON-encoded data. For example, the JSON encoding does not distinguish between int and long, float and double, records and maps, enums and strings, etc.
Sort Order
Avro defines a standard sort order for data. This permits data written by one system to be efficiently sorted by another system. This can be an important optimization, as sort order comparisons are sometimes the most frequent per-object operation. Note also that Avro binary-encoded data can be efficiently ordered without deserializing it to objects.
Data items may only be compared if they have identical schemas. Pairwise comparisons are implemented recursively with a depth-first, left-to-right traversal of the schema. The first mismatch encountered determines the order of the items.
Two items with the same schema are compared according to the following rules.
- null data is always equal.
- boolean data is ordered with false before true.
- int, long, float and double data is ordered by ascending numeric value.
- bytes and fixed data are compared lexicographically by unsigned 8-bit values.
- string data is compared lexicographically by Unicode code point. Note that since UTF-8 is used as the binary encoding for strings, sorting of bytes and string binary data is identical.
- array data is compared lexicographically by element.
- enum data is ordered by the symbol’s position in the enum schema. For example, an enum whose symbols are
["z", "a"] would sort “z” values before “a” values.
- union data is first ordered by the branch within the union, and, within that, by the type of the branch. For example, an
["int", "string"] union would order all int values before all string values, with the ints and strings themselves ordered as defined above.
- record data is ordered lexicographically by field. If a field specifies that its order is:
- “ascending”, then the order of its values is unaltered.
- “descending”, then the order of its values is reversed.
- “ignore”, then its values are ignored when sorting.
- map data may not be compared. It is an error to attempt to compare data containing maps unless those maps are in an
"order":"ignore" record field.
Object Container Files
Avro includes a simple object container file format. A file has a schema, and all objects stored in the file must be written according to that schema, using binary encoding. Objects are stored in blocks that may be compressed. Syncronization markers are used between blocks to permit efficient splitting of files for MapReduce processing.
Files may include arbitrary user-specified metadata.
A file consists of:
- A file header, followed by
- one or more file data blocks.
A file header consists of:
- Four bytes, ASCII ‘O’, ‘b’, ‘j’, followed by 1.
- file metadata, including the schema.
- The 16-byte, randomly-generated sync marker for this file.
File metadata consists of:
A long indicating the number of metadata key/value pairs.
For each pair, a string key and bytes value.
All metadata properties that start with “avro.” are reserved. The following file metadata properties are currently used:
- avro.schema contains the schema of objects stored in the file, as JSON data (required).
- avro.codec the name of the compression codec used to compress blocks, as a string. Implementations are required to support the following codecs: “null” and “deflate”. If codec is absent, it is assumed to be “null”. The codecs are described with more detail below.
A file header is thus described by the following schema:
{"type": "record", "name": "org.apache.avro.file.Header",
"fields" : [
{"name": "magic", "type": {"type": "fixed", "name": "Magic", "size": 4}},
{"name": "meta", "type": {"type": "map", "values": "bytes"}},
{"name": "sync", "type": {"type": "fixed", "name": "Sync", "size": 16}}
]
}
A file data block consists of:
- A long indicating the count of objects in this block.
- A long indicating the size in bytes of the serialized objects in the current block, after any codec is applied
- The serialized objects. If a codec is specified, this is compressed by that codec.
- The file’s 16-byte sync marker.
Thus, each block’s binary data can be efficiently extracted or skipped without deserializing the contents. The combination of block size, object counts, and sync markers enable detection of corrupt blocks and help ensure data integrity.
Required Codecs
null
The “null” codec simply passes through data uncompressed.
deflate
The “deflate” codec writes the data block using the deflate algorithm as specified in RFC 1951, and typically implemented using the zlib library. Note that this format (unlike the “zlib format” in RFC 1950) does not have a checksum.
Optional Codecs
snappy
The “snappy” codec uses Google’s Snappy compression library. Each compressed block is followed by the 4-byte, big-endian CRC32 checksum.
Protocol Declaration
Avro protocols describe RPC interfaces. Like schemas, they are defined with JSON text.
A protocol is a JSON object with the following attributes:
- protocol, a string, the name of the protocol (required);
- namespace, an optional string that qualifies the name;
- doc, an optional string describing this protocol;
- types, an optional list of definitions of named types (records, enums, fixed and errors). An error definition is just like a record definition except it uses “error” instead of “record”. Note that forward references to named types are not permitted.
- messages, an optional JSON object whose keys are message names and whose values are objects whose attributes are described below. No two messages may have the same name.
The name and namespace qualification rules defined for schema objects apply to protocols as well.
Messages
A message has attributes:
- a doc, an optional description of the message,
- a request, a list of named, typed parameter schemas (this has the same form as the fields of a record declaration);
- a response schema;
- an optional union of declared error schemas. The effective union has “string” prepended to the declared union, to permit transmission of undeclared “system” errors. For example, if the declared error union is
["AccessError"], then the effective union is ["string", "AccessError"]. When no errors are declared, the effective error union is ["string"]. Errors are serialized using the effective union; however, a protocol’s JSON declaration contains only the declared union.
- an optional one-way boolean parameter.
A request parameter list is processed equivalently to an anonymous record. Since record field lists may vary between reader and writer, request parameters may also differ between the caller and responder, and such differences are resolved in the same manner as record field differences.
The one-way parameter may only be true when the response type is "null" and no errors are listed.
Sample Protocol
For example, one may define a simple HelloWorld protocol with:
{
"namespace": "com.acme",
"protocol": "HelloWorld",
"doc": "Protocol Greetings",
"types": [
{"name": "Greeting", "type": "record", "fields": [
{"name": "message", "type": "string"}]},
{"name": "Curse", "type": "error", "fields": [
{"name": "message", "type": "string"}]}
],
"messages": {
"hello": {
"doc": "Say hello.",
"request": [{"name": "greeting", "type": "Greeting" }],
"response": "Greeting",
"errors": ["Curse"]
}
}
}
Message Transport
Messages may be transmitted via different transport mechanisms.
To the transport, a message is an opaque byte sequence.
A transport is a system that supports:
- transmission of request messages
- receipt of corresponding response messages
Servers may send a response message back to the client corresponding to a request message. The mechanism of correspondance is transport-specific. For example, in HTTP it is implicit, since HTTP directly supports requests and responses. But a transport that multiplexes many client threads over a single socket would need to tag messages with unique identifiers.
Transports may be either stateless or stateful. In a stateless transport, messaging assumes no established connection state, while stateful transports establish connections that may be used for multiple messages. This distinction is discussed further in the handshake section below.
HTTP as Transport
When HTTP is used as a transport, each Avro message exchange is an HTTP request/response pair. All messages of an Avro protocol should share a single URL at an HTTP server. Other protocols may also use that URL. Both normal and error Avro response messages should use the 200 (OK) response code. The chunked encoding may be used for requests and responses, but, regardless the Avro request and response are the entire content of an HTTP request and response. The HTTP Content-Type of requests and responses should be specified as “avro/binary”. Requests should be made using the POST method.
HTTP is used by Avro as a stateless transport.
Message Framing
Avro messages are framed as a list of buffers.
Framing is a layer between messages and the transport. It exists to optimize certain operations.
The format of framed message data is:
- a series of buffers, where each buffer consists of:
- a four-byte, big-endian buffer length, followed by
- that many bytes of buffer data.
- A message is always terminated by a zero-lenghted buffer.
Framing is transparent to request and response message formats (described below). Any message may be presented as a single or multiple buffers.
Framing can permit readers to more efficiently get different buffers from different sources and for writers to more efficiently store different buffers to different destinations. In particular, it can reduce the number of times large binary objects are copied. For example, if an RPC parameter consists of a megabyte of file data, that data can be copied directly to a socket from a file descriptor, and, on the other end, it could be written directly to a file descriptor, never entering user space.
A simple, recommended, framing policy is for writers to create a new segment whenever a single binary object is written that is larger than a normal output buffer. Small objects are then appended in buffers, while larger objects are written as their own buffers. When a reader then tries to read a large object the runtime can hand it an entire buffer directly, without having to copy it.
Handshake
The purpose of the handshake is to ensure that the client and the server have each other’s protocol definition, so that the client can correctly deserialize responses, and the server can correctly deserialize requests. Both clients and servers should maintain a cache of recently seen protocols, so that, in most cases, a handshake will be completed without extra round-trip network exchanges or the transmission of full protocol text.
RPC requests and responses may not be processed until a handshake has been completed. With a stateless transport, all requests and responses are prefixed by handshakes. With a stateful transport, handshakes are only attached to requests and responses until a successful handshake response has been returned over a connection. After this, request and response payloads are sent without handshakes for the lifetime of that connection.
The handshake process uses the following record schemas:
{
"type": "record",
"name": "HandshakeRequest", "namespace":"org.apache.avro.ipc",
"fields": [
{"name": "clientHash",
"type": {"type": "fixed", "name": "MD5", "size": 16}},
{"name": "clientProtocol", "type": ["null", "string"]},
{"name": "serverHash", "type": "MD5"},
{"name": "meta", "type": ["null", {"type": "map", "values": "bytes"}]}
]
}
{
"type": "record",
"name": "HandshakeResponse", "namespace": "org.apache.avro.ipc",
"fields": [
{"name": "match",
"type": {"type": "enum", "name": "HandshakeMatch",
"symbols": ["BOTH", "CLIENT", "NONE"]}},
{"name": "serverProtocol",
"type": ["null", "string"]},
{"name": "serverHash",
"type": ["null", {"type": "fixed", "name": "MD5", "size": 16}]},
{"name": "meta",
"type": ["null", {"type": "map", "values": "bytes"}]}
]
}
- A client first prefixes each request with a
HandshakeRequest containing just the hash of its protocol and of the server’s protocol (clientHash!=null, clientProtocol=null, serverHash!=null), where the hashes are 128-bit MD5 hashes of the JSON protocol text. If a client has never connected to a given server, it sends its hash as a guess of the server’s hash, otherwise it sends the hash that it previously obtained from this server.
The server responds with a HandshakeResponse containing one of:
-
match=BOTH, serverProtocol=null, serverHash=null if the client sent the valid hash of the server’s protocol and the server knows what protocol corresponds to the client’s hash. In this case, the request is complete and the response data immediately follows the HandshakeResponse.
-
match=CLIENT, serverProtocol!=null, serverHash!=null if the server has previously seen the client’s protocol, but the client sent an incorrect hash of the server’s protocol. The request is complete and the response data immediately follows the HandshakeResponse. The client must use the returned protocol to process the response and should also cache that protocol and its hash for future interactions with this server.
-
match=NONE if the server has not previously seen the client’s protocol. The serverHash and serverProtocol may also be non-null if the server’s protocol hash was incorrect.
In this case the client must then re-submit its request with its protocol text (clientHash!=null, clientProtocol!=null, serverHash!=null) and the server should respond with a successful match (match=BOTH, serverProtocol=null, serverHash=null) as above.
The meta field is reserved for future handshake enhancements.
A call consists of a request message paired with its resulting response or error message. Requests and responses contain extensible metadata, and both kinds of messages are framed as described above.
The format of a call request is:
- request metadata, a map with values of type bytes
- the message name, an Avro string, followed by
- the message parameters. Parameters are serialized according to the message’s request declaration.
When a message is declared one-way and a stateful connection has been established by a successful handshake response, no response data is sent. Otherwise the format of the call response is:
- response metadata, a map with values of type bytes
- a one-byte error flag boolean, followed by either:
- if the error flag is false, the message response, serialized per the message’s response schema.
- if the error flag is true, the error, serialized per the message’s effective error union schema.
Schema Resolution
A reader of Avro data, whether from an RPC or a file, can always parse that data because its schema is provided. But that schema may not be exactly the schema that was expected. For example, if the data was written with a different version of the software than it is read, then records may have had fields added or removed. This section specifies how such schema differences should be resolved.
We refer to the schema used to write the data as the writer’s schema, and the schema that the application expects the reader’s schema. Differences between these should be resolved as follows:
- It is an error if the two schemas do not match.
To match, one of the following must hold:
-
both schemas are arrays whose item types match
-
both schemas are maps whose value types match
-
both schemas are enums whose names match
-
both schemas are fixed whose sizes and names match
-
both schemas are records with the same name
-
either schema is a union
-
both schemas have same primitive type
-
the writer’s schema may be promoted to the reader’s as follows:
- int is promotable to long, float, or double
- long is promotable to float or double
- float is promotable to double
- string is promotable to bytes
- bytes is promotable to string
-
if both are records:
- the ordering of fields may be different: fields are matched by name.
- schemas for fields with the same name in both records are resolved recursively.
- if the writer’s record contains a field with a name not present in the reader’s record, the writer’s value for that field is ignored.
- if the reader’s record schema has a field that contains a default value, and writer’s schema does not have a field with the same name, then the reader should use the default value from its field.
- if the reader’s record schema has a field with no default value, and writer’s schema does not have a field with the same name, an error is signalled.
-
if both are enums:
if the writer’s symbol is not present in the reader’s enum, then an error is signalled.
This resolution algorithm is applied recursively to the reader’s and writer’s array item schemas.
This resolution algorithm is applied recursively to the reader’s and writer’s value schemas.
The first schema in the reader’s union that matches the selected writer’s union schema is recursively resolved against it. if none match, an error is signalled.
- if reader’s is a union, but writer’s is not:
The first schema in the reader’s union that matches the writer’s schema is recursively resolved against it. If none match, an error is signalled.
- if writer’s is a union, but reader’s is not:
If the reader’s schema matches the selected writer’s schema, it is recursively resolved against it. If they do not match, an error is signalled.
A schema’s “doc” fields are ignored for the purposes of schema resolution. Hence, the “doc” portion of a schema may be dropped at serialization.
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
23.2 - IDL Language
Introduction
This document defines Avro IDL, a higher-level language for authoring Avro schemata. Before reading this document, you should have familiarity with the concepts of schemata and protocols, as well as the various primitive and complex types available in Avro.
Overview
Purpose
The aim of the Avro IDL language is to enable developers to author schemata in a way that feels more similar to common programming languages like Java, C++, or Python. Additionally, the Avro IDL language may feel more familiar for those users who have previously used the interface description languages (IDLs) in other frameworks like Thrift, Protocol Buffers, or CORBA.
Usage
Each Avro IDL file defines a single Avro Protocol, and thus generates as its output a JSON-format Avro Protocol file with extension .avpr.
To convert a .avdl file into a .avpr file, it may be processed by the idl tool. For example:
$ java -jar avroj-tools-1.4.0.jar idl src/test/idl/input/namespaces.avdl /tmp/namespaces.avpr
$ head /tmp/namespaces.avpr
{
"protocol" : "TestNamespace",
"namespace" : "avro.test.protocol",
The idl tool can also process input to and from stdin and stdout. See idl --help for full usage information.
Defining a Protocol in Avro IDL
An Avro IDL file consists of exactly one protocol definition. The minimal protocol is defined by the following code:
This is equivalent to (and generates) the following JSON protocol definition:
{
"protocol" : "MyProtocol",
"types" : [ ],
"messages" : {
}
}
The namespace of the protocol may be changed using the @namespace annotation:
@namespace("mynamespace")
protocol MyProtocol {
}
This notation is used throughout Avro IDL as a way of specifying properties for the annotated element, as will be described later in this document.
Protocols in Avro IDL can contain the following items:
- Imports of external protocol and schema files.
- Definitions of named schemata, including records, errors, enums, and fixeds.
- Definitions of RPC messages
Imports
Files may be imported in one of three formats:
-
An IDL file may be imported with a statement like:
import idl "foo.avdl";
-
A JSON protocol file may be imported with a statement like:
import protocol "foo.avpr";
-
A JSON schema file may be imported with a statement like:
import schema "foo.avsc";
Messages and types in the imported file are added to this file’s protocol.
Imported file names are resolved relative to the current IDL file.
Defining an Enumeration
Enums are defined in Avro IDL using a syntax similar to C or Java:
enum Suit {
SPADES, DIAMONDS, CLUBS, HEARTS
}
Note that, unlike the JSON format, anonymous enums cannot be defined.
Defining a Fixed Length Field
Fixed fields are defined using the following syntax:
fixed MD5(16);
This example defines a fixed-length type called MD5 which contains 16 bytes.
Defining Records and Errors
Records are defined in Avro IDL using a syntax similar to a struct definition in C:
record Employee {
string name;
boolean active = true;
long salary;
}
The above example defines a record with the name “Employee” with three fields.
To define an error, simply use the keyword error instead of record. For example:
error Kaboom {
string explanation;
int result_code = -1;
}
Each field in a record or error consists of a type and a name, optional property annotations and an optional default value.
A type reference in Avro IDL must be one of:
- A primitive type
- A logical type
- A named schema defined prior to this usage in the same Protocol
- A complex type (array, map, or union)
Primitive Types
The primitive types supported by Avro IDL are the same as those supported by Avro’s JSON format. This list includes int, long, string, boolean, float, double, null, and bytes.
References to Named Schemata
If a named schema has already been defined in the same Avro IDL file, it may be referenced by name as if it were a primitive type:
record Card {
Suit suit; // refers to the enum Card defined above
int number;
}
Default Values
Default values for fields may be optionally specified by using an equals sign after the field name followed by a JSON expression indicating the default value. This JSON is interpreted as described in the spec.
Complex Types
Arrays
Array types are written in a manner that will seem familiar to C++ or Java programmers. An array of any type t is denoted array<t>. For example, an array of strings is denoted array<string>, and a multidimensional array of Foo records would be array<array<Foo>>.
Maps
Map types are written similarly to array types. An array that contains values of type t is written map<t>. As in the JSON schema format, all maps contain string-type keys.
Unions
Union types are denoted as union { typeA, typeB, typeC, … }. For example, this record contains a string field that is optional (unioned with null):
record RecordWithUnion {
union { null, string } optionalString;
}
Note that the same restrictions apply to Avro IDL unions as apply to unions defined in the JSON format; namely, a record may not contain multiple elements of the same type.
Defining RPC Messages
The syntax to define an RPC message within a Avro IDL protocol is similar to the syntax for a method declaration within a C header file or a Java interface. To define an RPC message add which takes two arguments named foo and bar, returning an int, simply include the following definition within the protocol:
int add(int foo, int bar = 0);
Message arguments, like record fields, may specify default values.
To define a message with no response, you may use the alias void, equivalent to the Avro null type:
void logMessage(string message);
If you have previously defined an error type within the same protocol, you may declare that a message can throw this error using the syntax:
void goKaboom() throws Kaboom;
To define a one-way message, use the keyword oneway after the parameter list, for example:
void fireAndForget(string message) oneway;
Other Language Features
All Java-style comments are supported within a Avro IDL file. Any text following // on a line is ignored, as is any text between /* and */, possibly spanning multiple lines.
Comments that begin with /** are used as the documentation string for the type or field definition that follows the comment.
Escaping Identifiers
Occasionally, one will need to use a reserved language keyword as an identifier. In order to do so, backticks (`) may be used to escape the identifier. For example, to define a message with the literal name error, you may write:
This syntax is allowed anywhere an identifier is expected.
Annotations for Ordering and Namespaces
Java-style annotations may be used to add additional properties to types and fields throughout Avro IDL.
For example, to specify the sort order of a field within a record, one may use the @order annotation before the field name as follows:
record MyRecord {
string @order("ascending") myAscendingSortField;
string @order("descending") myDescendingField;
string @order("ignore") myIgnoredField;
}
A field’s type may also be preceded by annotations, e.g.:
record MyRecord {
@java-class("java.util.ArrayList") array<string> myStrings;
}
Similarly, a @namespace annotation may be used to modify the namespace when defining a named schema. For example:
@namespace("org.apache.avro.firstNamespace")
protocol MyProto {
@namespace("org.apache.avro.someOtherNamespace")
record Foo {}
record Bar {}
}
will define a protocol in the firstNamespace namespace. The record Foo will be defined in someOtherNamespace and Bar will be defined in firstNamespace as it inherits its default from its container.
Type and field aliases are specified with the @aliases annotation as follows:
@aliases(["org.old.OldRecord", "org.ancient.AncientRecord"])
record MyRecord {
string @aliases(["oldField", "ancientField"]) myNewField;
}
Complete Example
The following is a complete example of a Avro IDL file that shows most of the above features:
/**
* An example protocol in Avro IDL
*/
@namespace("org.apache.avro.test")
protocol Simple {
@aliases(["org.foo.KindOf"])
enum Kind {
FOO,
BAR, // the bar enum value
BAZ
}
fixed MD5(16);
record TestRecord {
@order("ignore")
string name;
@order("descending")
Kind kind;
MD5 hash;
union { MD5, null} @aliases(["hash"]) nullableHash;
array<long> arrayOfLongs;
}
error TestError {
string message;
}
string hello(string greeting);
TestRecord echo(TestRecord `record`);
int add(int arg1, int arg2);
bytes echoBytes(bytes data);
void `error`() throws TestError;
void ping() oneway;
}
Additional examples may be found in the Avro source tree under the src/test/idl/input directory.
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
23.3 - SASL profile
Introduction
SASL (RFC 2222) provides a framework for authentication and security of network protocols. Each protocol that uses SASL is meant to define a SASL profile. This document provides a SASL profile for connection-based Avro RPC.
Overview
SASL negotiation proceeds as a series of message interactions over a connection between a client and server using a selected SASL mechanism. The client starts this negotiation by sending its chosen mechanism name with an initial (possibly empty) message. Negotiation proceeds with the exchange of messages until either side indicates success or failure. The content of the messages is mechanism-specific. If the negotiation succeeds, then the session can proceed over the connection, otherwise it must be abandoned.
Some mechanisms continue to process session data after negotiation (e.g., encrypting it), while some specify that further session data is transmitted unmodified.
Negotiation
Commands
Avro SASL negotiation uses four one-byte commands.
- 0: START Used in a client’s initial message.
- 1: CONTINUE Used while negotiation is ongoing.
- 2: FAIL Terminates negotiation unsuccessfully.
- 3: COMPLETE Terminates negotiation successfully.
The format of a START message is:
| 0 | 4-byte mechanism name length | mechanism name | 4-byte payload length | payload data |
The format of a CONTINUE message is:
| 1 | 4-byte payload length | payload data |
The format of a FAIL message is:
| 2 | 4-byte message length | UTF-8 message |
The format of a COMPLETE message is:
| 3 | 4-byte payload length | payload data |
Process
Negotiation is initiated by a client sending a START command containing the client’s chosen mechanism name and any mechanism-specific payload data.
The server and client then interchange some number (possibly zero) of CONTINUE messages. Each message contains payload data that is processed by the security mechanism to generate the next message.
Once either the client or server send a FAIL message then negotiation has failed. UTF-8-encoded text is included in the failure message. Once either a FAIL message has been sent or received, or any other error occurs in the negotiation, further communication on this connection must cease.
Once either the client or server send a COMPLETE message then negotiation has completed successfully. Session data may now be transmitted over the connection until it is closed by either side.
Session Data
If no SASL QOP (quality of protection) is negotiated, then all subsequent writes to/reads over this connection are written/read unmodified. In particular, messages use Avro framing, and are of the form:
| 4-byte frame length | frame data | ... | 4 zero bytes |
If a SASL QOP is negotiated, then it must be used by the connection for all subsequent messages. This is done by wrapping each non-empty frame written using the security mechanism and unwrapping each non-empty frame read. The length written in each non-empty frame is the length of the wrapped data. Complete frames must be passed to the security mechanism for unwrapping. Unwrapped data is then passed to the application as the content of the frame.
If at any point processing fails due to wrapping, unwrapping or framing errors, then all further communication on this connection must cease.
Anonymous Mechanism
The SASL anonymous mechanism (RFC 2245) is quite simple to implement. In particular, an initial anonymous request may be prefixed by the following static sequence:
| 0 | 0009 | ANONYMOUS | 0000 |
If a server uses the anonymous mechanism, it should check that the mechanism name in the start message prefixing the first request received is ‘ANONYMOUS’, then simply prefix its initial response with a COMPLETE message of:
| 3 | 0000 |
If an anonymous server recieves some other mechanism name, then it may respond with a FAIL message as simple as:
| 2 | 0000 |
Note that the anonymous mechanism need add no additional round-trip messages between client and server. The START message can be piggybacked on the initial request and the COMPLETE or FAIL message can be piggybacked on the initial response.
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
23.4 - Wiki
The Wiki page can be found here.
23.5 - FAQ
What is Avro?
Avro is a data serialization system.
Avro provides:
- Rich data structures.
- A compact, fast, binary data format.
- A container file, to store persistent data.
- Remote procedure call (RPC).
- Simple integration with dynamic languages. Code generation is not required to read or write data files nor to use or implement RPC protocols. Code generation as an optional optimization, only worth implementing for statically typed languages.
How can I get started quickly with Avro?
Check out the Quick Start Guide.
How do I statically compile a schema or protocol into generated code?
In Java
- Add the avro jar, the jackson-mapper-asl.jar and jackson-core-asl.jar to your CLASSPATH.
- Run java org.apache.avro.specific.SpecificCompiler .
This appears to be out of date, the SpecificCompiler requires two arguments, presumably an input and and output file, but it isn’t clear that this does.
Or use the Schema or Protocol Ant tasks. Avro’s build.xml provides examples of how these are used.
Lastly, you can also use the “avro-tools” jar which ships with an Avro release. Just use the “compile (schema|protocol)” command.
How are Strings represented in Java?
They use org.apache.avro.util.Utf8, not java.lang.String.
More generally, how do Avro types map to Java types?
The mappings are documented in the package javadoc for generic, specific and reflect API.
From Doug Cutting:
HDFS splits files into blocks, and mapreduce runs a map task for each block. When the task starts, it needs to be able to seek into the file to the start of the block process through the block’s end. If the file were, e.g., a gzip file, this would not be possible, since gzip files must be decompressed from the start. One cannot seek into the middle of a gzip file and start decompressing. So Hadoop’s SequenceFile places a marker periodically (~64k) in the file at record and compression boundaries, where processing can be sensibly started. Then, when a map task starts processing an HDFS block, it finds the first marker after the block’s start and continues through the first marker in the next block of the file. This requires a bit of non-local access (~0.1%). Avro’s data file uses the same method as SequenceFile.
Why isn’t every value in Avro nullable?
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 of every value stored and
corresponding computational costs to create this bit on write and
interpret it on read. These costs are wasted when values may not in
fact be null, as is the case in many datasets. In Avro such costs are
only paid when values may actually be null.
Also, allowing values to be null is a well-known source of errors. In
Avro, a value declared as non-null will always be non-null and programs
need not test for null values when processing it nor will they ever fail
for lack of such tests.
Tony Hoare calls his invention of null references his “Billion Dollar
Mistake”.
http://qconlondon.com/london-2009/presentation/Null+References:+The+Billion+Dollar+Mistake
Also note that in some programming languages not all values are permitted to be null. For example, in Java, values of type boolean, byte, short, char, int, float, long, and double may not be null.
How can I serialize directly to/from a byte array?
As pointed out in the specification, Avro data should always be stored with its schema. The Avro provided classes DataFileWriter, DataFileReader, and DataFileStream all ensure this by serializing the Schema in a container header. In some special cases, such as when implementing a new storage system or writing unit tests, you may need to write and read directly with the bare Avro serialized values. The following examples use code generated for Java from an Avro Schema, specifically the User example from the Quickstart guide.
Serializing to a byte array
This example takes a User object and returns a newly allocated byte array with the Avro serialization of that user.
ByteArrayOutputStream out = new ByteArrayOutputStream();
BinaryEncoder encoder = EncoderFactory.get().binaryEncoder(out, null);
DatumWriter<User> writer = new SpecificDatumWriter<User>(User.getClassSchema());
writer.write(user, encoder);
encoder.flush();
out.close();
byte[] serializedBytes = out.toByteArray();
Deserializing from a byte array
This example takes a byte array containing the Avro serialization of a user and returns a User object.SpecificDatumReader reader = new SpecificDatumReader(User.getClassSchema());
Decoder decoder = DecoderFactory.get().binaryDecoder(bytes, null);
User user = reader.read(null, decoder);
Note: If you are serializing or deserializing in a loop or as a method, you should be reusing objects, readers and/or writers. Check the JavaDoc to see how to reuse the objects.
24 - Apache Avro™ 1.5.4 Documentation
Introduction
Apache Avro™ is a data serialization system.
Avro provides:
- Rich data structures.
- A compact, fast, binary data format.
- A container file, to store persistent data.
- Remote procedure call (RPC).
- Simple integration with dynamic languages. Code generation is not required to read or write data files nor to use or implement RPC protocols. Code generation as an optional optimization, only worth implementing for statically typed languages.
Schemas
Avro relies on schemas. When Avro data is read, the schema used when writing it is always present. This permits each datum to be written with no per-value overheads, making serialization both fast and small. This also facilitates use with dynamic, scripting languages, since data, together with its schema, is fully self-describing.
When Avro data is stored in a file, its schema is stored with it, so that files may be processed later by any program. If the program reading the data expects a different schema this can be easily resolved, since both schemas are present.
When Avro is used in RPC, the client and server exchange schemas in the connection handshake. (This can be optimized so that, for most calls, no schemas are actually transmitted.) Since both client and server both have the other’s full schema, correspondence between same named fields, missing fields, extra fields, etc. can all be easily resolved.
Avro schemas are defined with JSON . This facilitates implementation in languages that already have JSON libraries.
Comparison with other systems
Avro provides functionality similar to systems such as Thrift, Protocol Buffers, etc. Avro differs from these systems in the following fundamental aspects.
- Dynamic typing: Avro does not require that code be generated. Data is always accompanied by a schema that permits full processing of that data without code generation, static datatypes, etc. This facilitates construction of generic data-processing systems and languages.
- Untagged data: Since the schema is present when data is read, considerably less type information need be encoded with data, resulting in smaller serialization size.
- No manually-assigned field IDs: When a schema changes, both the old and new schema are always present when processing data, so differences may be resolved symbolically, using field names.
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
24.1 - Specification
Introduction
This document defines Apache Avro. It is intended to be the authoritative specification. Implementations of Avro must adhere to this document.
Schema Declaration
A Schema is represented in JSON by one of:
- A JSON string, naming a defined type.
- A JSON object, of the form:
{"type": "typeName" ...attributes...}
where typeName is either a primitive or derived type name, as defined below. Attributes not defined in this document are permitted as metadata, but must not affect the format of serialized data.
- A JSON array, representing a union of embedded types.
Primitive Types
The set of primitive type names is:
- null: no value
- boolean: a binary value
- int: 32-bit signed integer
- long: 64-bit signed integer
- float: single precision (32-bit) IEEE 754 floating-point number
- double: double precision (64-bit) IEEE 754 floating-point number
- bytes: sequence of 8-bit unsigned bytes
- string: unicode character sequence
Primitive types have no specified attributes.
Primitive type names are also defined type names. Thus, for example, the schema “string” is equivalent to:
Complex Types
Avro supports six kinds of complex types: records, enums, arrays, maps, unions and fixed.
Records
Records use the type name “record” and support three attributes:
- name: a JSON string providing the name of the record (required).
- namespace, a JSON string that qualifies the name;
- doc: a JSON string providing documentation to the user of this schema (optional).
- aliases: a JSON array of strings, providing alternate names for this record (optional).
- fields: a JSON array, listing fields (required). Each field is a JSON object with the following attributes:
- name: a JSON string providing the name of the field (required), and
- doc: a JSON string describing this field for users (optional).
- type: A JSON object defining a schema, or a JSON string naming a record definition (required).
- default: A default value for this field, used when reading instances that lack this field (optional). 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. Default values for bytes and fixed fields are JSON strings, where Unicode code points 0-255 are mapped to unsigned 8-bit byte values 0-255.
field default values
| avro type |
json type |
example |
| null |
null |
null |
| boolean |
boolean |
true |
| int,long |
integer |
1 |
| float,double |
number |
1.1 |
| bytes |
string |
"\u00FF" |
| string |
string |
"foo" |
| record |
object |
{"a": 1} |
| enum |
string |
"FOO" |
| array |
array |
[1] |
| map |
object |
{"a": 1} |
| fixed |
string |
"\u00ff" |
- order: specifies how this field impacts sort ordering of this record (optional). Valid values are “ascending” (the default), “descending”, or “ignore”. For more details on how this is used, see the sort order section below.
- aliases: a JSON array of strings, providing alternate names for this field (optional).
For example, a linked-list of 64-bit values may be defined with:
{
"type": "record",
"name": "LongList",
"aliases": ["LinkedLongs"], // old name for this
"fields" : [
{"name": "value", "type": "long"}, // each element has a long
{"name": "next", "type": ["LongList", "null"]} // optional next element
]
}
Enums
Enums use the type name “enum” and support the following attributes:
- name: a JSON string providing the name of the enum (required).
- namespace, a JSON string that qualifies the name ;
- aliases: a JSON array of strings, providing alternate names for this enum (optional).
- doc: a JSON string providing documentation to the user of this schema (optional).
- symbols: a JSON array, listing symbols, as JSON strings (required). All symbols in an enum must be unique; duplicates are prohibited.
For example, playing card suits might be defined with:
{
"type": "enum",
"name": "Suit",
"symbols" : ["SPADES", "HEARTS", "DIAMONDS", "CLUBS"]
}
Arrays
Arrays use the type name “array” and support a single attribute:
- items: the schema of the array’s items.
For example, an array of strings is declared with:
{
"type": "array",
"items" : "string"
}
Maps
Maps use the type name “map” and support one attribute:
- values: the schema of the map’s values.
Map keys are assumed to be strings.
For example, a map from string to long is declared with:
{
"type": "map",
"values" : "long"
}
Unions
Unions, as mentioned above, are represented using JSON arrays. For example, ["string", "null"] declares a schema which may be either a string or null.
Unions may not contain more than one schema with the same type, except for the named types record, fixed and enum. For example, unions containing two array types or two map types are not permitted, but two types with different names are permitted. (Names permit efficient resolution when reading and writing unions.)
Unions may not immediately contain other unions.
Fixed
Fixed uses the type name “fixed” and supports two attributes:
- name: a string naming this fixed (required).
- namespace, a string that qualifies the name;
- aliases: a JSON array of strings, providing alternate names for this enum (optional).
- size: an integer, specifying the number of bytes per value (required).
For example, 16-byte quantity may be declared with:
{"type": "fixed", "size": 16, "name": "md5"}
Names
Record, enums and fixed are named types. Each has a fullname that is composed of two parts; a name and a namespace. Equality of names is defined on the fullname.
The name portion of a fullname, record field names, and enum symbols must:
- start with [A-Za-z_]
- subsequently contain only [A-Za-z0-9_]
A namespace is a dot-separated sequence of such names.
In record, enum and fixed definitions, the fullname is determined in one of the following ways:
A name and namespace are both specified. For example, one might use “name”: “X”, “namespace”: “org.foo” to indicate the fullname org.foo.X.
-
A fullname is specified. If the name specified contains a dot, then it is assumed to be a fullname, and any namespace also specified is ignored. For example, use “name”: “org.foo.X” to indicate the fullname org.foo.X.
-
A name only is specified, i.e., a name that contains no dots. In this case the namespace is taken from the most tightly enclosing schema or protocol. For example, if “name”: “X” is specified, and this occurs within a field of the record definition of org.foo.Y, then the fullname is org.foo.X.
References to previously defined names are as in the latter two cases above: if they contain a dot they are a fullname, if they do not contain a dot, the namespace is the namespace of the enclosing definition.
Primitive type names have no namespace and their names may not be defined in any namespace. A schema may only contain multiple definitions of a fullname if the definitions are equivalent.
Aliases
Named types and fields may have aliases. An implementation may optionally use aliases to map a writer’s schema to the reader’s. This facilitates both schema evolution as well as processing disparate datasets.
Aliases function by re-writing the writer’s schema using aliases from the reader’s schema. For example, if the writer’s schema was named “Foo” and the reader’s schema is named “Bar” and has an alias of “Foo”, then the implementation would act as though “Foo” were named “Bar” when reading. Similarly, if data was written as a record with a field named “x” and is read as a record with a field named “y” with alias “x”, then the implementation would act as though “x” were named “y” when reading.
A type alias may be specified either as a fully namespace-qualified, or relative to the namespace of the name it is an alias for. For example, if a type named “a.b” has aliases of “c” and “x.y”, then the fully qualified names of its aliases are “a.c” and “x.y”.
Data Serialization
Avro data is always serialized with its schema. Files that store Avro data should always also include the schema for that data in the same file. Avro-based remote procedure call (RPC) systems must also guarantee that remote recipients of data have a copy of the schema used to write that data.
Because the schema used to write data is always available when the data is read, Avro data itself is not tagged with type information. The schema is required to parse data.
In general, both serialization and deserialization proceed as a depth-first, left-to-right traversal of the schema, serializing primitive types as they are encountered.
Encodings
Avro specifies two serialization encodings: binary and JSON. Most applications will use the binary encoding, as it is smaller and faster. But, for debugging and web-based applications, the JSON encoding may sometimes be appropriate.
Binary Encoding
Primitive Types
Primitive types are encoded in binary as follows:
- null is written as zero bytes.
- a boolean is written as a single byte whose value is either 0 (false) or 1 (true).
- int and long values are written using variable-length zig-zag coding. Some examples:
| value |
hex |
| 0 |
00 |
| -1 |
01 |
| 1 |
02 |
| -2 |
03 |
| 2 |
04 |
| … |
|
| -64 |
7f |
| 64 |
80 01 |
| … |
|
- a float is written as 4 bytes. The float is converted into a 32-bit integer using a method equivalent to Java’s floatToIntBits and then encoded in little-endian format.
- a double is written as 8 bytes. The double is converted into a 64-bit integer using a method equivalent to Java’s doubleToLongBits and then encoded in little-endian format.
- bytes are encoded as a long followed by that many bytes of data.
- a string is encoded as a long followed by that many bytes of UTF-8 encoded character data.
For example, the three-character string “foo” would be encoded as the long value 3 (encoded as hex 06) followed by the UTF-8 encoding of ‘f’, ‘o’, and ‘o’ (the hex bytes 66 6f 6f):
06 66 6f 6f
Complex Types
Complex types are encoded in binary as follows:
Records
A record is encoded by encoding the values of its fields in the order that they are declared. In other words, a record is encoded as just the concatenation of the encodings of its fields. Field values are encoded per their schema.
For example, the record schema
{
"type": "record",
"name": "test",
"fields" : [
{"name": "a", "type": "long"},
{"name": "b", "type": "string"}
]
}
An instance of this record whose a field has value 27 (encoded as hex 36) and whose b field has value “foo” (encoded as hex bytes 06 66 6f 6f), would be encoded simply as the concatenation of these, namely the hex byte sequence:
36 06 66 6f 6f
Enums
An enum is encoded by a int, representing the zero-based position of the symbol in the schema.
For example, consider the enum:
{"type": "enum", "name": "Foo", "symbols": ["A", "B", "C", "D"] }
This would be encoded by an int between zero and three, with zero indicating “A”, and 3 indicating “D”.
Arrays
Arrays are encoded as a series of blocks. Each block consists of a long count value, followed by that many array items. A block with count zero indicates the end of the array. Each item is encoded per the array’s item schema.
If a block’s count is negative, its absolute value is used, and the count is followed immediately by a long block size indicating the number of bytes in the block. This block size permits fast skipping through data, e.g., when projecting a record to a subset of its fields.
For example, the array schema
{"type": "array", "items": "long"}
an array containing the items 3 and 27 could be encoded as the long value 2 (encoded as hex 04) followed by long values 3 and 27 (encoded as hex 06 36) terminated by zero:
04 06 36 00
The blocked representation permits one to read and write arrays larger than can be buffered in memory, since one can start writing items without knowing the full length of the array.
Maps
Maps are encoded as a series of blocks. Each block consists of a long count value, followed by that many key/value pairs. A block with count zero indicates the end of the map. Each item is encoded per the map’s value schema.
If a block’s count is negative, its absolute value is used, and the count is followed immediately by a long block size indicating the number of bytes in the block. This block size permits fast skipping through data, e.g., when projecting a record to a subset of its fields.
The blocked representation permits one to read and write maps larger than can be buffered in memory, since one can start writing items without knowing the full length of the map.
Unions
A union is encoded by first writing a long value indicating the zero-based position within the union of the schema of its value. The value is then encoded per the indicated schema within the union.
For example, the union schema ["string","null"] would encode:
-
null as the integer 1 (the index of “null” in the union, encoded as hex 02):
02
-
the string “a” as zero (the index of “string” in the union), followed by the serialized string:
02 02 61
Fixed
Fixed instances are encoded using the number of bytes declared in the schema.
JSON Encoding
Except for unions, the JSON encoding is the same as is used to encode field default values.
The value of a union is encoded in JSON as follows:
- if its type is null, then it is encoded as a JSON null;
- otherwise it is encoded as a JSON object with one name/value pair whose name is the type’s name and whose value is the recursively encoded value. For Avro’s named types (record, fixed or enum) the user-specified name is used, for other types the type name is used.
For example, the union schema ["null","string","Foo"], where Foo is a record name, would encode:
- null as null;
- the string “a” as
{"string": "a"} and
- a Foo instance as
{"Foo": {...}}, where {...} indicates the JSON encoding of a Foo instance.
Note that a schema is still required to correctly process JSON-encoded data. For example, the JSON encoding does not distinguish between int and long, float and double, records and maps, enums and strings, etc.
Sort Order
Avro defines a standard sort order for data. This permits data written by one system to be efficiently sorted by another system. This can be an important optimization, as sort order comparisons are sometimes the most frequent per-object operation. Note also that Avro binary-encoded data can be efficiently ordered without deserializing it to objects.
Data items may only be compared if they have identical schemas. Pairwise comparisons are implemented recursively with a depth-first, left-to-right traversal of the schema. The first mismatch encountered determines the order of the items.
Two items with the same schema are compared according to the following rules.
- null data is always equal.
- boolean data is ordered with false before true.
- int, long, float and double data is ordered by ascending numeric value.
- bytes and fixed data are compared lexicographically by unsigned 8-bit values.
- string data is compared lexicographically by Unicode code point. Note that since UTF-8 is used as the binary encoding for strings, sorting of bytes and string binary data is identical.
- array data is compared lexicographically by element.
- enum data is ordered by the symbol’s position in the enum schema. For example, an enum whose symbols are
["z", "a"] would sort “z” values before “a” values.
- union data is first ordered by the branch within the union, and, within that, by the type of the branch. For example, an
["int", "string"] union would order all int values before all string values, with the ints and strings themselves ordered as defined above.
- record data is ordered lexicographically by field. If a field specifies that its order is:
- “ascending”, then the order of its values is unaltered.
- “descending”, then the order of its values is reversed.
- “ignore”, then its values are ignored when sorting.
- map data may not be compared. It is an error to attempt to compare data containing maps unless those maps are in an
"order":"ignore" record field.
Object Container Files
Avro includes a simple object container file format. A file has a schema, and all objects stored in the file must be written according to that schema, using binary encoding. Objects are stored in blocks that may be compressed. Syncronization markers are used between blocks to permit efficient splitting of files for MapReduce processing.
Files may include arbitrary user-specified metadata.
A file consists of:
- A file header, followed by
- one or more file data blocks.
A file header consists of:
- Four bytes, ASCII ‘O’, ‘b’, ‘j’, followed by 1.
- file metadata, including the schema.
- The 16-byte, randomly-generated sync marker for this file.
File metadata consists of:
A long indicating the number of metadata key/value pairs.
For each pair, a string key and bytes value.
All metadata properties that start with “avro.” are reserved. The following file metadata properties are currently used:
- avro.schema contains the schema of objects stored in the file, as JSON data (required).
- avro.codec the name of the compression codec used to compress blocks, as a string. Implementations are required to support the following codecs: “null” and “deflate”. If codec is absent, it is assumed to be “null”. The codecs are described with more detail below.
A file header is thus described by the following schema:
{"type": "record", "name": "org.apache.avro.file.Header",
"fields" : [
{"name": "magic", "type": {"type": "fixed", "name": "Magic", "size": 4}},
{"name": "meta", "type": {"type": "map", "values": "bytes"}},
{"name": "sync", "type": {"type": "fixed", "name": "Sync", "size": 16}}
]
}
A file data block consists of:
- A long indicating the count of objects in this block.
- A long indicating the size in bytes of the serialized objects in the current block, after any codec is applied
- The serialized objects. If a codec is specified, this is compressed by that codec.
- The file’s 16-byte sync marker.
Thus, each block’s binary data can be efficiently extracted or skipped without deserializing the contents. The combination of block size, object counts, and sync markers enable detection of corrupt blocks and help ensure data integrity.
Required Codecs
null
The “null” codec simply passes through data uncompressed.
deflate
The “deflate” codec writes the data block using the deflate algorithm as specified in RFC 1951, and typically implemented using the zlib library. Note that this format (unlike the “zlib format” in RFC 1950) does not have a checksum.
Optional Codecs
snappy
The “snappy” codec uses Google’s Snappy compression library. Each compressed block is followed by the 4-byte, big-endian CRC32 checksum.
Protocol Declaration
Avro protocols describe RPC interfaces. Like schemas, they are defined with JSON text.
A protocol is a JSON object with the following attributes:
- protocol, a string, the name of the protocol (required);
- namespace, an optional string that qualifies the name;
- doc, an optional string describing this protocol;
- types, an optional list of definitions of named types (records, enums, fixed and errors). An error definition is just like a record definition except it uses “error” instead of “record”. Note that forward references to named types are not permitted.
- messages, an optional JSON object whose keys are message names and whose values are objects whose attributes are described below. No two messages may have the same name.
The name and namespace qualification rules defined for schema objects apply to protocols as well.
Messages
A message has attributes:
- a doc, an optional description of the message,
- a request, a list of named, typed parameter schemas (this has the same form as the fields of a record declaration);
- a response schema;
- an optional union of declared error schemas. The effective union has “string” prepended to the declared union, to permit transmission of undeclared “system” errors. For example, if the declared error union is
["AccessError"], then the effective union is ["string", "AccessError"]. When no errors are declared, the effective error union is ["string"]. Errors are serialized using the effective union; however, a protocol’s JSON declaration contains only the declared union.
- an optional one-way boolean parameter.
A request parameter list is processed equivalently to an anonymous record. Since record field lists may vary between reader and writer, request parameters may also differ between the caller and responder, and such differences are resolved in the same manner as record field differences.
The one-way parameter may only be true when the response type is "null" and no errors are listed.
Sample Protocol
For example, one may define a simple HelloWorld protocol with:
{
"namespace": "com.acme",
"protocol": "HelloWorld",
"doc": "Protocol Greetings",
"types": [
{"name": "Greeting", "type": "record", "fields": [
{"name": "message", "type": "string"}]},
{"name": "Curse", "type": "error", "fields": [
{"name": "message", "type": "string"}]}
],
"messages": {
"hello": {
"doc": "Say hello.",
"request": [{"name": "greeting", "type": "Greeting" }],
"response": "Greeting",
"errors": ["Curse"]
}
}
}
Message Transport
Messages may be transmitted via different transport mechanisms.
To the transport, a message is an opaque byte sequence.
A transport is a system that supports:
- transmission of request messages
- receipt of corresponding response messages
Servers may send a response message back to the client corresponding to a request message. The mechanism of correspondance is transport-specific. For example, in HTTP it is implicit, since HTTP directly supports requests and responses. But a transport that multiplexes many client threads over a single socket would need to tag messages with unique identifiers.
Transports may be either stateless or stateful. In a stateless transport, messaging assumes no established connection state, while stateful transports establish connections that may be used for multiple messages. This distinction is discussed further in the handshake section below.
HTTP as Transport
When HTTP is used as a transport, each Avro message exchange is an HTTP request/response pair. All messages of an Avro protocol should share a single URL at an HTTP server. Other protocols may also use that URL. Both normal and error Avro response messages should use the 200 (OK) response code. The chunked encoding may be used for requests and responses, but, regardless the Avro request and response are the entire content of an HTTP request and response. The HTTP Content-Type of requests and responses should be specified as “avro/binary”. Requests should be made using the POST method.
HTTP is used by Avro as a stateless transport.
Message Framing
Avro messages are framed as a list of buffers.
Framing is a layer between messages and the transport. It exists to optimize certain operations.
The format of framed message data is:
- a series of buffers, where each buffer consists of:
- a four-byte, big-endian buffer length, followed by
- that many bytes of buffer data.
- A message is always terminated by a zero-lenghted buffer.
Framing is transparent to request and response message formats (described below). Any message may be presented as a single or multiple buffers.
Framing can permit readers to more efficiently get different buffers from different sources and for writers to more efficiently store different buffers to different destinations. In particular, it can reduce the number of times large binary objects are copied. For example, if an RPC parameter consists of a megabyte of file data, that data can be copied directly to a socket from a file descriptor, and, on the other end, it could be written directly to a file descriptor, never entering user space.
A simple, recommended, framing policy is for writers to create a new segment whenever a single binary object is written that is larger than a normal output buffer. Small objects are then appended in buffers, while larger objects are written as their own buffers. When a reader then tries to read a large object the runtime can hand it an entire buffer directly, without having to copy it.
Handshake
The purpose of the handshake is to ensure that the client and the server have each other’s protocol definition, so that the client can correctly deserialize responses, and the server can correctly deserialize requests. Both clients and servers should maintain a cache of recently seen protocols, so that, in most cases, a handshake will be completed without extra round-trip network exchanges or the transmission of full protocol text.
RPC requests and responses may not be processed until a handshake has been completed. With a stateless transport, all requests and responses are prefixed by handshakes. With a stateful transport, handshakes are only attached to requests and responses until a successful handshake response has been returned over a connection. After this, request and response payloads are sent without handshakes for the lifetime of that connection.
The handshake process uses the following record schemas:
{
"type": "record",
"name": "HandshakeRequest", "namespace":"org.apache.avro.ipc",
"fields": [
{"name": "clientHash",
"type": {"type": "fixed", "name": "MD5", "size": 16}},
{"name": "clientProtocol", "type": ["null", "string"]},
{"name": "serverHash", "type": "MD5"},
{"name": "meta", "type": ["null", {"type": "map", "values": "bytes"}]}
]
}
{
"type": "record",
"name": "HandshakeResponse", "namespace": "org.apache.avro.ipc",
"fields": [
{"name": "match",
"type": {"type": "enum", "name": "HandshakeMatch",
"symbols": ["BOTH", "CLIENT", "NONE"]}},
{"name": "serverProtocol",
"type": ["null", "string"]},
{"name": "serverHash",
"type": ["null", {"type": "fixed", "name": "MD5", "size": 16}]},
{"name": "meta",
"type": ["null", {"type": "map", "values": "bytes"}]}
]
}
- A client first prefixes each request with a
HandshakeRequest containing just the hash of its protocol and of the server’s protocol (clientHash!=null, clientProtocol=null, serverHash!=null), where the hashes are 128-bit MD5 hashes of the JSON protocol text. If a client has never connected to a given server, it sends its hash as a guess of the server’s hash, otherwise it sends the hash that it previously obtained from this server.
The server responds with a HandshakeResponse containing one of:
-
match=BOTH, serverProtocol=null, serverHash=null if the client sent the valid hash of the server’s protocol and the server knows what protocol corresponds to the client’s hash. In this case, the request is complete and the response data immediately follows the HandshakeResponse.
-
match=CLIENT, serverProtocol!=null, serverHash!=null if the server has previously seen the client’s protocol, but the client sent an incorrect hash of the server’s protocol. The request is complete and the response data immediately follows the HandshakeResponse. The client must use the returned protocol to process the response and should also cache that protocol and its hash for future interactions with this server.
-
match=NONE if the server has not previously seen the client’s protocol. The serverHash and serverProtocol may also be non-null if the server’s protocol hash was incorrect.
In this case the client must then re-submit its request with its protocol text (clientHash!=null, clientProtocol!=null, serverHash!=null) and the server should respond with a successful match (match=BOTH, serverProtocol=null, serverHash=null) as above.
The meta field is reserved for future handshake enhancements.
A call consists of a request message paired with its resulting response or error message. Requests and responses contain extensible metadata, and both kinds of messages are framed as described above.
The format of a call request is:
- request metadata, a map with values of type bytes
- the message name, an Avro string, followed by
- the message parameters. Parameters are serialized according to the message’s request declaration.
When a message is declared one-way and a stateful connection has been established by a successful handshake response, no response data is sent. Otherwise the format of the call response is:
- response metadata, a map with values of type bytes
- a one-byte error flag boolean, followed by either:
- if the error flag is false, the message response, serialized per the message’s response schema.
- if the error flag is true, the error, serialized per the message’s effective error union schema.
Schema Resolution
A reader of Avro data, whether from an RPC or a file, can always parse that data because its schema is provided. But that schema may not be exactly the schema that was expected. For example, if the data was written with a different version of the software than it is read, then records may have had fields added or removed. This section specifies how such schema differences should be resolved.
We refer to the schema used to write the data as the writer’s schema, and the schema that the application expects the reader’s schema. Differences between these should be resolved as follows:
- It is an error if the two schemas do not match.
To match, one of the following must hold:
-
both schemas are arrays whose item types match
-
both schemas are maps whose value types match
-
both schemas are enums whose names match
-
both schemas are fixed whose sizes and names match
-
both schemas are records with the same name
-
either schema is a union
-
both schemas have same primitive type
-
the writer’s schema may be promoted to the reader’s as follows:
- int is promotable to long, float, or double
- long is promotable to float or double
- float is promotable to double
- string is promotable to bytes
- bytes is promotable to string
-
if both are records:
- the ordering of fields may be different: fields are matched by name.
- schemas for fields with the same name in both records are resolved recursively.
- if the writer’s record contains a field with a name not present in the reader’s record, the writer’s value for that field is ignored.
- if the reader’s record schema has a field that contains a default value, and writer’s schema does not have a field with the same name, then the reader should use the default value from its field.
- if the reader’s record schema has a field with no default value, and writer’s schema does not have a field with the same name, an error is signalled.
-
if both are enums:
if the writer’s symbol is not present in the reader’s enum, then an error is signalled.
This resolution algorithm is applied recursively to the reader’s and writer’s array item schemas.
This resolution algorithm is applied recursively to the reader’s and writer’s value schemas.
The first schema in the reader’s union that matches the selected writer’s union schema is recursively resolved against it. if none match, an error is signalled.
- if reader’s is a union, but writer’s is not:
The first schema in the reader’s union that matches the writer’s schema is recursively resolved against it. If none match, an error is signalled.
- if writer’s is a union, but reader’s is not:
If the reader’s schema matches the selected writer’s schema, it is recursively resolved against it. If they do not match, an error is signalled.
A schema’s “doc” fields are ignored for the purposes of schema resolution. Hence, the “doc” portion of a schema may be dropped at serialization.
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
24.2 - IDL Language
Introduction
This document defines Avro IDL, a higher-level language for authoring Avro schemata. Before reading this document, you should have familiarity with the concepts of schemata and protocols, as well as the various primitive and complex types available in Avro.
Overview
Purpose
The aim of the Avro IDL language is to enable developers to author schemata in a way that feels more similar to common programming languages like Java, C++, or Python. Additionally, the Avro IDL language may feel more familiar for those users who have previously used the interface description languages (IDLs) in other frameworks like Thrift, Protocol Buffers, or CORBA.
Usage
Each Avro IDL file defines a single Avro Protocol, and thus generates as its output a JSON-format Avro Protocol file with extension .avpr.
To convert a .avdl file into a .avpr file, it may be processed by the idl tool. For example:
$ java -jar avroj-tools-1.4.0.jar idl src/test/idl/input/namespaces.avdl /tmp/namespaces.avpr
$ head /tmp/namespaces.avpr
{
"protocol" : "TestNamespace",
"namespace" : "avro.test.protocol",
The idl tool can also process input to and from stdin and stdout. See idl --help for full usage information.
Defining a Protocol in Avro IDL
An Avro IDL file consists of exactly one protocol definition. The minimal protocol is defined by the following code:
This is equivalent to (and generates) the following JSON protocol definition:
{
"protocol" : "MyProtocol",
"types" : [ ],
"messages" : {
}
}
The namespace of the protocol may be changed using the @namespace annotation:
@namespace("mynamespace")
protocol MyProtocol {
}
This notation is used throughout Avro IDL as a way of specifying properties for the annotated element, as will be described later in this document.
Protocols in Avro IDL can contain the following items:
- Imports of external protocol and schema files.
- Definitions of named schemata, including records, errors, enums, and fixeds.
- Definitions of RPC messages
Imports
Files may be imported in one of three formats:
-
An IDL file may be imported with a statement like:
import idl "foo.avdl";
-
A JSON protocol file may be imported with a statement like:
import protocol "foo.avpr";
-
A JSON schema file may be imported with a statement like:
import schema "foo.avsc";
Messages and types in the imported file are added to this file’s protocol.
Imported file names are resolved relative to the current IDL file.
Defining an Enumeration
Enums are defined in Avro IDL using a syntax similar to C or Java:
enum Suit {
SPADES, DIAMONDS, CLUBS, HEARTS
}
Note that, unlike the JSON format, anonymous enums cannot be defined.
Defining a Fixed Length Field
Fixed fields are defined using the following syntax:
fixed MD5(16);
This example defines a fixed-length type called MD5 which contains 16 bytes.
Defining Records and Errors
Records are defined in Avro IDL using a syntax similar to a struct definition in C:
record Employee {
string name;
boolean active = true;
long salary;
}
The above example defines a record with the name “Employee” with three fields.
To define an error, simply use the keyword error instead of record. For example:
error Kaboom {
string explanation;
int result_code = -1;
}
Each field in a record or error consists of a type and a name, optional property annotations and an optional default value.
A type reference in Avro IDL must be one of:
- A primitive type
- A logical type
- A named schema defined prior to this usage in the same Protocol
- A complex type (array, map, or union)
Primitive Types
The primitive types supported by Avro IDL are the same as those supported by Avro’s JSON format. This list includes int, long, string, boolean, float, double, null, and bytes.
References to Named Schemata
If a named schema has already been defined in the same Avro IDL file, it may be referenced by name as if it were a primitive type:
record Card {
Suit suit; // refers to the enum Card defined above
int number;
}
Default Values
Default values for fields may be optionally specified by using an equals sign after the field name followed by a JSON expression indicating the default value. This JSON is interpreted as described in the spec.
Complex Types
Arrays
Array types are written in a manner that will seem familiar to C++ or Java programmers. An array of any type t is denoted array<t>. For example, an array of strings is denoted array<string>, and a multidimensional array of Foo records would be array<array<Foo>>.
Maps
Map types are written similarly to array types. An array that contains values of type t is written map<t>. As in the JSON schema format, all maps contain string-type keys.
Unions
Union types are denoted as union { typeA, typeB, typeC, … }. For example, this record contains a string field that is optional (unioned with null):
record RecordWithUnion {
union { null, string } optionalString;
}
Note that the same restrictions apply to Avro IDL unions as apply to unions defined in the JSON format; namely, a record may not contain multiple elements of the same type.
Defining RPC Messages
The syntax to define an RPC message within a Avro IDL protocol is similar to the syntax for a method declaration within a C header file or a Java interface. To define an RPC message add which takes two arguments named foo and bar, returning an int, simply include the following definition within the protocol:
int add(int foo, int bar = 0);
Message arguments, like record fields, may specify default values.
To define a message with no response, you may use the alias void, equivalent to the Avro null type:
void logMessage(string message);
If you have previously defined an error type within the same protocol, you may declare that a message can throw this error using the syntax:
void goKaboom() throws Kaboom;
To define a one-way message, use the keyword oneway after the parameter list, for example:
void fireAndForget(string message) oneway;
Other Language Features
All Java-style comments are supported within a Avro IDL file. Any text following // on a line is ignored, as is any text between /* and */, possibly spanning multiple lines.
Comments that begin with /** are used as the documentation string for the type or field definition that follows the comment.
Escaping Identifiers
Occasionally, one will need to use a reserved language keyword as an identifier. In order to do so, backticks (`) may be used to escape the identifier. For example, to define a message with the literal name error, you may write:
This syntax is allowed anywhere an identifier is expected.
Annotations for Ordering and Namespaces
Java-style annotations may be used to add additional properties to types and fields throughout Avro IDL.
For example, to specify the sort order of a field within a record, one may use the @order annotation before the field name as follows:
record MyRecord {
string @order("ascending") myAscendingSortField;
string @order("descending") myDescendingField;
string @order("ignore") myIgnoredField;
}
A field’s type may also be preceded by annotations, e.g.:
record MyRecord {
@java-class("java.util.ArrayList") array<string> myStrings;
}
Similarly, a @namespace annotation may be used to modify the namespace when defining a named schema. For example:
@namespace("org.apache.avro.firstNamespace")
protocol MyProto {
@namespace("org.apache.avro.someOtherNamespace")
record Foo {}
record Bar {}
}
will define a protocol in the firstNamespace namespace. The record Foo will be defined in someOtherNamespace and Bar will be defined in firstNamespace as it inherits its default from its container.
Type and field aliases are specified with the @aliases annotation as follows:
@aliases(["org.old.OldRecord", "org.ancient.AncientRecord"])
record MyRecord {
string @aliases(["oldField", "ancientField"]) myNewField;
}
Complete Example
The following is a complete example of a Avro IDL file that shows most of the above features:
/**
* An example protocol in Avro IDL
*/
@namespace("org.apache.avro.test")
protocol Simple {
@aliases(["org.foo.KindOf"])
enum Kind {
FOO,
BAR, // the bar enum value
BAZ
}
fixed MD5(16);
record TestRecord {
@order("ignore")
string name;
@order("descending")
Kind kind;
MD5 hash;
union { MD5, null} @aliases(["hash"]) nullableHash;
array<long> arrayOfLongs;
}
error TestError {
string message;
}
string hello(string greeting);
TestRecord echo(TestRecord `record`);
int add(int arg1, int arg2);
bytes echoBytes(bytes data);
void `error`() throws TestError;
void ping() oneway;
}
Additional examples may be found in the Avro source tree under the src/test/idl/input directory.
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
24.3 - SASL profile
Introduction
SASL (RFC 2222) provides a framework for authentication and security of network protocols. Each protocol that uses SASL is meant to define a SASL profile. This document provides a SASL profile for connection-based Avro RPC.
Overview
SASL negotiation proceeds as a series of message interactions over a connection between a client and server using a selected SASL mechanism. The client starts this negotiation by sending its chosen mechanism name with an initial (possibly empty) message. Negotiation proceeds with the exchange of messages until either side indicates success or failure. The content of the messages is mechanism-specific. If the negotiation succeeds, then the session can proceed over the connection, otherwise it must be abandoned.
Some mechanisms continue to process session data after negotiation (e.g., encrypting it), while some specify that further session data is transmitted unmodified.
Negotiation
Commands
Avro SASL negotiation uses four one-byte commands.
- 0: START Used in a client’s initial message.
- 1: CONTINUE Used while negotiation is ongoing.
- 2: FAIL Terminates negotiation unsuccessfully.
- 3: COMPLETE Terminates negotiation successfully.
The format of a START message is:
| 0 | 4-byte mechanism name length | mechanism name | 4-byte payload length | payload data |
The format of a CONTINUE message is:
| 1 | 4-byte payload length | payload data |
The format of a FAIL message is:
| 2 | 4-byte message length | UTF-8 message |
The format of a COMPLETE message is:
| 3 | 4-byte payload length | payload data |
Process
Negotiation is initiated by a client sending a START command containing the client’s chosen mechanism name and any mechanism-specific payload data.
The server and client then interchange some number (possibly zero) of CONTINUE messages. Each message contains payload data that is processed by the security mechanism to generate the next message.
Once either the client or server send a FAIL message then negotiation has failed. UTF-8-encoded text is included in the failure message. Once either a FAIL message has been sent or received, or any other error occurs in the negotiation, further communication on this connection must cease.
Once either the client or server send a COMPLETE message then negotiation has completed successfully. Session data may now be transmitted over the connection until it is closed by either side.
Session Data
If no SASL QOP (quality of protection) is negotiated, then all subsequent writes to/reads over this connection are written/read unmodified. In particular, messages use Avro framing, and are of the form:
| 4-byte frame length | frame data | ... | 4 zero bytes |
If a SASL QOP is negotiated, then it must be used by the connection for all subsequent messages. This is done by wrapping each non-empty frame written using the security mechanism and unwrapping each non-empty frame read. The length written in each non-empty frame is the length of the wrapped data. Complete frames must be passed to the security mechanism for unwrapping. Unwrapped data is then passed to the application as the content of the frame.
If at any point processing fails due to wrapping, unwrapping or framing errors, then all further communication on this connection must cease.
Anonymous Mechanism
The SASL anonymous mechanism (RFC 2245) is quite simple to implement. In particular, an initial anonymous request may be prefixed by the following static sequence:
| 0 | 0009 | ANONYMOUS | 0000 |
If a server uses the anonymous mechanism, it should check that the mechanism name in the start message prefixing the first request received is ‘ANONYMOUS’, then simply prefix its initial response with a COMPLETE message of:
| 3 | 0000 |
If an anonymous server recieves some other mechanism name, then it may respond with a FAIL message as simple as:
| 2 | 0000 |
Note that the anonymous mechanism need add no additional round-trip messages between client and server. The START message can be piggybacked on the initial request and the COMPLETE or FAIL message can be piggybacked on the initial response.
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
24.4 - Wiki
The Wiki page can be found here.
24.5 - FAQ
What is Avro?
Avro is a data serialization system.
Avro provides:
- Rich data structures.
- A compact, fast, binary data format.
- A container file, to store persistent data.
- Remote procedure call (RPC).
- Simple integration with dynamic languages. Code generation is not required to read or write data files nor to use or implement RPC protocols. Code generation as an optional optimization, only worth implementing for statically typed languages.
How can I get started quickly with Avro?
Check out the Quick Start Guide.
How do I statically compile a schema or protocol into generated code?
In Java
- Add the avro jar, the jackson-mapper-asl.jar and jackson-core-asl.jar to your CLASSPATH.
- Run java org.apache.avro.specific.SpecificCompiler .
This appears to be out of date, the SpecificCompiler requires two arguments, presumably an input and and output file, but it isn’t clear that this does.
Or use the Schema or Protocol Ant tasks. Avro’s build.xml provides examples of how these are used.
Lastly, you can also use the “avro-tools” jar which ships with an Avro release. Just use the “compile (schema|protocol)” command.
How are Strings represented in Java?
They use org.apache.avro.util.Utf8, not java.lang.String.
More generally, how do Avro types map to Java types?
The mappings are documented in the package javadoc for generic, specific and reflect API.
From Doug Cutting:
HDFS splits files into blocks, and mapreduce runs a map task for each block. When the task starts, it needs to be able to seek into the file to the start of the block process through the block’s end. If the file were, e.g., a gzip file, this would not be possible, since gzip files must be decompressed from the start. One cannot seek into the middle of a gzip file and start decompressing. So Hadoop’s SequenceFile places a marker periodically (~64k) in the file at record and compression boundaries, where processing can be sensibly started. Then, when a map task starts processing an HDFS block, it finds the first marker after the block’s start and continues through the first marker in the next block of the file. This requires a bit of non-local access (~0.1%). Avro’s data file uses the same method as SequenceFile.
Why isn’t every value in Avro nullable?
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 of every value stored and
corresponding computational costs to create this bit on write and
interpret it on read. These costs are wasted when values may not in
fact be null, as is the case in many datasets. In Avro such costs are
only paid when values may actually be null.
Also, allowing values to be null is a well-known source of errors. In
Avro, a value declared as non-null will always be non-null and programs
need not test for null values when processing it nor will they ever fail
for lack of such tests.
Tony Hoare calls his invention of null references his “Billion Dollar
Mistake”.
http://qconlondon.com/london-2009/presentation/Null+References:+The+Billion+Dollar+Mistake
Also note that in some programming languages not all values are permitted to be null. For example, in Java, values of type boolean, byte, short, char, int, float, long, and double may not be null.
How can I serialize directly to/from a byte array?
As pointed out in the specification, Avro data should always be stored with its schema. The Avro provided classes DataFileWriter, DataFileReader, and DataFileStream all ensure this by serializing the Schema in a container header. In some special cases, such as when implementing a new storage system or writing unit tests, you may need to write and read directly with the bare Avro serialized values. The following examples use code generated for Java from an Avro Schema, specifically the User example from the Quickstart guide.
Serializing to a byte array
This example takes a User object and returns a newly allocated byte array with the Avro serialization of that user.
ByteArrayOutputStream out = new ByteArrayOutputStream();
BinaryEncoder encoder = EncoderFactory.get().binaryEncoder(out, null);
DatumWriter<User> writer = new SpecificDatumWriter<User>(User.getClassSchema());
writer.write(user, encoder);
encoder.flush();
out.close();
byte[] serializedBytes = out.toByteArray();
Deserializing from a byte array
This example takes a byte array containing the Avro serialization of a user and returns a User object.SpecificDatumReader reader = new SpecificDatumReader(User.getClassSchema());
Decoder decoder = DecoderFactory.get().binaryDecoder(bytes, null);
User user = reader.read(null, decoder);
Note: If you are serializing or deserializing in a loop or as a method, you should be reusing objects, readers and/or writers. Check the JavaDoc to see how to reuse the objects.
25 - Apache Avro™ 1.5.3 Documentation
Introduction
Apache Avro™ is a data serialization system.
Avro provides:
- Rich data structures.
- A compact, fast, binary data format.
- A container file, to store persistent data.
- Remote procedure call (RPC).
- Simple integration with dynamic languages. Code generation is not required to read or write data files nor to use or implement RPC protocols. Code generation as an optional optimization, only worth implementing for statically typed languages.
Schemas
Avro relies on schemas. When Avro data is read, the schema used when writing it is always present. This permits each datum to be written with no per-value overheads, making serialization both fast and small. This also facilitates use with dynamic, scripting languages, since data, together with its schema, is fully self-describing.
When Avro data is stored in a file, its schema is stored with it, so that files may be processed later by any program. If the program reading the data expects a different schema this can be easily resolved, since both schemas are present.
When Avro is used in RPC, the client and server exchange schemas in the connection handshake. (This can be optimized so that, for most calls, no schemas are actually transmitted.) Since both client and server both have the other’s full schema, correspondence between same named fields, missing fields, extra fields, etc. can all be easily resolved.
Avro schemas are defined with JSON . This facilitates implementation in languages that already have JSON libraries.
Comparison with other systems
Avro provides functionality similar to systems such as Thrift, Protocol Buffers, etc. Avro differs from these systems in the following fundamental aspects.
- Dynamic typing: Avro does not require that code be generated. Data is always accompanied by a schema that permits full processing of that data without code generation, static datatypes, etc. This facilitates construction of generic data-processing systems and languages.
- Untagged data: Since the schema is present when data is read, considerably less type information need be encoded with data, resulting in smaller serialization size.
- No manually-assigned field IDs: When a schema changes, both the old and new schema are always present when processing data, so differences may be resolved symbolically, using field names.
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
25.1 - Specification
Introduction
This document defines Apache Avro. It is intended to be the authoritative specification. Implementations of Avro must adhere to this document.
Schema Declaration
A Schema is represented in JSON by one of:
- A JSON string, naming a defined type.
- A JSON object, of the form:
{"type": "typeName" ...attributes...}
where typeName is either a primitive or derived type name, as defined below. Attributes not defined in this document are permitted as metadata, but must not affect the format of serialized data.
- A JSON array, representing a union of embedded types.
Primitive Types
The set of primitive type names is:
- null: no value
- boolean: a binary value
- int: 32-bit signed integer
- long: 64-bit signed integer
- float: single precision (32-bit) IEEE 754 floating-point number
- double: double precision (64-bit) IEEE 754 floating-point number
- bytes: sequence of 8-bit unsigned bytes
- string: unicode character sequence
Primitive types have no specified attributes.
Primitive type names are also defined type names. Thus, for example, the schema “string” is equivalent to:
Complex Types
Avro supports six kinds of complex types: records, enums, arrays, maps, unions and fixed.
Records
Records use the type name “record” and support three attributes:
- name: a JSON string providing the name of the record (required).
- namespace, a JSON string that qualifies the name;
- doc: a JSON string providing documentation to the user of this schema (optional).
- aliases: a JSON array of strings, providing alternate names for this record (optional).
- fields: a JSON array, listing fields (required). Each field is a JSON object with the following attributes:
- name: a JSON string providing the name of the field (required), and
- doc: a JSON string describing this field for users (optional).
- type: A JSON object defining a schema, or a JSON string naming a record definition (required).
- default: A default value for this field, used when reading instances that lack this field (optional). 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. Default values for bytes and fixed fields are JSON strings, where Unicode code points 0-255 are mapped to unsigned 8-bit byte values 0-255.
field default values
| avro type |
json type |
example |
| null |
null |
null |
| boolean |
boolean |
true |
| int,long |
integer |
1 |
| float,double |
number |
1.1 |
| bytes |
string |
"\u00FF" |
| string |
string |
"foo" |
| record |
object |
{"a": 1} |
| enum |
string |
"FOO" |
| array |
array |
[1] |
| map |
object |
{"a": 1} |
| fixed |
string |
"\u00ff" |
- order: specifies how this field impacts sort ordering of this record (optional). Valid values are “ascending” (the default), “descending”, or “ignore”. For more details on how this is used, see the sort order section below.
- aliases: a JSON array of strings, providing alternate names for this field (optional).
For example, a linked-list of 64-bit values may be defined with:
{
"type": "record",
"name": "LongList",
"aliases": ["LinkedLongs"], // old name for this
"fields" : [
{"name": "value", "type": "long"}, // each element has a long
{"name": "next", "type": ["LongList", "null"]} // optional next element
]
}
Enums
Enums use the type name “enum” and support the following attributes:
- name: a JSON string providing the name of the enum (required).
- namespace, a JSON string that qualifies the name ;
- aliases: a JSON array of strings, providing alternate names for this enum (optional).
- doc: a JSON string providing documentation to the user of this schema (optional).
- symbols: a JSON array, listing symbols, as JSON strings (required). All symbols in an enum must be unique; duplicates are prohibited.
For example, playing card suits might be defined with:
{
"type": "enum",
"name": "Suit",
"symbols" : ["SPADES", "HEARTS", "DIAMONDS", "CLUBS"]
}
Arrays
Arrays use the type name “array” and support a single attribute:
- items: the schema of the array’s items.
For example, an array of strings is declared with:
{
"type": "array",
"items" : "string"
}
Maps
Maps use the type name “map” and support one attribute:
- values: the schema of the map’s values.
Map keys are assumed to be strings.
For example, a map from string to long is declared with:
{
"type": "map",
"values" : "long"
}
Unions
Unions, as mentioned above, are represented using JSON arrays. For example, ["string", "null"] declares a schema which may be either a string or null.
Unions may not contain more than one schema with the same type, except for the named types record, fixed and enum. For example, unions containing two array types or two map types are not permitted, but two types with different names are permitted. (Names permit efficient resolution when reading and writing unions.)
Unions may not immediately contain other unions.
Fixed
Fixed uses the type name “fixed” and supports two attributes:
- name: a string naming this fixed (required).
- namespace, a string that qualifies the name;
- aliases: a JSON array of strings, providing alternate names for this enum (optional).
- size: an integer, specifying the number of bytes per value (required).
For example, 16-byte quantity may be declared with:
{"type": "fixed", "size": 16, "name": "md5"}
Names
Record, enums and fixed are named types. Each has a fullname that is composed of two parts; a name and a namespace. Equality of names is defined on the fullname.
The name portion of a fullname, and record field names must:
- start with [A-Za-z_]
- subsequently contain only [A-Za-z0-9_]
A namespace is a dot-separated sequence of such names.
In record, enum and fixed definitions, the fullname is determined in one of the following ways:
A name and namespace are both specified. For example, one might use “name”: “X”, “namespace”: “org.foo” to indicate the fullname org.foo.X.
-
A fullname is specified. If the name specified contains a dot, then it is assumed to be a fullname, and any namespace also specified is ignored. For example, use “name”: “org.foo.X” to indicate the fullname org.foo.X.
-
A name only is specified, i.e., a name that contains no dots. In this case the namespace is taken from the most tightly enclosing schema or protocol. For example, if “name”: “X” is specified, and this occurs within a field of the record definition of org.foo.Y, then the fullname is org.foo.X.
References to previously defined names are as in the latter two cases above: if they contain a dot they are a fullname, if they do not contain a dot, the namespace is the namespace of the enclosing definition.
Primitive type names have no namespace and their names may not be defined in any namespace. A schema may only contain multiple definitions of a fullname if the definitions are equivalent.
Aliases
Named types and fields may have aliases. An implementation may optionally use aliases to map a writer’s schema to the reader’s. This facilitates both schema evolution as well as processing disparate datasets.
Aliases function by re-writing the writer’s schema using aliases from the reader’s schema. For example, if the writer’s schema was named “Foo” and the reader’s schema is named “Bar” and has an alias of “Foo”, then the implementation would act as though “Foo” were named “Bar” when reading. Similarly, if data was written as a record with a field named “x” and is read as a record with a field named “y” with alias “x”, then the implementation would act as though “x” were named “y” when reading.
A type alias may be specified either as a fully namespace-qualified, or relative to the namespace of the name it is an alias for. For example, if a type named “a.b” has aliases of “c” and “x.y”, then the fully qualified names of its aliases are “a.c” and “x.y”.
Data Serialization
Avro data is always serialized with its schema. Files that store Avro data should always also include the schema for that data in the same file. Avro-based remote procedure call (RPC) systems must also guarantee that remote recipients of data have a copy of the schema used to write that data.
Because the schema used to write data is always available when the data is read, Avro data itself is not tagged with type information. The schema is required to parse data.
In general, both serialization and deserialization proceed as a depth-first, left-to-right traversal of the schema, serializing primitive types as they are encountered.
Encodings
Avro specifies two serialization encodings: binary and JSON. Most applications will use the binary encoding, as it is smaller and faster. But, for debugging and web-based applications, the JSON encoding may sometimes be appropriate.
Binary Encoding
Primitive Types
Primitive types are encoded in binary as follows:
- null is written as zero bytes.
- a boolean is written as a single byte whose value is either 0 (false) or 1 (true).
- int and long values are written using variable-length zig-zag coding. Some examples:
| value |
hex |
| 0 |
00 |
| -1 |
01 |
| 1 |
02 |
| -2 |
03 |
| 2 |
04 |
| … |
|
| -64 |
7f |
| 64 |
80 01 |
| … |
|
- a float is written as 4 bytes. The float is converted into a 32-bit integer using a method equivalent to Java’s floatToIntBits and then encoded in little-endian format.
- a double is written as 8 bytes. The double is converted into a 64-bit integer using a method equivalent to Java’s doubleToLongBits and then encoded in little-endian format.
- bytes are encoded as a long followed by that many bytes of data.
- a string is encoded as a long followed by that many bytes of UTF-8 encoded character data.
For example, the three-character string “foo” would be encoded as the long value 3 (encoded as hex 06) followed by the UTF-8 encoding of ‘f’, ‘o’, and ‘o’ (the hex bytes 66 6f 6f):
06 66 6f 6f
Complex Types
Complex types are encoded in binary as follows:
Records
A record is encoded by encoding the values of its fields in the order that they are declared. In other words, a record is encoded as just the concatenation of the encodings of its fields. Field values are encoded per their schema.
For example, the record schema
{
"type": "record",
"name": "test",
"fields" : [
{"name": "a", "type": "long"},
{"name": "b", "type": "string"}
]
}
An instance of this record whose a field has value 27 (encoded as hex 36) and whose b field has value “foo” (encoded as hex bytes 06 66 6f 6f), would be encoded simply as the concatenation of these, namely the hex byte sequence:
36 06 66 6f 6f
Enums
An enum is encoded by a int, representing the zero-based position of the symbol in the schema.
For example, consider the enum:
{"type": "enum", "name": "Foo", "symbols": ["A", "B", "C", "D"] }
This would be encoded by an int between zero and three, with zero indicating “A”, and 3 indicating “D”.
Arrays
Arrays are encoded as a series of blocks. Each block consists of a long count value, followed by that many array items. A block with count zero indicates the end of the array. Each item is encoded per the array’s item schema.
If a block’s count is negative, its absolute value is used, and the count is followed immediately by a long block size indicating the number of bytes in the block. This block size permits fast skipping through data, e.g., when projecting a record to a subset of its fields.
For example, the array schema
{"type": "array", "items": "long"}
an array containing the items 3 and 27 could be encoded as the long value 2 (encoded as hex 04) followed by long values 3 and 27 (encoded as hex 06 36) terminated by zero:
04 06 36 00
The blocked representation permits one to read and write arrays larger than can be buffered in memory, since one can start writing items without knowing the full length of the array.
Maps
Maps are encoded as a series of blocks. Each block consists of a long count value, followed by that many key/value pairs. A block with count zero indicates the end of the map. Each item is encoded per the map’s value schema.
If a block’s count is negative, its absolute value is used, and the count is followed immediately by a long block size indicating the number of bytes in the block. This block size permits fast skipping through data, e.g., when projecting a record to a subset of its fields.
The blocked representation permits one to read and write maps larger than can be buffered in memory, since one can start writing items without knowing the full length of the map.
Unions
A union is encoded by first writing a long value indicating the zero-based position within the union of the schema of its value. The value is then encoded per the indicated schema within the union.
For example, the union schema ["string","null"] would encode:
-
null as the integer 1 (the index of “null” in the union, encoded as hex 02):
02
-
the string “a” as zero (the index of “string” in the union), followed by the serialized string:
02 02 61
Fixed
Fixed instances are encoded using the number of bytes declared in the schema.
JSON Encoding
Except for unions, the JSON encoding is the same as is used to encode field default values.
The value of a union is encoded in JSON as follows:
- if its type is null, then it is encoded as a JSON null;
- otherwise it is encoded as a JSON object with one name/value pair whose name is the type’s name and whose value is the recursively encoded value. For Avro’s named types (record, fixed or enum) the user-specified name is used, for other types the type name is used.
For example, the union schema ["null","string","Foo"], where Foo is a record name, would encode:
- null as null;
- the string “a” as
{"string": "a"} and
- a Foo instance as
{"Foo": {...}}, where {...} indicates the JSON encoding of a Foo instance.
Note that a schema is still required to correctly process JSON-encoded data. For example, the JSON encoding does not distinguish between int and long, float and double, records and maps, enums and strings, etc.
Sort Order
Avro defines a standard sort order for data. This permits data written by one system to be efficiently sorted by another system. This can be an important optimization, as sort order comparisons are sometimes the most frequent per-object operation. Note also that Avro binary-encoded data can be efficiently ordered without deserializing it to objects.
Data items may only be compared if they have identical schemas. Pairwise comparisons are implemented recursively with a depth-first, left-to-right traversal of the schema. The first mismatch encountered determines the order of the items.
Two items with the same schema are compared according to the following rules.
- null data is always equal.
- boolean data is ordered with false before true.
- int, long, float and double data is ordered by ascending numeric value.
- bytes and fixed data are compared lexicographically by unsigned 8-bit values.
- string data is compared lexicographically by Unicode code point. Note that since UTF-8 is used as the binary encoding for strings, sorting of bytes and string binary data is identical.
- array data is compared lexicographically by element.
- enum data is ordered by the symbol’s position in the enum schema. For example, an enum whose symbols are
["z", "a"] would sort “z” values before “a” values.
- union data is first ordered by the branch within the union, and, within that, by the type of the branch. For example, an
["int", "string"] union would order all int values before all string values, with the ints and strings themselves ordered as defined above.
- record data is ordered lexicographically by field. If a field specifies that its order is:
- “ascending”, then the order of its values is unaltered.
- “descending”, then the order of its values is reversed.
- “ignore”, then its values are ignored when sorting.
- map data may not be compared. It is an error to attempt to compare data containing maps unless those maps are in an
"order":"ignore" record field.
Object Container Files
Avro includes a simple object container file format. A file has a schema, and all objects stored in the file must be written according to that schema, using binary encoding. Objects are stored in blocks that may be compressed. Syncronization markers are used between blocks to permit efficient splitting of files for MapReduce processing.
Files may include arbitrary user-specified metadata.
A file consists of:
- A file header, followed by
- one or more file data blocks.
A file header consists of:
- Four bytes, ASCII ‘O’, ‘b’, ‘j’, followed by 1.
- file metadata, including the schema.
- The 16-byte, randomly-generated sync marker for this file.
File metadata consists of:
A long indicating the number of metadata key/value pairs.
For each pair, a string key and bytes value.
All metadata properties that start with “avro.” are reserved. The following file metadata properties are currently used:
- avro.schema contains the schema of objects stored in the file, as JSON data (required).
- avro.codec the name of the compression codec used to compress blocks, as a string. Implementations are required to support the following codecs: “null” and “deflate”. If codec is absent, it is assumed to be “null”. The codecs are described with more detail below.
A file header is thus described by the following schema:
{"type": "record", "name": "org.apache.avro.file.Header",
"fields" : [
{"name": "magic", "type": {"type": "fixed", "name": "Magic", "size": 4}},
{"name": "meta", "type": {"type": "map", "values": "bytes"}},
{"name": "sync", "type": {"type": "fixed", "name": "Sync", "size": 16}}
]
}
A file data block consists of:
- A long indicating the count of objects in this block.
- A long indicating the size in bytes of the serialized objects in the current block, after any codec is applied
- The serialized objects. If a codec is specified, this is compressed by that codec.
- The file’s 16-byte sync marker.
Thus, each block’s binary data can be efficiently extracted or skipped without deserializing the contents. The combination of block size, object counts, and sync markers enable detection of corrupt blocks and help ensure data integrity.
Required Codecs
null
The “null” codec simply passes through data uncompressed.
deflate
The “deflate” codec writes the data block using the deflate algorithm as specified in RFC 1951, and typically implemented using the zlib library. Note that this format (unlike the “zlib format” in RFC 1950) does not have a checksum.
Optional Codecs
snappy
The “snappy” codec uses Google’s Snappy compression library. Each compressed block is followed by the 4-byte, big-endian CRC32 checksum.
Protocol Declaration
Avro protocols describe RPC interfaces. Like schemas, they are defined with JSON text.
A protocol is a JSON object with the following attributes:
- protocol, a string, the name of the protocol (required);
- namespace, an optional string that qualifies the name;
- doc, an optional string describing this protocol;
- types, an optional list of definitions of named types (records, enums, fixed and errors). An error definition is just like a record definition except it uses “error” instead of “record”. Note that forward references to named types are not permitted.
- messages, an optional JSON object whose keys are message names and whose values are objects whose attributes are described below. No two messages may have the same name.
The name and namespace qualification rules defined for schema objects apply to protocols as well.
Messages
A message has attributes:
- a doc, an optional description of the message,
- a request, a list of named, typed parameter schemas (this has the same form as the fields of a record declaration);
- a response schema;
- an optional union of declared error schemas. The effective union has “string” prepended to the declared union, to permit transmission of undeclared “system” errors. For example, if the declared error union is
["AccessError"], then the effective union is ["string", "AccessError"]. When no errors are declared, the effective error union is ["string"]. Errors are serialized using the effective union; however, a protocol’s JSON declaration contains only the declared union.
- an optional one-way boolean parameter.
A request parameter list is processed equivalently to an anonymous record. Since record field lists may vary between reader and writer, request parameters may also differ between the caller and responder, and such differences are resolved in the same manner as record field differences.
The one-way parameter may only be true when the response type is "null" and no errors are listed.
Sample Protocol
For example, one may define a simple HelloWorld protocol with:
{
"namespace": "com.acme",
"protocol": "HelloWorld",
"doc": "Protocol Greetings",
"types": [
{"name": "Greeting", "type": "record", "fields": [
{"name": "message", "type": "string"}]},
{"name": "Curse", "type": "error", "fields": [
{"name": "message", "type": "string"}]}
],
"messages": {
"hello": {
"doc": "Say hello.",
"request": [{"name": "greeting", "type": "Greeting" }],
"response": "Greeting",
"errors": ["Curse"]
}
}
}
Message Transport
Messages may be transmitted via different transport mechanisms.
To the transport, a message is an opaque byte sequence.
A transport is a system that supports:
- transmission of request messages
- receipt of corresponding response messages
Servers may send a response message back to the client corresponding to a request message. The mechanism of correspondance is transport-specific. For example, in HTTP it is implicit, since HTTP directly supports requests and responses. But a transport that multiplexes many client threads over a single socket would need to tag messages with unique identifiers.
Transports may be either stateless or stateful. In a stateless transport, messaging assumes no established connection state, while stateful transports establish connections that may be used for multiple messages. This distinction is discussed further in the handshake section below.
HTTP as Transport
When HTTP is used as a transport, each Avro message exchange is an HTTP request/response pair. All messages of an Avro protocol should share a single URL at an HTTP server. Other protocols may also use that URL. Both normal and error Avro response messages should use the 200 (OK) response code. The chunked encoding may be used for requests and responses, but, regardless the Avro request and response are the entire content of an HTTP request and response. The HTTP Content-Type of requests and responses should be specified as “avro/binary”. Requests should be made using the POST method.
HTTP is used by Avro as a stateless transport.
Message Framing
Avro messages are framed as a list of buffers.
Framing is a layer between messages and the transport. It exists to optimize certain operations.
The format of framed message data is:
- a series of buffers, where each buffer consists of:
- a four-byte, big-endian buffer length, followed by
- that many bytes of buffer data.
- A message is always terminated by a zero-lenghted buffer.
Framing is transparent to request and response message formats (described below). Any message may be presented as a single or multiple buffers.
Framing can permit readers to more efficiently get different buffers from different sources and for writers to more efficiently store different buffers to different destinations. In particular, it can reduce the number of times large binary objects are copied. For example, if an RPC parameter consists of a megabyte of file data, that data can be copied directly to a socket from a file descriptor, and, on the other end, it could be written directly to a file descriptor, never entering user space.
A simple, recommended, framing policy is for writers to create a new segment whenever a single binary object is written that is larger than a normal output buffer. Small objects are then appended in buffers, while larger objects are written as their own buffers. When a reader then tries to read a large object the runtime can hand it an entire buffer directly, without having to copy it.
Handshake
The purpose of the handshake is to ensure that the client and the server have each other’s protocol definition, so that the client can correctly deserialize responses, and the server can correctly deserialize requests. Both clients and servers should maintain a cache of recently seen protocols, so that, in most cases, a handshake will be completed without extra round-trip network exchanges or the transmission of full protocol text.
RPC requests and responses may not be processed until a handshake has been completed. With a stateless transport, all requests and responses are prefixed by handshakes. With a stateful transport, handshakes are only attached to requests and responses until a successful handshake response has been returned over a connection. After this, request and response payloads are sent without handshakes for the lifetime of that connection.
The handshake process uses the following record schemas:
{
"type": "record",
"name": "HandshakeRequest", "namespace":"org.apache.avro.ipc",
"fields": [
{"name": "clientHash",
"type": {"type": "fixed", "name": "MD5", "size": 16}},
{"name": "clientProtocol", "type": ["null", "string"]},
{"name": "serverHash", "type": "MD5"},
{"name": "meta", "type": ["null", {"type": "map", "values": "bytes"}]}
]
}
{
"type": "record",
"name": "HandshakeResponse", "namespace": "org.apache.avro.ipc",
"fields": [
{"name": "match",
"type": {"type": "enum", "name": "HandshakeMatch",
"symbols": ["BOTH", "CLIENT", "NONE"]}},
{"name": "serverProtocol",
"type": ["null", "string"]},
{"name": "serverHash",
"type": ["null", {"type": "fixed", "name": "MD5", "size": 16}]},
{"name": "meta",
"type": ["null", {"type": "map", "values": "bytes"}]}
]
}
- A client first prefixes each request with a
HandshakeRequest containing just the hash of its protocol and of the server’s protocol (clientHash!=null, clientProtocol=null, serverHash!=null), where the hashes are 128-bit MD5 hashes of the JSON protocol text. If a client has never connected to a given server, it sends its hash as a guess of the server’s hash, otherwise it sends the hash that it previously obtained from this server.
The server responds with a HandshakeResponse containing one of:
-
match=BOTH, serverProtocol=null, serverHash=null if the client sent the valid hash of the server’s protocol and the server knows what protocol corresponds to the client’s hash. In this case, the request is complete and the response data immediately follows the HandshakeResponse.
-
match=CLIENT, serverProtocol!=null, serverHash!=null if the server has previously seen the client’s protocol, but the client sent an incorrect hash of the server’s protocol. The request is complete and the response data immediately follows the HandshakeResponse. The client must use the returned protocol to process the response and should also cache that protocol and its hash for future interactions with this server.
-
match=NONE if the server has not previously seen the client’s protocol. The serverHash and serverProtocol may also be non-null if the server’s protocol hash was incorrect.
In this case the client must then re-submit its request with its protocol text (clientHash!=null, clientProtocol!=null, serverHash!=null) and the server should respond with a successful match (match=BOTH, serverProtocol=null, serverHash=null) as above.
The meta field is reserved for future handshake enhancements.
A call consists of a request message paired with its resulting response or error message. Requests and responses contain extensible metadata, and both kinds of messages are framed as described above.
The format of a call request is:
- request metadata, a map with values of type bytes
- the message name, an Avro string, followed by
- the message parameters. Parameters are serialized according to the message’s request declaration.
When a message is declared one-way and a stateful connection has been established by a successful handshake response, no response data is sent. Otherwise the format of the call response is:
- response metadata, a map with values of type bytes
- a one-byte error flag boolean, followed by either:
- if the error flag is false, the message response, serialized per the message’s response schema.
- if the error flag is true, the error, serialized per the message’s effective error union schema.
Schema Resolution
A reader of Avro data, whether from an RPC or a file, can always parse that data because its schema is provided. But that schema may not be exactly the schema that was expected. For example, if the data was written with a different version of the software than it is read, then records may have had fields added or removed. This section specifies how such schema differences should be resolved.
We refer to the schema used to write the data as the writer’s schema, and the schema that the application expects the reader’s schema. Differences between these should be resolved as follows:
- It is an error if the two schemas do not match.
To match, one of the following must hold:
-
both schemas are arrays whose item types match
-
both schemas are maps whose value types match
-
both schemas are enums whose names match
-
both schemas are fixed whose sizes and names match
-
both schemas are records with the same name
-
either schema is a union
-
both schemas have same primitive type
-
the writer’s schema may be promoted to the reader’s as follows:
- int is promotable to long, float, or double
- long is promotable to float or double
- float is promotable to double
- string is promotable to bytes
- bytes is promotable to string
-
if both are records:
- the ordering of fields may be different: fields are matched by name.
- schemas for fields with the same name in both records are resolved recursively.
- if the writer’s record contains a field with a name not present in the reader’s record, the writer’s value for that field is ignored.
- if the reader’s record schema has a field that contains a default value, and writer’s schema does not have a field with the same name, then the reader should use the default value from its field.
- if the reader’s record schema has a field with no default value, and writer’s schema does not have a field with the same name, an error is signalled.
-
if both are enums:
if the writer’s symbol is not present in the reader’s enum, then an error is signalled.
This resolution algorithm is applied recursively to the reader’s and writer’s array item schemas.
This resolution algorithm is applied recursively to the reader’s and writer’s value schemas.
The first schema in the reader’s union that matches the selected writer’s union schema is recursively resolved against it. if none match, an error is signalled.
- if reader’s is a union, but writer’s is not:
The first schema in the reader’s union that matches the writer’s schema is recursively resolved against it. If none match, an error is signalled.
- if writer’s is a union, but reader’s is not:
If the reader’s schema matches the selected writer’s schema, it is recursively resolved against it. If they do not match, an error is signalled.
A schema’s “doc” fields are ignored for the purposes of schema resolution. Hence, the “doc” portion of a schema may be dropped at serialization.
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
25.2 - IDL Language
Introduction
This document defines Avro IDL, an experimental higher-level language for authoring Avro schemata. Before reading this document, you should have familiarity with the concepts of schemata and protocols, as well as the various primitive and complex types available in Avro.
N.B. This feature is considered experimental in the current version of Avro and the language has not been finalized. Although major changes are unlikely, some syntax may change in future versions of Avro.
Overview
Purpose
The aim of the Avro IDL language is to enable developers to author schemata in a way that feels more similar to common programming languages like Java, C++, or Python. Additionally, the Avro IDL language may feel more familiar for those users who have previously used the interface description languages (IDLs) in other frameworks like Thrift, Protocol Buffers, or CORBA.
Usage
Each Avro IDL file defines a single Avro Protocol, and thus generates as its output a JSON-format Avro Protocol file with extension .avpr.
To convert a .avdl file into a .avpr file, it must be processed by the idl tool. For example:
$ java -jar avroj-tools-1.4.0.jar idl src/test/idl/input/namespaces.avdl /tmp/namespaces.avpr
$ head /tmp/namespaces.avpr
{
"protocol" : "TestNamespace",
"namespace" : "avro.test.protocol",
The idl tool can also process input to and from stdin and stdout. See idl --help for full usage information.
Defining a Protocol in Avro IDL
An Avro IDL file consists of exactly one protocol definition. The minimal protocol is defined by the following code:
This is equivalent to (and generates) the following JSON protocol definition:
{
"protocol" : "MyProtocol",
"types" : [ ],
"messages" : {
}
}
The namespace of the protocol may be changed using the @namespace annotation:
@namespace("mynamespace")
protocol MyProtocol {
}
This notation is used throughout Avro IDL as a way of specifying properties for the annotated element, as will be described later in this document.
Protocols in Avro IDL can contain the following items:
- Imports of external protocol and schema files.
- Definitions of named schemata, including records, errors, enums, and fixeds.
- Definitions of RPC messages
Imports
Files may be imported in one of three formats:
-
An IDL file may be imported with a statement like:
import idl "foo.avdl";
-
A JSON protocol file may be imported with a statement like:
import protocol "foo.avpr";
-
A JSON schema file may be imported with a statement like:
import schema "foo.avsc";
Messages and types in the imported file are added to this file’s protocol.
Imported file names are resolved relative to the current IDL file.
Defining an Enumeration
Enums are defined in Avro IDL using a syntax similar to C or Java:
enum Suit {
SPADES, DIAMONDS, CLUBS, HEARTS
}
Note that, unlike the JSON format, anonymous enums cannot be defined.
Defining a Fixed Length Field
Fixed fields are defined using the following syntax:
fixed MD5(16);
This example defines a fixed-length type called MD5 which contains 16 bytes.
Defining Records and Errors
Records are defined in Avro IDL using a syntax similar to a struct definition in C:
record Employee {
string name;
boolean active = true;
long salary;
}
The above example defines a record with the name “Employee” with three fields.
To define an error, simply use the keyword error instead of record. For example:
error Kaboom {
string explanation;
int result_code = -1;
}
Each field in a record or error consists of a type and a name, optional property annotations and an optional default value.
A type reference in Avro IDL must be one of:
- A primitive type
- A logical type
- A named schema defined prior to this usage in the same Protocol
- A complex type (array, map, or union)
Primitive Types
The primitive types supported by Avro IDL are the same as those supported by Avro’s JSON format. This list includes int, long, string, boolean, float, double, null, and bytes.
References to Named Schemata
If a named schema has already been defined in the same Avro IDL file, it may be referenced by name as if it were a primitive type:
record Card {
Suit suit; // refers to the enum Card defined above
int number;
}
Default Values
Default values for fields may be optionally specified by using an equals sign after the field name followed by a JSON expression indicating the default value. This JSON is interpreted as described in the spec.
Complex Types
Arrays
Array types are written in a manner that will seem familiar to C++ or Java programmers. An array of any type t is denoted array<t>. For example, an array of strings is denoted array<string>, and a multidimensional array of Foo records would be array<array<Foo>>.
Maps
Map types are written similarly to array types. An array that contains values of type t is written map<t>. As in the JSON schema format, all maps contain string-type keys.
Unions
Union types are denoted as union { typeA, typeB, typeC, … }. For example, this record contains a string field that is optional (unioned with null):
record RecordWithUnion {
union { null, string } optionalString;
}
Note that the same restrictions apply to Avro IDL unions as apply to unions defined in the JSON format; namely, a record may not contain multiple elements of the same type.
Defining RPC Messages
The syntax to define an RPC message within a Avro IDL protocol is similar to the syntax for a method declaration within a C header file or a Java interface. To define an RPC message add which takes two arguments named foo and bar, returning an int, simply include the following definition within the protocol:
int add(int foo, int bar = 0);
Message arguments, like record fields, may specify default values.
To define a message with no response, you may use the alias void, equivalent to the Avro null type:
void logMessage(string message);
If you have previously defined an error type within the same protocol, you may declare that a message can throw this error using the syntax:
void goKaboom() throws Kaboom;
To define a one-way message, use the keyword oneway after the parameter list, for example:
void fireAndForget(string message) oneway;
Other Language Features
All Java-style comments are supported within a Avro IDL file. Any text following // on a line is ignored, as is any text between /* and */, possibly spanning multiple lines.
Comments that begin with /** are used as the documentation string for the type or field definition that follows the comment.
Escaping Identifiers
Occasionally, one will need to use a reserved language keyword as an identifier. In order to do so, backticks (`) may be used to escape the identifier. For example, to define a message with the literal name error, you may write:
This syntax is allowed anywhere an identifier is expected.
Annotations for Ordering and Namespaces
Java-style annotations may be used to add additional properties to types and fields throughout Avro IDL.
For example, to specify the sort order of a field within a record, one may use the @order annotation before the field name as follows:
record MyRecord {
string @order("ascending") myAscendingSortField;
string @order("descending") myDescendingField;
string @order("ignore") myIgnoredField;
}
A field’s type may also be preceded by annotations, e.g.:
record MyRecord {
@java-class("java.util.ArrayList") array<string> myStrings;
}
Similarly, a @namespace annotation may be used to modify the namespace when defining a named schema. For example:
@namespace("org.apache.avro.firstNamespace")
protocol MyProto {
@namespace("org.apache.avro.someOtherNamespace")
record Foo {}
record Bar {}
}
will define a protocol in the firstNamespace namespace. The record Foo will be defined in someOtherNamespace and Bar will be defined in firstNamespace as it inherits its default from its container.
Type and field aliases are specified with the @aliases annotation as follows:
@aliases(["org.old.OldRecord", "org.ancient.AncientRecord"])
record MyRecord {
string @aliases(["oldField", "ancientField"]) myNewField;
}
Complete Example
The following is a complete example of a Avro IDL file that shows most of the above features:
/**
* An example protocol in Avro IDL
*/
@namespace("org.apache.avro.test")
protocol Simple {
@aliases(["org.foo.KindOf"])
enum Kind {
FOO,
BAR, // the bar enum value
BAZ
}
fixed MD5(16);
record TestRecord {
@order("ignore")
string name;
@order("descending")
Kind kind;
MD5 hash;
union { MD5, null} @aliases(["hash"]) nullableHash;
array<long> arrayOfLongs;
}
error TestError {
string message;
}
string hello(string greeting);
TestRecord echo(TestRecord `record`);
int add(int arg1, int arg2);
bytes echoBytes(bytes data);
void `error`() throws TestError;
void ping() oneway;
}
Additional examples may be found in the Avro source tree under the src/test/idl/input directory.
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
25.3 - SASL profile
Introduction
SASL (RFC 2222) provides a framework for authentication and security of network protocols. Each protocol that uses SASL is meant to define a SASL profile. This document provides a SASL profile for connection-based Avro RPC.
Overview
SASL negotiation proceeds as a series of message interactions over a connection between a client and server using a selected SASL mechanism. The client starts this negotiation by sending its chosen mechanism name with an initial (possibly empty) message. Negotiation proceeds with the exchange of messages until either side indicates success or failure. The content of the messages is mechanism-specific. If the negotiation succeeds, then the session can proceed over the connection, otherwise it must be abandoned.
Some mechanisms continue to process session data after negotiation (e.g., encrypting it), while some specify that further session data is transmitted unmodified.
Negotiation
Commands
Avro SASL negotiation uses four one-byte commands.
- 0: START Used in a client’s initial message.
- 1: CONTINUE Used while negotiation is ongoing.
- 2: FAIL Terminates negotiation unsuccessfully.
- 3: COMPLETE Terminates negotiation successfully.
The format of a START message is:
| 0 | 4-byte mechanism name length | mechanism name | 4-byte payload length | payload data |
The format of a CONTINUE message is:
| 1 | 4-byte payload length | payload data |
The format of a FAIL message is:
| 2 | 4-byte message length | UTF-8 message |
The format of a COMPLETE message is:
| 3 | 4-byte payload length | payload data |
Process
Negotiation is initiated by a client sending a START command containing the client’s chosen mechanism name and any mechanism-specific payload data.
The server and client then interchange some number (possibly zero) of CONTINUE messages. Each message contains payload data that is processed by the security mechanism to generate the next message.
Once either the client or server send a FAIL message then negotiation has failed. UTF-8-encoded text is included in the failure message. Once either a FAIL message has been sent or received, or any other error occurs in the negotiation, further communication on this connection must cease.
Once either the client or server send a COMPLETE message then negotiation has completed successfully. Session data may now be transmitted over the connection until it is closed by either side.
Session Data
If no SASL QOP (quality of protection) is negotiated, then all subsequent writes to/reads over this connection are written/read unmodified. In particular, messages use Avro framing, and are of the form:
| 4-byte frame length | frame data | ... | 4 zero bytes |
If a SASL QOP is negotiated, then it must be used by the connection for all subsequent messages. This is done by wrapping each non-empty frame written using the security mechanism and unwrapping each non-empty frame read. The length written in each non-empty frame is the length of the wrapped data. Complete frames must be passed to the security mechanism for unwrapping. Unwrapped data is then passed to the application as the content of the frame.
If at any point processing fails due to wrapping, unwrapping or framing errors, then all further communication on this connection must cease.
Anonymous Mechanism
The SASL anonymous mechanism (RFC 2245) is quite simple to implement. In particular, an initial anonymous request may be prefixed by the following static sequence:
| 0 | 0009 | ANONYMOUS | 0000 |
If a server uses the anonymous mechanism, it should check that the mechanism name in the start message prefixing the first request received is ‘ANONYMOUS’, then simply prefix its initial response with a COMPLETE message of:
| 3 | 0000 |
If an anonymous server recieves some other mechanism name, then it may respond with a FAIL message as simple as:
| 2 | 0000 |
Note that the anonymous mechanism need add no additional round-trip messages between client and server. The START message can be piggybacked on the initial request and the COMPLETE or FAIL message can be piggybacked on the initial response.
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
25.4 - Wiki
The Wiki page can be found here.
25.5 - FAQ
What is Avro?
Avro is a data serialization system.
Avro provides:
- Rich data structures.
- A compact, fast, binary data format.
- A container file, to store persistent data.
- Remote procedure call (RPC).
- Simple integration with dynamic languages. Code generation is not required to read or write data files nor to use or implement RPC protocols. Code generation as an optional optimization, only worth implementing for statically typed languages.
How can I get started quickly with Avro?
Check out the Quick Start Guide.
How do I statically compile a schema or protocol into generated code?
In Java
- Add the avro jar, the jackson-mapper-asl.jar and jackson-core-asl.jar to your CLASSPATH.
- Run java org.apache.avro.specific.SpecificCompiler .
This appears to be out of date, the SpecificCompiler requires two arguments, presumably an input and and output file, but it isn’t clear that this does.
Or use the Schema or Protocol Ant tasks. Avro’s build.xml provides examples of how these are used.
Lastly, you can also use the “avro-tools” jar which ships with an Avro release. Just use the “compile (schema|protocol)” command.
How are Strings represented in Java?
They use org.apache.avro.util.Utf8, not java.lang.String.
More generally, how do Avro types map to Java types?
The mappings are documented in the package javadoc for generic, specific and reflect API.
From Doug Cutting:
HDFS splits files into blocks, and mapreduce runs a map task for each block. When the task starts, it needs to be able to seek into the file to the start of the block process through the block’s end. If the file were, e.g., a gzip file, this would not be possible, since gzip files must be decompressed from the start. One cannot seek into the middle of a gzip file and start decompressing. So Hadoop’s SequenceFile places a marker periodically (~64k) in the file at record and compression boundaries, where processing can be sensibly started. Then, when a map task starts processing an HDFS block, it finds the first marker after the block’s start and continues through the first marker in the next block of the file. This requires a bit of non-local access (~0.1%). Avro’s data file uses the same method as SequenceFile.
Why isn’t every value in Avro nullable?
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 of every value stored and
corresponding computational costs to create this bit on write and
interpret it on read. These costs are wasted when values may not in
fact be null, as is the case in many datasets. In Avro such costs are
only paid when values may actually be null.
Also, allowing values to be null is a well-known source of errors. In
Avro, a value declared as non-null will always be non-null and programs
need not test for null values when processing it nor will they ever fail
for lack of such tests.
Tony Hoare calls his invention of null references his “Billion Dollar
Mistake”.
http://qconlondon.com/london-2009/presentation/Null+References:+The+Billion+Dollar+Mistake
Also note that in some programming languages not all values are permitted to be null. For example, in Java, values of type boolean, byte, short, char, int, float, long, and double may not be null.
How can I serialize directly to/from a byte array?
As pointed out in the specification, Avro data should always be stored with its schema. The Avro provided classes DataFileWriter, DataFileReader, and DataFileStream all ensure this by serializing the Schema in a container header. In some special cases, such as when implementing a new storage system or writing unit tests, you may need to write and read directly with the bare Avro serialized values. The following examples use code generated for Java from an Avro Schema, specifically the User example from the Quickstart guide.
Serializing to a byte array
This example takes a User object and returns a newly allocated byte array with the Avro serialization of that user.
ByteArrayOutputStream out = new ByteArrayOutputStream();
BinaryEncoder encoder = EncoderFactory.get().binaryEncoder(out, null);
DatumWriter<User> writer = new SpecificDatumWriter<User>(User.getClassSchema());
writer.write(user, encoder);
encoder.flush();
out.close();
byte[] serializedBytes = out.toByteArray();
Deserializing from a byte array
This example takes a byte array containing the Avro serialization of a user and returns a User object.SpecificDatumReader reader = new SpecificDatumReader(User.getClassSchema());
Decoder decoder = DecoderFactory.get().binaryDecoder(bytes, null);
User user = reader.read(null, decoder);
Note: If you are serializing or deserializing in a loop or as a method, you should be reusing objects, readers and/or writers. Check the JavaDoc to see how to reuse the objects.
26 - Apache Avro™ 1.5.2 Documentation
Introduction
Apache Avro™ is a data serialization system.
Avro provides:
- Rich data structures.
- A compact, fast, binary data format.
- A container file, to store persistent data.
- Remote procedure call (RPC).
- Simple integration with dynamic languages. Code generation is not required to read or write data files nor to use or implement RPC protocols. Code generation as an optional optimization, only worth implementing for statically typed languages.
Schemas
Avro relies on schemas. When Avro data is read, the schema used when writing it is always present. This permits each datum to be written with no per-value overheads, making serialization both fast and small. This also facilitates use with dynamic, scripting languages, since data, together with its schema, is fully self-describing.
When Avro data is stored in a file, its schema is stored with it, so that files may be processed later by any program. If the program reading the data expects a different schema this can be easily resolved, since both schemas are present.
When Avro is used in RPC, the client and server exchange schemas in the connection handshake. (This can be optimized so that, for most calls, no schemas are actually transmitted.) Since both client and server both have the other’s full schema, correspondence between same named fields, missing fields, extra fields, etc. can all be easily resolved.
Avro schemas are defined with JSON . This facilitates implementation in languages that already have JSON libraries.
Comparison with other systems
Avro provides functionality similar to systems such as Thrift, Protocol Buffers, etc. Avro differs from these systems in the following fundamental aspects.
- Dynamic typing: Avro does not require that code be generated. Data is always accompanied by a schema that permits full processing of that data without code generation, static datatypes, etc. This facilitates construction of generic data-processing systems and languages.
- Untagged data: Since the schema is present when data is read, considerably less type information need be encoded with data, resulting in smaller serialization size.
- No manually-assigned field IDs: When a schema changes, both the old and new schema are always present when processing data, so differences may be resolved symbolically, using field names.
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
26.1 - Specification
Introduction
This document defines Apache Avro. It is intended to be the authoritative specification. Implementations of Avro must adhere to this document.
Schema Declaration
A Schema is represented in JSON by one of:
- A JSON string, naming a defined type.
- A JSON object, of the form:
{"type": "typeName" ...attributes...}
where typeName is either a primitive or derived type name, as defined below. Attributes not defined in this document are permitted as metadata, but must not affect the format of serialized data.
- A JSON array, representing a union of embedded types.
Primitive Types
The set of primitive type names is:
- null: no value
- boolean: a binary value
- int: 32-bit signed integer
- long: 64-bit signed integer
- float: single precision (32-bit) IEEE 754 floating-point number
- double: double precision (64-bit) IEEE 754 floating-point number
- bytes: sequence of 8-bit unsigned bytes
- string: unicode character sequence
Primitive types have no specified attributes.
Primitive type names are also defined type names. Thus, for example, the schema “string” is equivalent to:
Complex Types
Avro supports six kinds of complex types: records, enums, arrays, maps, unions and fixed.
Records
Records use the type name “record” and support three attributes:
- name: a JSON string providing the name of the record (required).
- namespace, a JSON string that qualifies the name;
- doc: a JSON string providing documentation to the user of this schema (optional).
- aliases: a JSON array of strings, providing alternate names for this record (optional).
- fields: a JSON array, listing fields (required). Each field is a JSON object with the following attributes:
- name: a JSON string providing the name of the field (required), and
- doc: a JSON string describing this field for users (optional).
- type: A JSON object defining a schema, or a JSON string naming a record definition (required).
- default: A default value for this field, used when reading instances that lack this field (optional). 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. Default values for bytes and fixed fields are JSON strings, where Unicode code points 0-255 are mapped to unsigned 8-bit byte values 0-255.
field default values
| avro type |
json type |
example |
| null |
null |
null |
| boolean |
boolean |
true |
| int,long |
integer |
1 |
| float,double |
number |
1.1 |
| bytes |
string |
"\u00FF" |
| string |
string |
"foo" |
| record |
object |
{"a": 1} |
| enum |
string |
"FOO" |
| array |
array |
[1] |
| map |
object |
{"a": 1} |
| fixed |
string |
"\u00ff" |
- order: specifies how this field impacts sort ordering of this record (optional). Valid values are “ascending” (the default), “descending”, or “ignore”. For more details on how this is used, see the sort order section below.
- aliases: a JSON array of strings, providing alternate names for this field (optional).
For example, a linked-list of 64-bit values may be defined with:
{
"type": "record",
"name": "LongList",
"aliases": ["LinkedLongs"], // old name for this
"fields" : [
{"name": "value", "type": "long"}, // each element has a long
{"name": "next", "type": ["LongList", "null"]} // optional next element
]
}
Enums
Enums use the type name “enum” and support the following attributes:
- name: a JSON string providing the name of the enum (required).
- namespace, a JSON string that qualifies the name ;
- aliases: a JSON array of strings, providing alternate names for this enum (optional).
- doc: a JSON string providing documentation to the user of this schema (optional).
- symbols: a JSON array, listing symbols, as JSON strings (required). All symbols in an enum must be unique; duplicates are prohibited.
For example, playing card suits might be defined with:
{
"type": "enum",
"name": "Suit",
"symbols" : ["SPADES", "HEARTS", "DIAMONDS", "CLUBS"]
}
Arrays
Arrays use the type name “array” and support a single attribute:
- items: the schema of the array’s items.
For example, an array of strings is declared with:
{
"type": "array",
"items" : "string"
}
Maps
Maps use the type name “map” and support one attribute:
- values: the schema of the map’s values.
Map keys are assumed to be strings.
For example, a map from string to long is declared with:
{
"type": "map",
"values" : "long"
}
Unions
Unions, as mentioned above, are represented using JSON arrays. For example, ["string", "null"] declares a schema which may be either a string or null.
Unions may not contain more than one schema with the same type, except for the named types record, fixed and enum. For example, unions containing two array types or two map types are not permitted, but two types with different names are permitted. (Names permit efficient resolution when reading and writing unions.)
Unions may not immediately contain other unions.
Fixed
Fixed uses the type name “fixed” and supports two attributes:
- name: a string naming this fixed (required).
- namespace, a string that qualifies the name;
- aliases: a JSON array of strings, providing alternate names for this enum (optional).
- size: an integer, specifying the number of bytes per value (required).
For example, 16-byte quantity may be declared with:
{"type": "fixed", "size": 16, "name": "md5"}
Names
Record, enums and fixed are named types. Each has a fullname that is composed of two parts; a name and a namespace. Equality of names is defined on the fullname.
The name portion of a fullname, and record field names must:
- start with [A-Za-z_]
- subsequently contain only [A-Za-z0-9_]
A namespace is a dot-separated sequence of such names.
In record, enum and fixed definitions, the fullname is determined in one of the following ways:
A name and namespace are both specified. For example, one might use “name”: “X”, “namespace”: “org.foo” to indicate the fullname org.foo.X.
-
A fullname is specified. If the name specified contains a dot, then it is assumed to be a fullname, and any namespace also specified is ignored. For example, use “name”: “org.foo.X” to indicate the fullname org.foo.X.
-
A name only is specified, i.e., a name that contains no dots. In this case the namespace is taken from the most tightly enclosing schema or protocol. For example, if “name”: “X” is specified, and this occurs within a field of the record definition of org.foo.Y, then the fullname is org.foo.X.
References to previously defined names are as in the latter two cases above: if they contain a dot they are a fullname, if they do not contain a dot, the namespace is the namespace of the enclosing definition.
Primitive type names have no namespace and their names may not be defined in any namespace. A schema may only contain multiple definitions of a fullname if the definitions are equivalent.
Aliases
Named types and fields may have aliases. An implementation may optionally use aliases to map a writer’s schema to the reader’s. This facilitates both schema evolution as well as processing disparate datasets.
Aliases function by re-writing the writer’s schema using aliases from the reader’s schema. For example, if the writer’s schema was named “Foo” and the reader’s schema is named “Bar” and has an alias of “Foo”, then the implementation would act as though “Foo” were named “Bar” when reading. Similarly, if data was written as a record with a field named “x” and is read as a record with a field named “y” with alias “x”, then the implementation would act as though “x” were named “y” when reading.
A type alias may be specified either as a fully namespace-qualified, or relative to the namespace of the name it is an alias for. For example, if a type named “a.b” has aliases of “c” and “x.y”, then the fully qualified names of its aliases are “a.c” and “x.y”.
Data Serialization
Avro data is always serialized with its schema. Files that store Avro data should always also include the schema for that data in the same file. Avro-based remote procedure call (RPC) systems must also guarantee that remote recipients of data have a copy of the schema used to write that data.
Because the schema used to write data is always available when the data is read, Avro data itself is not tagged with type information. The schema is required to parse data.
In general, both serialization and deserialization proceed as a depth-first, left-to-right traversal of the schema, serializing primitive types as they are encountered.
Encodings
Avro specifies two serialization encodings: binary and JSON. Most applications will use the binary encoding, as it is smaller and faster. But, for debugging and web-based applications, the JSON encoding may sometimes be appropriate.
Binary Encoding
Primitive Types
Primitive types are encoded in binary as follows:
- null is written as zero bytes.
- a boolean is written as a single byte whose value is either 0 (false) or 1 (true).
- int and long values are written using variable-length zig-zag coding. Some examples:
| value |
hex |
| 0 |
00 |
| -1 |
01 |
| 1 |
02 |
| -2 |
03 |
| 2 |
04 |
| … |
|
| -64 |
7f |
| 64 |
80 01 |
| … |
|
- a float is written as 4 bytes. The float is converted into a 32-bit integer using a method equivalent to Java’s floatToIntBits and then encoded in little-endian format.
- a double is written as 8 bytes. The double is converted into a 64-bit integer using a method equivalent to Java’s doubleToLongBits and then encoded in little-endian format.
- bytes are encoded as a long followed by that many bytes of data.
- a string is encoded as a long followed by that many bytes of UTF-8 encoded character data.
For example, the three-character string “foo” would be encoded as the long value 3 (encoded as hex 06) followed by the UTF-8 encoding of ‘f’, ‘o’, and ‘o’ (the hex bytes 66 6f 6f):
06 66 6f 6f
Complex Types
Complex types are encoded in binary as follows:
Records
A record is encoded by encoding the values of its fields in the order that they are declared. In other words, a record is encoded as just the concatenation of the encodings of its fields. Field values are encoded per their schema.
For example, the record schema
{
"type": "record",
"name": "test",
"fields" : [
{"name": "a", "type": "long"},
{"name": "b", "type": "string"}
]
}
An instance of this record whose a field has value 27 (encoded as hex 36) and whose b field has value “foo” (encoded as hex bytes 06 66 6f 6f), would be encoded simply as the concatenation of these, namely the hex byte sequence:
36 06 66 6f 6f
Enums
An enum is encoded by a int, representing the zero-based position of the symbol in the schema.
For example, consider the enum:
{"type": "enum", "name": "Foo", "symbols": ["A", "B", "C", "D"] }
This would be encoded by an int between zero and three, with zero indicating “A”, and 3 indicating “D”.
Arrays
Arrays are encoded as a series of blocks. Each block consists of a long count value, followed by that many array items. A block with count zero indicates the end of the array. Each item is encoded per the array’s item schema.
If a block’s count is negative, its absolute value is used, and the count is followed immediately by a long block size indicating the number of bytes in the block. This block size permits fast skipping through data, e.g., when projecting a record to a subset of its fields.
For example, the array schema
{"type": "array", "items": "long"}
an array containing the items 3 and 27 could be encoded as the long value 2 (encoded as hex 04) followed by long values 3 and 27 (encoded as hex 06 36) terminated by zero:
04 06 36 00
The blocked representation permits one to read and write arrays larger than can be buffered in memory, since one can start writing items without knowing the full length of the array.
Maps
Maps are encoded as a series of blocks. Each block consists of a long count value, followed by that many key/value pairs. A block with count zero indicates the end of the map. Each item is encoded per the map’s value schema.
If a block’s count is negative, its absolute value is used, and the count is followed immediately by a long block size indicating the number of bytes in the block. This block size permits fast skipping through data, e.g., when projecting a record to a subset of its fields.
The blocked representation permits one to read and write maps larger than can be buffered in memory, since one can start writing items without knowing the full length of the map.
Unions
A union is encoded by first writing a long value indicating the zero-based position within the union of the schema of its value. The value is then encoded per the indicated schema within the union.
For example, the union schema ["string","null"] would encode:
-
null as the integer 1 (the index of “null” in the union, encoded as hex 02):
02
-
the string “a” as zero (the index of “string” in the union), followed by the serialized string:
02 02 61
Fixed
Fixed instances are encoded using the number of bytes declared in the schema.
JSON Encoding
Except for unions, the JSON encoding is the same as is used to encode field default values.
The value of a union is encoded in JSON as follows:
- if its type is null, then it is encoded as a JSON null;
- otherwise it is encoded as a JSON object with one name/value pair whose name is the type’s name and whose value is the recursively encoded value. For Avro’s named types (record, fixed or enum) the user-specified name is used, for other types the type name is used.
For example, the union schema ["null","string","Foo"], where Foo is a record name, would encode:
- null as null;
- the string “a” as
{"string": "a"} and
- a Foo instance as
{"Foo": {...}}, where {...} indicates the JSON encoding of a Foo instance.
Note that a schema is still required to correctly process JSON-encoded data. For example, the JSON encoding does not distinguish between int and long, float and double, records and maps, enums and strings, etc.
Sort Order
Avro defines a standard sort order for data. This permits data written by one system to be efficiently sorted by another system. This can be an important optimization, as sort order comparisons are sometimes the most frequent per-object operation. Note also that Avro binary-encoded data can be efficiently ordered without deserializing it to objects.
Data items may only be compared if they have identical schemas. Pairwise comparisons are implemented recursively with a depth-first, left-to-right traversal of the schema. The first mismatch encountered determines the order of the items.
Two items with the same schema are compared according to the following rules.
- null data is always equal.
- boolean data is ordered with false before true.
- int, long, float and double data is ordered by ascending numeric value.
- bytes and fixed data are compared lexicographically by unsigned 8-bit values.
- string data is compared lexicographically by Unicode code point. Note that since UTF-8 is used as the binary encoding for strings, sorting of bytes and string binary data is identical.
- array data is compared lexicographically by element.
- enum data is ordered by the symbol’s position in the enum schema. For example, an enum whose symbols are
["z", "a"] would sort “z” values before “a” values.
- union data is first ordered by the branch within the union, and, within that, by the type of the branch. For example, an
["int", "string"] union would order all int values before all string values, with the ints and strings themselves ordered as defined above.
- record data is ordered lexicographically by field. If a field specifies that its order is:
- “ascending”, then the order of its values is unaltered.
- “descending”, then the order of its values is reversed.
- “ignore”, then its values are ignored when sorting.
- map data may not be compared. It is an error to attempt to compare data containing maps unless those maps are in an
"order":"ignore" record field.
Object Container Files
Avro includes a simple object container file format. A file has a schema, and all objects stored in the file must be written according to that schema, using binary encoding. Objects are stored in blocks that may be compressed. Syncronization markers are used between blocks to permit efficient splitting of files for MapReduce processing.
Files may include arbitrary user-specified metadata.
A file consists of:
- A file header, followed by
- one or more file data blocks.
A file header consists of:
- Four bytes, ASCII ‘O’, ‘b’, ‘j’, followed by 1.
- file metadata, including the schema.
- The 16-byte, randomly-generated sync marker for this file.
File metadata consists of:
A long indicating the number of metadata key/value pairs.
For each pair, a string key and bytes value.
All metadata properties that start with “avro.” are reserved. The following file metadata properties are currently used:
- avro.schema contains the schema of objects stored in the file, as JSON data (required).
- avro.codec the name of the compression codec used to compress blocks, as a string. Implementations are required to support the following codecs: “null” and “deflate”. If codec is absent, it is assumed to be “null”. The codecs are described with more detail below.
A file header is thus described by the following schema:
{"type": "record", "name": "org.apache.avro.file.Header",
"fields" : [
{"name": "magic", "type": {"type": "fixed", "name": "Magic", "size": 4}},
{"name": "meta", "type": {"type": "map", "values": "bytes"}},
{"name": "sync", "type": {"type": "fixed", "name": "Sync", "size": 16}}
]
}
A file data block consists of:
- A long indicating the count of objects in this block.
- A long indicating the size in bytes of the serialized objects in the current block, after any codec is applied
- The serialized objects. If a codec is specified, this is compressed by that codec.
- The file’s 16-byte sync marker.
Thus, each block’s binary data can be efficiently extracted or skipped without deserializing the contents. The combination of block size, object counts, and sync markers enable detection of corrupt blocks and help ensure data integrity.
Required Codecs
null
The “null” codec simply passes through data uncompressed.
deflate
The “deflate” codec writes the data block using the deflate algorithm as specified in RFC 1951, and typically implemented using the zlib library. Note that this format (unlike the “zlib format” in RFC 1950) does not have a checksum.
Optional Codecs
snappy
The “snappy” codec uses Google’s Snappy compression library. Each compressed block is followed by the 4-byte, big-endian CRC32 checksum.
Protocol Declaration
Avro protocols describe RPC interfaces. Like schemas, they are defined with JSON text.
A protocol is a JSON object with the following attributes:
- protocol, a string, the name of the protocol (required);
- namespace, an optional string that qualifies the name;
- doc, an optional string describing this protocol;
- types, an optional list of definitions of named types (records, enums, fixed and errors). An error definition is just like a record definition except it uses “error” instead of “record”. Note that forward references to named types are not permitted.
- messages, an optional JSON object whose keys are message names and whose values are objects whose attributes are described below. No two messages may have the same name.
The name and namespace qualification rules defined for schema objects apply to protocols as well.
Messages
A message has attributes:
- a doc, an optional description of the message,
- a request, a list of named, typed parameter schemas (this has the same form as the fields of a record declaration);
- a response schema;
- an optional union of declared error schemas. The effective union has “string” prepended to the declared union, to permit transmission of undeclared “system” errors. For example, if the declared error union is
["AccessError"], then the effective union is ["string", "AccessError"]. When no errors are declared, the effective error union is ["string"]. Errors are serialized using the effective union; however, a protocol’s JSON declaration contains only the declared union.
- an optional one-way boolean parameter.
A request parameter list is processed equivalently to an anonymous record. Since record field lists may vary between reader and writer, request parameters may also differ between the caller and responder, and such differences are resolved in the same manner as record field differences.
The one-way parameter may only be true when the response type is "null" and no errors are listed.
Sample Protocol
For example, one may define a simple HelloWorld protocol with:
{
"namespace": "com.acme",
"protocol": "HelloWorld",
"doc": "Protocol Greetings",
"types": [
{"name": "Greeting", "type": "record", "fields": [
{"name": "message", "type": "string"}]},
{"name": "Curse", "type": "error", "fields": [
{"name": "message", "type": "string"}]}
],
"messages": {
"hello": {
"doc": "Say hello.",
"request": [{"name": "greeting", "type": "Greeting" }],
"response": "Greeting",
"errors": ["Curse"]
}
}
}
Message Transport
Messages may be transmitted via different transport mechanisms.
To the transport, a message is an opaque byte sequence.
A transport is a system that supports:
- transmission of request messages
- receipt of corresponding response messages
Servers may send a response message back to the client corresponding to a request message. The mechanism of correspondance is transport-specific. For example, in HTTP it is implicit, since HTTP directly supports requests and responses. But a transport that multiplexes many client threads over a single socket would need to tag messages with unique identifiers.
Transports may be either stateless or stateful. In a stateless transport, messaging assumes no established connection state, while stateful transports establish connections that may be used for multiple messages. This distinction is discussed further in the handshake section below.
HTTP as Transport
When HTTP is used as a transport, each Avro message exchange is an HTTP request/response pair. All messages of an Avro protocol should share a single URL at an HTTP server. Other protocols may also use that URL. Both normal and error Avro response messages should use the 200 (OK) response code. The chunked encoding may be used for requests and responses, but, regardless the Avro request and response are the entire content of an HTTP request and response. The HTTP Content-Type of requests and responses should be specified as “avro/binary”. Requests should be made using the POST method.
HTTP is used by Avro as a stateless transport.
Message Framing
Avro messages are framed as a list of buffers.
Framing is a layer between messages and the transport. It exists to optimize certain operations.
The format of framed message data is:
- a series of buffers, where each buffer consists of:
- a four-byte, big-endian buffer length, followed by
- that many bytes of buffer data.
- A message is always terminated by a zero-lenghted buffer.
Framing is transparent to request and response message formats (described below). Any message may be presented as a single or multiple buffers.
Framing can permit readers to more efficiently get different buffers from different sources and for writers to more efficiently store different buffers to different destinations. In particular, it can reduce the number of times large binary objects are copied. For example, if an RPC parameter consists of a megabyte of file data, that data can be copied directly to a socket from a file descriptor, and, on the other end, it could be written directly to a file descriptor, never entering user space.
A simple, recommended, framing policy is for writers to create a new segment whenever a single binary object is written that is larger than a normal output buffer. Small objects are then appended in buffers, while larger objects are written as their own buffers. When a reader then tries to read a large object the runtime can hand it an entire buffer directly, without having to copy it.
Handshake
The purpose of the handshake is to ensure that the client and the server have each other’s protocol definition, so that the client can correctly deserialize responses, and the server can correctly deserialize requests. Both clients and servers should maintain a cache of recently seen protocols, so that, in most cases, a handshake will be completed without extra round-trip network exchanges or the transmission of full protocol text.
RPC requests and responses may not be processed until a handshake has been completed. With a stateless transport, all requests and responses are prefixed by handshakes. With a stateful transport, handshakes are only attached to requests and responses until a successful handshake response has been returned over a connection. After this, request and response payloads are sent without handshakes for the lifetime of that connection.
The handshake process uses the following record schemas:
{
"type": "record",
"name": "HandshakeRequest", "namespace":"org.apache.avro.ipc",
"fields": [
{"name": "clientHash",
"type": {"type": "fixed", "name": "MD5", "size": 16}},
{"name": "clientProtocol", "type": ["null", "string"]},
{"name": "serverHash", "type": "MD5"},
{"name": "meta", "type": ["null", {"type": "map", "values": "bytes"}]}
]
}
{
"type": "record",
"name": "HandshakeResponse", "namespace": "org.apache.avro.ipc",
"fields": [
{"name": "match",
"type": {"type": "enum", "name": "HandshakeMatch",
"symbols": ["BOTH", "CLIENT", "NONE"]}},
{"name": "serverProtocol",
"type": ["null", "string"]},
{"name": "serverHash",
"type": ["null", {"type": "fixed", "name": "MD5", "size": 16}]},
{"name": "meta",
"type": ["null", {"type": "map", "values": "bytes"}]}
]
}
- A client first prefixes each request with a
HandshakeRequest containing just the hash of its protocol and of the server’s protocol (clientHash!=null, clientProtocol=null, serverHash!=null), where the hashes are 128-bit MD5 hashes of the JSON protocol text. If a client has never connected to a given server, it sends its hash as a guess of the server’s hash, otherwise it sends the hash that it previously obtained from this server.
The server responds with a HandshakeResponse containing one of:
-
match=BOTH, serverProtocol=null, serverHash=null if the client sent the valid hash of the server’s protocol and the server knows what protocol corresponds to the client’s hash. In this case, the request is complete and the response data immediately follows the HandshakeResponse.
-
match=CLIENT, serverProtocol!=null, serverHash!=null if the server has previously seen the client’s protocol, but the client sent an incorrect hash of the server’s protocol. The request is complete and the response data immediately follows the HandshakeResponse. The client must use the returned protocol to process the response and should also cache that protocol and its hash for future interactions with this server.
-
match=NONE if the server has not previously seen the client’s protocol. The serverHash and serverProtocol may also be non-null if the server’s protocol hash was incorrect.
In this case the client must then re-submit its request with its protocol text (clientHash!=null, clientProtocol!=null, serverHash!=null) and the server should respond with a successful match (match=BOTH, serverProtocol=null, serverHash=null) as above.
The meta field is reserved for future handshake enhancements.
A call consists of a request message paired with its resulting response or error message. Requests and responses contain extensible metadata, and both kinds of messages are framed as described above.
The format of a call request is:
- request metadata, a map with values of type bytes
- the message name, an Avro string, followed by
- the message parameters. Parameters are serialized according to the message’s request declaration.
When a message is declared one-way and a stateful connection has been established by a successful handshake response, no response data is sent. Otherwise the format of the call response is:
- response metadata, a map with values of type bytes
- a one-byte error flag boolean, followed by either:
- if the error flag is false, the message response, serialized per the message’s response schema.
- if the error flag is true, the error, serialized per the message’s effective error union schema.
Schema Resolution
A reader of Avro data, whether from an RPC or a file, can always parse that data because its schema is provided. But that schema may not be exactly the schema that was expected. For example, if the data was written with a different version of the software than it is read, then records may have had fields added or removed. This section specifies how such schema differences should be resolved.
We refer to the schema used to write the data as the writer’s schema, and the schema that the application expects the reader’s schema. Differences between these should be resolved as follows:
- It is an error if the two schemas do not match.
To match, one of the following must hold:
-
both schemas are arrays whose item types match
-
both schemas are maps whose value types match
-
both schemas are enums whose names match
-
both schemas are fixed whose sizes and names match
-
both schemas are records with the same name
-
either schema is a union
-
both schemas have same primitive type
-
the writer’s schema may be promoted to the reader’s as follows:
- int is promotable to long, float, or double
- long is promotable to float or double
- float is promotable to double
- string is promotable to bytes
- bytes is promotable to string
-
if both are records:
- the ordering of fields may be different: fields are matched by name.
- schemas for fields with the same name in both records are resolved recursively.
- if the writer’s record contains a field with a name not present in the reader’s record, the writer’s value for that field is ignored.
- if the reader’s record schema has a field that contains a default value, and writer’s schema does not have a field with the same name, then the reader should use the default value from its field.
- if the reader’s record schema has a field with no default value, and writer’s schema does not have a field with the same name, an error is signalled.
-
if both are enums:
if the writer’s symbol is not present in the reader’s enum, then an error is signalled.
This resolution algorithm is applied recursively to the reader’s and writer’s array item schemas.
This resolution algorithm is applied recursively to the reader’s and writer’s value schemas.
The first schema in the reader’s union that matches the selected writer’s union schema is recursively resolved against it. if none match, an error is signalled.
- if reader’s is a union, but writer’s is not:
The first schema in the reader’s union that matches the writer’s schema is recursively resolved against it. If none match, an error is signalled.
- if writer’s is a union, but reader’s is not:
If the reader’s schema matches the selected writer’s schema, it is recursively resolved against it. If they do not match, an error is signalled.
A schema’s “doc” fields are ignored for the purposes of schema resolution. Hence, the “doc” portion of a schema may be dropped at serialization.
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
26.2 - IDL Language
Introduction
This document defines Avro IDL, an experimental higher-level language for authoring Avro schemata. Before reading this document, you should have familiarity with the concepts of schemata and protocols, as well as the various primitive and complex types available in Avro.
N.B. This feature is considered experimental in the current version of Avro and the language has not been finalized. Although major changes are unlikely, some syntax may change in future versions of Avro.
Overview
Purpose
The aim of the Avro IDL language is to enable developers to author schemata in a way that feels more similar to common programming languages like Java, C++, or Python. Additionally, the Avro IDL language may feel more familiar for those users who have previously used the interface description languages (IDLs) in other frameworks like Thrift, Protocol Buffers, or CORBA.
Usage
Each Avro IDL file defines a single Avro Protocol, and thus generates as its output a JSON-format Avro Protocol file with extension .avpr.
To convert a .avdl file into a .avpr file, it must be processed by the idl tool. For example:
$ java -jar avroj-tools-1.4.0.jar idl src/test/idl/input/namespaces.avdl /tmp/namespaces.avpr
$ head /tmp/namespaces.avpr
{
"protocol" : "TestNamespace",
"namespace" : "avro.test.protocol",
The idl tool can also process input to and from stdin and stdout. See idl --help for full usage information.
Defining a Protocol in Avro IDL
An Avro IDL file consists of exactly one protocol definition. The minimal protocol is defined by the following code:
This is equivalent to (and generates) the following JSON protocol definition:
{
"protocol" : "MyProtocol",
"types" : [ ],
"messages" : {
}
}
The namespace of the protocol may be changed using the @namespace annotation:
@namespace("mynamespace")
protocol MyProtocol {
}
This notation is used throughout Avro IDL as a way of specifying properties for the annotated element, as will be described later in this document.
Protocols in Avro IDL can contain the following items:
- Imports of external protocol and schema files.
- Definitions of named schemata, including records, errors, enums, and fixeds.
- Definitions of RPC messages
Imports
Files may be imported in one of three formats:
-
An IDL file may be imported with a statement like:
import idl "foo.avdl";
-
A JSON protocol file may be imported with a statement like:
import protocol "foo.avpr";
-
A JSON schema file may be imported with a statement like:
import schema "foo.avsc";
Messages and types in the imported file are added to this file’s protocol.
Imported file names are resolved relative to the current IDL file.
Defining an Enumeration
Enums are defined in Avro IDL using a syntax similar to C or Java:
enum Suit {
SPADES, DIAMONDS, CLUBS, HEARTS
}
Note that, unlike the JSON format, anonymous enums cannot be defined.
Defining a Fixed Length Field
Fixed fields are defined using the following syntax:
fixed MD5(16);
This example defines a fixed-length type called MD5 which contains 16 bytes.
Defining Records and Errors
Records are defined in Avro IDL using a syntax similar to a struct definition in C:
record Employee {
string name;
boolean active = true;
long salary;
}
The above example defines a record with the name “Employee” with three fields.
To define an error, simply use the keyword error instead of record. For example:
error Kaboom {
string explanation;
int result_code = -1;
}
Each field in a record or error consists of a type and a name, optional property annotations and an optional default value.
A type reference in Avro IDL must be one of:
- A primitive type
- A logical type
- A named schema defined prior to this usage in the same Protocol
- A complex type (array, map, or union)
Primitive Types
The primitive types supported by Avro IDL are the same as those supported by Avro’s JSON format. This list includes int, long, string, boolean, float, double, null, and bytes.
References to Named Schemata
If a named schema has already been defined in the same Avro IDL file, it may be referenced by name as if it were a primitive type:
record Card {
Suit suit; // refers to the enum Card defined above
int number;
}
Default Values
Default values for fields may be optionally specified by using an equals sign after the field name followed by a JSON expression indicating the default value. This JSON is interpreted as described in the spec.
Complex Types
Arrays
Array types are written in a manner that will seem familiar to C++ or Java programmers. An array of any type t is denoted array<t>. For example, an array of strings is denoted array<string>, and a multidimensional array of Foo records would be array<array<Foo>>.
Maps
Map types are written similarly to array types. An array that contains values of type t is written map<t>. As in the JSON schema format, all maps contain string-type keys.
Unions
Union types are denoted as union { typeA, typeB, typeC, … }. For example, this record contains a string field that is optional (unioned with null):
record RecordWithUnion {
union { null, string } optionalString;
}
Note that the same restrictions apply to Avro IDL unions as apply to unions defined in the JSON format; namely, a record may not contain multiple elements of the same type.
Defining RPC Messages
The syntax to define an RPC message within a Avro IDL protocol is similar to the syntax for a method declaration within a C header file or a Java interface. To define an RPC message add which takes two arguments named foo and bar, returning an int, simply include the following definition within the protocol:
int add(int foo, int bar = 0);
Message arguments, like record fields, may specify default values.
To define a message with no response, you may use the alias void, equivalent to the Avro null type:
void logMessage(string message);
If you have previously defined an error type within the same protocol, you may declare that a message can throw this error using the syntax:
void goKaboom() throws Kaboom;
To define a one-way message, use the keyword oneway after the parameter list, for example:
void fireAndForget(string message) oneway;
Other Language Features
All Java-style comments are supported within a Avro IDL file. Any text following // on a line is ignored, as is any text between /* and */, possibly spanning multiple lines.
Comments that begin with /** are used as the documentation string for the type or field definition that follows the comment.
Escaping Identifiers
Occasionally, one will need to use a reserved language keyword as an identifier. In order to do so, backticks (`) may be used to escape the identifier. For example, to define a message with the literal name error, you may write:
This syntax is allowed anywhere an identifier is expected.
Annotations for Ordering and Namespaces
Java-style annotations may be used to add additional properties to types and fields throughout Avro IDL.
For example, to specify the sort order of a field within a record, one may use the @order annotation before the field name as follows:
record MyRecord {
string @order("ascending") myAscendingSortField;
string @order("descending") myDescendingField;
string @order("ignore") myIgnoredField;
}
A field’s type may also be preceded by annotations, e.g.:
record MyRecord {
@java-class("java.util.ArrayList") array<string> myStrings;
}
Similarly, a @namespace annotation may be used to modify the namespace when defining a named schema. For example:
@namespace("org.apache.avro.firstNamespace")
protocol MyProto {
@namespace("org.apache.avro.someOtherNamespace")
record Foo {}
record Bar {}
}
will define a protocol in the firstNamespace namespace. The record Foo will be defined in someOtherNamespace and Bar will be defined in firstNamespace as it inherits its default from its container.
Type and field aliases are specified with the @aliases annotation as follows:
@aliases(["org.old.OldRecord", "org.ancient.AncientRecord"])
record MyRecord {
string @aliases(["oldField", "ancientField"]) myNewField;
}
Complete Example
The following is a complete example of a Avro IDL file that shows most of the above features:
/**
* An example protocol in Avro IDL
*/
@namespace("org.apache.avro.test")
protocol Simple {
@aliases(["org.foo.KindOf"])
enum Kind {
FOO,
BAR, // the bar enum value
BAZ
}
fixed MD5(16);
record TestRecord {
@order("ignore")
string name;
@order("descending")
Kind kind;
MD5 hash;
union { MD5, null} @aliases(["hash"]) nullableHash;
array<long> arrayOfLongs;
}
error TestError {
string message;
}
string hello(string greeting);
TestRecord echo(TestRecord `record`);
int add(int arg1, int arg2);
bytes echoBytes(bytes data);
void `error`() throws TestError;
void ping() oneway;
}
Additional examples may be found in the Avro source tree under the src/test/idl/input directory.
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
26.3 - SASL profile
Introduction
SASL (RFC 2222) provides a framework for authentication and security of network protocols. Each protocol that uses SASL is meant to define a SASL profile. This document provides a SASL profile for connection-based Avro RPC.
Overview
SASL negotiation proceeds as a series of message interactions over a connection between a client and server using a selected SASL mechanism. The client starts this negotiation by sending its chosen mechanism name with an initial (possibly empty) message. Negotiation proceeds with the exchange of messages until either side indicates success or failure. The content of the messages is mechanism-specific. If the negotiation succeeds, then the session can proceed over the connection, otherwise it must be abandoned.
Some mechanisms continue to process session data after negotiation (e.g., encrypting it), while some specify that further session data is transmitted unmodified.
Negotiation
Commands
Avro SASL negotiation uses four one-byte commands.
- 0: START Used in a client’s initial message.
- 1: CONTINUE Used while negotiation is ongoing.
- 2: FAIL Terminates negotiation unsuccessfully.
- 3: COMPLETE Terminates negotiation successfully.
The format of a START message is:
| 0 | 4-byte mechanism name length | mechanism name | 4-byte payload length | payload data |
The format of a CONTINUE message is:
| 1 | 4-byte payload length | payload data |
The format of a FAIL message is:
| 2 | 4-byte message length | UTF-8 message |
The format of a COMPLETE message is:
| 3 | 4-byte payload length | payload data |
Process
Negotiation is initiated by a client sending a START command containing the client’s chosen mechanism name and any mechanism-specific payload data.
The server and client then interchange some number (possibly zero) of CONTINUE messages. Each message contains payload data that is processed by the security mechanism to generate the next message.
Once either the client or server send a FAIL message then negotiation has failed. UTF-8-encoded text is included in the failure message. Once either a FAIL message has been sent or received, or any other error occurs in the negotiation, further communication on this connection must cease.
Once either the client or server send a COMPLETE message then negotiation has completed successfully. Session data may now be transmitted over the connection until it is closed by either side.
Session Data
If no SASL QOP (quality of protection) is negotiated, then all subsequent writes to/reads over this connection are written/read unmodified. In particular, messages use Avro framing, and are of the form:
| 4-byte frame length | frame data | ... | 4 zero bytes |
If a SASL QOP is negotiated, then it must be used by the connection for all subsequent messages. This is done by wrapping each non-empty frame written using the security mechanism and unwrapping each non-empty frame read. The length written in each non-empty frame is the length of the wrapped data. Complete frames must be passed to the security mechanism for unwrapping. Unwrapped data is then passed to the application as the content of the frame.
If at any point processing fails due to wrapping, unwrapping or framing errors, then all further communication on this connection must cease.
Anonymous Mechanism
The SASL anonymous mechanism (RFC 2245) is quite simple to implement. In particular, an initial anonymous request may be prefixed by the following static sequence:
| 0 | 0009 | ANONYMOUS | 0000 |
If a server uses the anonymous mechanism, it should check that the mechanism name in the start message prefixing the first request received is ‘ANONYMOUS’, then simply prefix its initial response with a COMPLETE message of:
| 3 | 0000 |
If an anonymous server recieves some other mechanism name, then it may respond with a FAIL message as simple as:
| 2 | 0000 |
Note that the anonymous mechanism need add no additional round-trip messages between client and server. The START message can be piggybacked on the initial request and the COMPLETE or FAIL message can be piggybacked on the initial response.
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
26.4 - Wiki
The Wiki page can be found here.
26.5 - FAQ
What is Avro?
Avro is a data serialization system.
Avro provides:
- Rich data structures.
- A compact, fast, binary data format.
- A container file, to store persistent data.
- Remote procedure call (RPC).
- Simple integration with dynamic languages. Code generation is not required to read or write data files nor to use or implement RPC protocols. Code generation as an optional optimization, only worth implementing for statically typed languages.
How can I get started quickly with Avro?
Check out the Quick Start Guide.
How do I statically compile a schema or protocol into generated code?
In Java
- Add the avro jar, the jackson-mapper-asl.jar and jackson-core-asl.jar to your CLASSPATH.
- Run java org.apache.avro.specific.SpecificCompiler .
This appears to be out of date, the SpecificCompiler requires two arguments, presumably an input and and output file, but it isn’t clear that this does.
Or use the Schema or Protocol Ant tasks. Avro’s build.xml provides examples of how these are used.
Lastly, you can also use the “avro-tools” jar which ships with an Avro release. Just use the “compile (schema|protocol)” command.
How are Strings represented in Java?
They use org.apache.avro.util.Utf8, not java.lang.String.
More generally, how do Avro types map to Java types?
The mappings are documented in the package javadoc for generic, specific and reflect API.
From Doug Cutting:
HDFS splits files into blocks, and mapreduce runs a map task for each block. When the task starts, it needs to be able to seek into the file to the start of the block process through the block’s end. If the file were, e.g., a gzip file, this would not be possible, since gzip files must be decompressed from the start. One cannot seek into the middle of a gzip file and start decompressing. So Hadoop’s SequenceFile places a marker periodically (~64k) in the file at record and compression boundaries, where processing can be sensibly started. Then, when a map task starts processing an HDFS block, it finds the first marker after the block’s start and continues through the first marker in the next block of the file. This requires a bit of non-local access (~0.1%). Avro’s data file uses the same method as SequenceFile.
Why isn’t every value in Avro nullable?
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 of every value stored and
corresponding computational costs to create this bit on write and
interpret it on read. These costs are wasted when values may not in
fact be null, as is the case in many datasets. In Avro such costs are
only paid when values may actually be null.
Also, allowing values to be null is a well-known source of errors. In
Avro, a value declared as non-null will always be non-null and programs
need not test for null values when processing it nor will they ever fail
for lack of such tests.
Tony Hoare calls his invention of null references his “Billion Dollar
Mistake”.
http://qconlondon.com/london-2009/presentation/Null+References:+The+Billion+Dollar+Mistake
Also note that in some programming languages not all values are permitted to be null. For example, in Java, values of type boolean, byte, short, char, int, float, long, and double may not be null.
How can I serialize directly to/from a byte array?
As pointed out in the specification, Avro data should always be stored with its schema. The Avro provided classes DataFileWriter, DataFileReader, and DataFileStream all ensure this by serializing the Schema in a container header. In some special cases, such as when implementing a new storage system or writing unit tests, you may need to write and read directly with the bare Avro serialized values. The following examples use code generated for Java from an Avro Schema, specifically the User example from the Quickstart guide.
Serializing to a byte array
This example takes a User object and returns a newly allocated byte array with the Avro serialization of that user.
ByteArrayOutputStream out = new ByteArrayOutputStream();
BinaryEncoder encoder = EncoderFactory.get().binaryEncoder(out, null);
DatumWriter<User> writer = new SpecificDatumWriter<User>(User.getClassSchema());
writer.write(user, encoder);
encoder.flush();
out.close();
byte[] serializedBytes = out.toByteArray();
Deserializing from a byte array
This example takes a byte array containing the Avro serialization of a user and returns a User object.SpecificDatumReader reader = new SpecificDatumReader(User.getClassSchema());
Decoder decoder = DecoderFactory.get().binaryDecoder(bytes, null);
User user = reader.read(null, decoder);
Note: If you are serializing or deserializing in a loop or as a method, you should be reusing objects, readers and/or writers. Check the JavaDoc to see how to reuse the objects.
27 - Apache Avro™ 1.5.1 Documentation
Introduction
Apache Avro™ is a data serialization system.
Avro provides:
- Rich data structures.
- A compact, fast, binary data format.
- A container file, to store persistent data.
- Remote procedure call (RPC).
- Simple integration with dynamic languages. Code generation is not required to read or write data files nor to use or implement RPC protocols. Code generation as an optional optimization, only worth implementing for statically typed languages.
Schemas
Avro relies on schemas. When Avro data is read, the schema used when writing it is always present. This permits each datum to be written with no per-value overheads, making serialization both fast and small. This also facilitates use with dynamic, scripting languages, since data, together with its schema, is fully self-describing.
When Avro data is stored in a file, its schema is stored with it, so that files may be processed later by any program. If the program reading the data expects a different schema this can be easily resolved, since both schemas are present.
When Avro is used in RPC, the client and server exchange schemas in the connection handshake. (This can be optimized so that, for most calls, no schemas are actually transmitted.) Since both client and server both have the other’s full schema, correspondence between same named fields, missing fields, extra fields, etc. can all be easily resolved.
Avro schemas are defined with JSON . This facilitates implementation in languages that already have JSON libraries.
Comparison with other systems
Avro provides functionality similar to systems such as Thrift, Protocol Buffers, etc. Avro differs from these systems in the following fundamental aspects.
- Dynamic typing: Avro does not require that code be generated. Data is always accompanied by a schema that permits full processing of that data without code generation, static datatypes, etc. This facilitates construction of generic data-processing systems and languages.
- Untagged data: Since the schema is present when data is read, considerably less type information need be encoded with data, resulting in smaller serialization size.
- No manually-assigned field IDs: When a schema changes, both the old and new schema are always present when processing data, so differences may be resolved symbolically, using field names.
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
27.1 - Specification
Introduction
This document defines Apache Avro. It is intended to be the authoritative specification. Implementations of Avro must adhere to this document.
Schema Declaration
A Schema is represented in JSON by one of:
- A JSON string, naming a defined type.
- A JSON object, of the form:
{"type": "typeName" ...attributes...}
where typeName is either a primitive or derived type name, as defined below. Attributes not defined in this document are permitted as metadata, but must not affect the format of serialized data.
- A JSON array, representing a union of embedded types.
Primitive Types
The set of primitive type names is:
- null: no value
- boolean: a binary value
- int: 32-bit signed integer
- long: 64-bit signed integer
- float: single precision (32-bit) IEEE 754 floating-point number
- double: double precision (64-bit) IEEE 754 floating-point number
- bytes: sequence of 8-bit unsigned bytes
- string: unicode character sequence
Primitive types have no specified attributes.
Primitive type names are also defined type names. Thus, for example, the schema “string” is equivalent to:
Complex Types
Avro supports six kinds of complex types: records, enums, arrays, maps, unions and fixed.
Records
Records use the type name “record” and support three attributes:
- name: a JSON string providing the name of the record (required).
- namespace, a JSON string that qualifies the name;
- doc: a JSON string providing documentation to the user of this schema (optional).
- aliases: a JSON array of strings, providing alternate names for this record (optional).
- fields: a JSON array, listing fields (required). Each field is a JSON object with the following attributes:
- name: a JSON string providing the name of the field (required), and
- doc: a JSON string describing this field for users (optional).
- type: A JSON object defining a schema, or a JSON string naming a record definition (required).
- default: A default value for this field, used when reading instances that lack this field (optional). 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. Default values for bytes and fixed fields are JSON strings, where Unicode code points 0-255 are mapped to unsigned 8-bit byte values 0-255.
field default values
| avro type |
json type |
example |
| null |
null |
null |
| boolean |
boolean |
true |
| int,long |
integer |
1 |
| float,double |
number |
1.1 |
| bytes |
string |
"\u00FF" |
| string |
string |
"foo" |
| record |
object |
{"a": 1} |
| enum |
string |
"FOO" |
| array |
array |
[1] |
| map |
object |
{"a": 1} |
| fixed |
string |
"\u00ff" |
- order: specifies how this field impacts sort ordering of this record (optional). Valid values are “ascending” (the default), “descending”, or “ignore”. For more details on how this is used, see the sort order section below.
- aliases: a JSON array of strings, providing alternate names for this field (optional).
For example, a linked-list of 64-bit values may be defined with:
{
"type": "record",
"name": "LongList",
"aliases": ["LinkedLongs"], // old name for this
"fields" : [
{"name": "value", "type": "long"}, // each element has a long
{"name": "next", "type": ["LongList", "null"]} // optional next element
]
}
Enums
Enums use the type name “enum” and support the following attributes:
- name: a JSON string providing the name of the enum (required).
- namespace, a JSON string that qualifies the name ;
- aliases: a JSON array of strings, providing alternate names for this enum (optional).
- doc: a JSON string providing documentation to the user of this schema (optional).
- symbols: a JSON array, listing symbols, as JSON strings (required). All symbols in an enum must be unique; duplicates are prohibited.
For example, playing card suits might be defined with:
{
"type": "enum",
"name": "Suit",
"symbols" : ["SPADES", "HEARTS", "DIAMONDS", "CLUBS"]
}
Arrays
Arrays use the type name “array” and support a single attribute:
- items: the schema of the array’s items.
For example, an array of strings is declared with:
{
"type": "array",
"items" : "string"
}
Maps
Maps use the type name “map” and support one attribute:
- values: the schema of the map’s values.
Map keys are assumed to be strings.
For example, a map from string to long is declared with:
{
"type": "map",
"values" : "long"
}
Unions
Unions, as mentioned above, are represented using JSON arrays. For example, ["string", "null"] declares a schema which may be either a string or null.
Unions may not contain more than one schema with the same type, except for the named types record, fixed and enum. For example, unions containing two array types or two map types are not permitted, but two types with different names are permitted. (Names permit efficient resolution when reading and writing unions.)
Unions may not immediately contain other unions.
Fixed
Fixed uses the type name “fixed” and supports two attributes:
- name: a string naming this fixed (required).
- namespace, a string that qualifies the name;
- aliases: a JSON array of strings, providing alternate names for this enum (optional).
- size: an integer, specifying the number of bytes per value (required).
For example, 16-byte quantity may be declared with:
{"type": "fixed", "size": 16, "name": "md5"}
Names
Record, enums and fixed are named types. Each has a fullname that is composed of two parts; a name and a namespace. Equality of names is defined on the fullname.
The name portion of a fullname, and record field names must:
- start with [A-Za-z_]
- subsequently contain only [A-Za-z0-9_]
A namespace is a dot-separated sequence of such names.
In record, enum and fixed definitions, the fullname is determined in one of the following ways:
A name and namespace are both specified. For example, one might use “name”: “X”, “namespace”: “org.foo” to indicate the fullname org.foo.X.
-
A fullname is specified. If the name specified contains a dot, then it is assumed to be a fullname, and any namespace also specified is ignored. For example, use “name”: “org.foo.X” to indicate the fullname org.foo.X.
-
A name only is specified, i.e., a name that contains no dots. In this case the namespace is taken from the most tightly enclosing schema or protocol. For example, if “name”: “X” is specified, and this occurs within a field of the record definition of org.foo.Y, then the fullname is org.foo.X.
References to previously defined names are as in the latter two cases above: if they contain a dot they are a fullname, if they do not contain a dot, the namespace is the namespace of the enclosing definition.
Primitive type names have no namespace and their names may not be defined in any namespace. A schema may only contain multiple definitions of a fullname if the definitions are equivalent.
Aliases
Named types and fields may have aliases. An implementation may optionally use aliases to map a writer’s schema to the reader’s. This facilitates both schema evolution as well as processing disparate datasets.
Aliases function by re-writing the writer’s schema using aliases from the reader’s schema. For example, if the writer’s schema was named “Foo” and the reader’s schema is named “Bar” and has an alias of “Foo”, then the implementation would act as though “Foo” were named “Bar” when reading. Similarly, if data was written as a record with a field named “x” and is read as a record with a field named “y” with alias “x”, then the implementation would act as though “x” were named “y” when reading.
A type alias may be specified either as a fully namespace-qualified, or relative to the namespace of the name it is an alias for. For example, if a type named “a.b” has aliases of “c” and “x.y”, then the fully qualified names of its aliases are “a.c” and “x.y”.
Data Serialization
Avro data is always serialized with its schema. Files that store Avro data should always also include the schema for that data in the same file. Avro-based remote procedure call (RPC) systems must also guarantee that remote recipients of data have a copy of the schema used to write that data.
Because the schema used to write data is always available when the data is read, Avro data itself is not tagged with type information. The schema is required to parse data.
In general, both serialization and deserialization proceed as a depth-first, left-to-right traversal of the schema, serializing primitive types as they are encountered.
Encodings
Avro specifies two serialization encodings: binary and JSON. Most applications will use the binary encoding, as it is smaller and faster. But, for debugging and web-based applications, the JSON encoding may sometimes be appropriate.
Binary Encoding
Primitive Types
Primitive types are encoded in binary as follows:
- null is written as zero bytes.
- a boolean is written as a single byte whose value is either 0 (false) or 1 (true).
- int and long values are written using variable-length zig-zag coding. Some examples:
| value |
hex |
| 0 |
00 |
| -1 |
01 |
| 1 |
02 |
| -2 |
03 |
| 2 |
04 |
| … |
|
| -64 |
7f |
| 64 |
80 01 |
| … |
|
- a float is written as 4 bytes. The float is converted into a 32-bit integer using a method equivalent to Java’s floatToIntBits and then encoded in little-endian format.
- a double is written as 8 bytes. The double is converted into a 64-bit integer using a method equivalent to Java’s doubleToLongBits and then encoded in little-endian format.
- bytes are encoded as a long followed by that many bytes of data.
- a string is encoded as a long followed by that many bytes of UTF-8 encoded character data.
For example, the three-character string “foo” would be encoded as the long value 3 (encoded as hex 06) followed by the UTF-8 encoding of ‘f’, ‘o’, and ‘o’ (the hex bytes 66 6f 6f):
06 66 6f 6f
Complex Types
Complex types are encoded in binary as follows:
Records
A record is encoded by encoding the values of its fields in the order that they are declared. In other words, a record is encoded as just the concatenation of the encodings of its fields. Field values are encoded per their schema.
For example, the record schema
{
"type": "record",
"name": "test",
"fields" : [
{"name": "a", "type": "long"},
{"name": "b", "type": "string"}
]
}
An instance of this record whose a field has value 27 (encoded as hex 36) and whose b field has value “foo” (encoded as hex bytes 06 66 6f 6f), would be encoded simply as the concatenation of these, namely the hex byte sequence:
36 06 66 6f 6f
Enums
An enum is encoded by a int, representing the zero-based position of the symbol in the schema.
For example, consider the enum:
{"type": "enum", "name": "Foo", "symbols": ["A", "B", "C", "D"] }
This would be encoded by an int between zero and three, with zero indicating “A”, and 3 indicating “D”.
Arrays
Arrays are encoded as a series of blocks. Each block consists of a long count value, followed by that many array items. A block with count zero indicates the end of the array. Each item is encoded per the array’s item schema.
If a block’s count is negative, its absolute value is used, and the count is followed immediately by a long block size indicating the number of bytes in the block. This block size permits fast skipping through data, e.g., when projecting a record to a subset of its fields.
For example, the array schema
{"type": "array", "items": "long"}
an array containing the items 3 and 27 could be encoded as the long value 2 (encoded as hex 04) followed by long values 3 and 27 (encoded as hex 06 36) terminated by zero:
04 06 36 00
The blocked representation permits one to read and write arrays larger than can be buffered in memory, since one can start writing items without knowing the full length of the array.
Maps
Maps are encoded as a series of blocks. Each block consists of a long count value, followed by that many key/value pairs. A block with count zero indicates the end of the map. Each item is encoded per the map’s value schema.
If a block’s count is negative, its absolute value is used, and the count is followed immediately by a long block size indicating the number of bytes in the block. This block size permits fast skipping through data, e.g., when projecting a record to a subset of its fields.
The blocked representation permits one to read and write maps larger than can be buffered in memory, since one can start writing items without knowing the full length of the map.
Unions
A union is encoded by first writing a long value indicating the zero-based position within the union of the schema of its value. The value is then encoded per the indicated schema within the union.
For example, the union schema ["string","null"] would encode:
-
null as the integer 1 (the index of “null” in the union, encoded as hex 02):
02
-
the string “a” as zero (the index of “string” in the union), followed by the serialized string:
02 02 61
Fixed
Fixed instances are encoded using the number of bytes declared in the schema.
JSON Encoding
Except for unions, the JSON encoding is the same as is used to encode field default values.
The value of a union is encoded in JSON as follows:
- if its type is null, then it is encoded as a JSON null;
- otherwise it is encoded as a JSON object with one name/value pair whose name is the type’s name and whose value is the recursively encoded value. For Avro’s named types (record, fixed or enum) the user-specified name is used, for other types the type name is used.
For example, the union schema ["null","string","Foo"], where Foo is a record name, would encode:
- null as null;
- the string “a” as
{"string": "a"} and
- a Foo instance as
{"Foo": {...}}, where {...} indicates the JSON encoding of a Foo instance.
Note that a schema is still required to correctly process JSON-encoded data. For example, the JSON encoding does not distinguish between int and long, float and double, records and maps, enums and strings, etc.
Sort Order
Avro defines a standard sort order for data. This permits data written by one system to be efficiently sorted by another system. This can be an important optimization, as sort order comparisons are sometimes the most frequent per-object operation. Note also that Avro binary-encoded data can be efficiently ordered without deserializing it to objects.
Data items may only be compared if they have identical schemas. Pairwise comparisons are implemented recursively with a depth-first, left-to-right traversal of the schema. The first mismatch encountered determines the order of the items.
Two items with the same schema are compared according to the following rules.
- null data is always equal.
- boolean data is ordered with false before true.
- int, long, float and double data is ordered by ascending numeric value.
- bytes and fixed data are compared lexicographically by unsigned 8-bit values.
- string data is compared lexicographically by Unicode code point. Note that since UTF-8 is used as the binary encoding for strings, sorting of bytes and string binary data is identical.
- array data is compared lexicographically by element.
- enum data is ordered by the symbol’s position in the enum schema. For example, an enum whose symbols are
["z", "a"] would sort “z” values before “a” values.
- union data is first ordered by the branch within the union, and, within that, by the type of the branch. For example, an
["int", "string"] union would order all int values before all string values, with the ints and strings themselves ordered as defined above.
- record data is ordered lexicographically by field. If a field specifies that its order is:
- “ascending”, then the order of its values is unaltered.
- “descending”, then the order of its values is reversed.
- “ignore”, then its values are ignored when sorting.
- map data may not be compared. It is an error to attempt to compare data containing maps unless those maps are in an
"order":"ignore" record field.
Object Container Files
Avro includes a simple object container file format. A file has a schema, and all objects stored in the file must be written according to that schema, using binary encoding. Objects are stored in blocks that may be compressed. Syncronization markers are used between blocks to permit efficient splitting of files for MapReduce processing.
Files may include arbitrary user-specified metadata.
A file consists of:
- A file header, followed by
- one or more file data blocks.
A file header consists of:
- Four bytes, ASCII ‘O’, ‘b’, ‘j’, followed by 1.
- file metadata, including the schema.
- The 16-byte, randomly-generated sync marker for this file.
File metadata consists of:
A long indicating the number of metadata key/value pairs.
For each pair, a string key and bytes value.
All metadata properties that start with “avro.” are reserved. The following file metadata properties are currently used:
- avro.schema contains the schema of objects stored in the file, as JSON data (required).
- avro.codec the name of the compression codec used to compress blocks, as a string. Implementations are required to support the following codecs: “null” and “deflate”. If codec is absent, it is assumed to be “null”. The codecs are described with more detail below.
A file header is thus described by the following schema:
{"type": "record", "name": "org.apache.avro.file.Header",
"fields" : [
{"name": "magic", "type": {"type": "fixed", "name": "Magic", "size": 4}},
{"name": "meta", "type": {"type": "map", "values": "bytes"}},
{"name": "sync", "type": {"type": "fixed", "name": "Sync", "size": 16}}
]
}
A file data block consists of:
- A long indicating the count of objects in this block.
- A long indicating the size in bytes of the serialized objects in the current block, after any codec is applied
- The serialized objects. If a codec is specified, this is compressed by that codec.
- The file’s 16-byte sync marker.
Thus, each block’s binary data can be efficiently extracted or skipped without deserializing the contents. The combination of block size, object counts, and sync markers enable detection of corrupt blocks and help ensure data integrity.
Required Codecs
null
The “null” codec simply passes through data uncompressed.
deflate
The “deflate” codec writes the data block using the deflate algorithm as specified in RFC 1951, and typically implemented using the zlib library. Note that this format (unlike the “zlib format” in RFC 1950) does not have a checksum.
Optional Codecs
snappy
The “snappy” codec uses Google’s Snappy compression library. Each compressed block is followed by the 4-byte, big-endian CRC32 checksum.
Protocol Declaration
Avro protocols describe RPC interfaces. Like schemas, they are defined with JSON text.
A protocol is a JSON object with the following attributes:
- protocol, a string, the name of the protocol (required);
- namespace, an optional string that qualifies the name;
- doc, an optional string describing this protocol;
- types, an optional list of definitions of named types (records, enums, fixed and errors). An error definition is just like a record definition except it uses “error” instead of “record”. Note that forward references to named types are not permitted.
- messages, an optional JSON object whose keys are message names and whose values are objects whose attributes are described below. No two messages may have the same name.
The name and namespace qualification rules defined for schema objects apply to protocols as well.
Messages
A message has attributes:
- a doc, an optional description of the message,
- a request, a list of named, typed parameter schemas (this has the same form as the fields of a record declaration);
- a response schema;
- an optional union of declared error schemas. The effective union has “string” prepended to the declared union, to permit transmission of undeclared “system” errors. For example, if the declared error union is
["AccessError"], then the effective union is ["string", "AccessError"]. When no errors are declared, the effective error union is ["string"]. Errors are serialized using the effective union; however, a protocol’s JSON declaration contains only the declared union.
- an optional one-way boolean parameter.
A request parameter list is processed equivalently to an anonymous record. Since record field lists may vary between reader and writer, request parameters may also differ between the caller and responder, and such differences are resolved in the same manner as record field differences.
The one-way parameter may only be true when the response type is "null" and no errors are listed.
Sample Protocol
For example, one may define a simple HelloWorld protocol with:
{
"namespace": "com.acme",
"protocol": "HelloWorld",
"doc": "Protocol Greetings",
"types": [
{"name": "Greeting", "type": "record", "fields": [
{"name": "message", "type": "string"}]},
{"name": "Curse", "type": "error", "fields": [
{"name": "message", "type": "string"}]}
],
"messages": {
"hello": {
"doc": "Say hello.",
"request": [{"name": "greeting", "type": "Greeting" }],
"response": "Greeting",
"errors": ["Curse"]
}
}
}
Message Transport
Messages may be transmitted via different transport mechanisms.
To the transport, a message is an opaque byte sequence.
A transport is a system that supports:
- transmission of request messages
- receipt of corresponding response messages
Servers may send a response message back to the client corresponding to a request message. The mechanism of correspondance is transport-specific. For example, in HTTP it is implicit, since HTTP directly supports requests and responses. But a transport that multiplexes many client threads over a single socket would need to tag messages with unique identifiers.
Transports may be either stateless or stateful. In a stateless transport, messaging assumes no established connection state, while stateful transports establish connections that may be used for multiple messages. This distinction is discussed further in the handshake section below.
HTTP as Transport
When HTTP is used as a transport, each Avro message exchange is an HTTP request/response pair. All messages of an Avro protocol should share a single URL at an HTTP server. Other protocols may also use that URL. Both normal and error Avro response messages should use the 200 (OK) response code. The chunked encoding may be used for requests and responses, but, regardless the Avro request and response are the entire content of an HTTP request and response. The HTTP Content-Type of requests and responses should be specified as “avro/binary”. Requests should be made using the POST method.
HTTP is used by Avro as a stateless transport.
Message Framing
Avro messages are framed as a list of buffers.
Framing is a layer between messages and the transport. It exists to optimize certain operations.
The format of framed message data is:
- a series of buffers, where each buffer consists of:
- a four-byte, big-endian buffer length, followed by
- that many bytes of buffer data.
- A message is always terminated by a zero-lenghted buffer.
Framing is transparent to request and response message formats (described below). Any message may be presented as a single or multiple buffers.
Framing can permit readers to more efficiently get different buffers from different sources and for writers to more efficiently store different buffers to different destinations. In particular, it can reduce the number of times large binary objects are copied. For example, if an RPC parameter consists of a megabyte of file data, that data can be copied directly to a socket from a file descriptor, and, on the other end, it could be written directly to a file descriptor, never entering user space.
A simple, recommended, framing policy is for writers to create a new segment whenever a single binary object is written that is larger than a normal output buffer. Small objects are then appended in buffers, while larger objects are written as their own buffers. When a reader then tries to read a large object the runtime can hand it an entire buffer directly, without having to copy it.
Handshake
The purpose of the handshake is to ensure that the client and the server have each other’s protocol definition, so that the client can correctly deserialize responses, and the server can correctly deserialize requests. Both clients and servers should maintain a cache of recently seen protocols, so that, in most cases, a handshake will be completed without extra round-trip network exchanges or the transmission of full protocol text.
RPC requests and responses may not be processed until a handshake has been completed. With a stateless transport, all requests and responses are prefixed by handshakes. With a stateful transport, handshakes are only attached to requests and responses until a successful handshake response has been returned over a connection. After this, request and response payloads are sent without handshakes for the lifetime of that connection.
The handshake process uses the following record schemas:
{
"type": "record",
"name": "HandshakeRequest", "namespace":"org.apache.avro.ipc",
"fields": [
{"name": "clientHash",
"type": {"type": "fixed", "name": "MD5", "size": 16}},
{"name": "clientProtocol", "type": ["null", "string"]},
{"name": "serverHash", "type": "MD5"},
{"name": "meta", "type": ["null", {"type": "map", "values": "bytes"}]}
]
}
{
"type": "record",
"name": "HandshakeResponse", "namespace": "org.apache.avro.ipc",
"fields": [
{"name": "match",
"type": {"type": "enum", "name": "HandshakeMatch",
"symbols": ["BOTH", "CLIENT", "NONE"]}},
{"name": "serverProtocol",
"type": ["null", "string"]},
{"name": "serverHash",
"type": ["null", {"type": "fixed", "name": "MD5", "size": 16}]},
{"name": "meta",
"type": ["null", {"type": "map", "values": "bytes"}]}
]
}
- A client first prefixes each request with a
HandshakeRequest containing just the hash of its protocol and of the server’s protocol (clientHash!=null, clientProtocol=null, serverHash!=null), where the hashes are 128-bit MD5 hashes of the JSON protocol text. If a client has never connected to a given server, it sends its hash as a guess of the server’s hash, otherwise it sends the hash that it previously obtained from this server.
The server responds with a HandshakeResponse containing one of:
-
match=BOTH, serverProtocol=null, serverHash=null if the client sent the valid hash of the server’s protocol and the server knows what protocol corresponds to the client’s hash. In this case, the request is complete and the response data immediately follows the HandshakeResponse.
-
match=CLIENT, serverProtocol!=null, serverHash!=null if the server has previously seen the client’s protocol, but the client sent an incorrect hash of the server’s protocol. The request is complete and the response data immediately follows the HandshakeResponse. The client must use the returned protocol to process the response and should also cache that protocol and its hash for future interactions with this server.
-
match=NONE if the server has not previously seen the client’s protocol. The serverHash and serverProtocol may also be non-null if the server’s protocol hash was incorrect.
In this case the client must then re-submit its request with its protocol text (clientHash!=null, clientProtocol!=null, serverHash!=null) and the server should respond with a successful match (match=BOTH, serverProtocol=null, serverHash=null) as above.
The meta field is reserved for future handshake enhancements.
A call consists of a request message paired with its resulting response or error message. Requests and responses contain extensible metadata, and both kinds of messages are framed as described above.
The format of a call request is:
- request metadata, a map with values of type bytes
- the message name, an Avro string, followed by
- the message parameters. Parameters are serialized according to the message’s request declaration.
When a message is declared one-way and a stateful connection has been established by a successful handshake response, no response data is sent. Otherwise the format of the call response is:
- response metadata, a map with values of type bytes
- a one-byte error flag boolean, followed by either:
- if the error flag is false, the message response, serialized per the message’s response schema.
- if the error flag is true, the error, serialized per the message’s effective error union schema.
Schema Resolution
A reader of Avro data, whether from an RPC or a file, can always parse that data because its schema is provided. But that schema may not be exactly the schema that was expected. For example, if the data was written with a different version of the software than it is read, then records may have had fields added or removed. This section specifies how such schema differences should be resolved.
We refer to the schema used to write the data as the writer’s schema, and the schema that the application expects the reader’s schema. Differences between these should be resolved as follows:
- It is an error if the two schemas do not match.
To match, one of the following must hold:
-
both schemas are arrays whose item types match
-
both schemas are maps whose value types match
-
both schemas are enums whose names match
-
both schemas are fixed whose sizes and names match
-
both schemas are records with the same name
-
either schema is a union
-
both schemas have same primitive type
-
the writer’s schema may be promoted to the reader’s as follows:
- int is promotable to long, float, or double
- long is promotable to float or double
- float is promotable to double
- string is promotable to bytes
- bytes is promotable to string
-
if both are records:
- the ordering of fields may be different: fields are matched by name.
- schemas for fields with the same name in both records are resolved recursively.
- if the writer’s record contains a field with a name not present in the reader’s record, the writer’s value for that field is ignored.
- if the reader’s record schema has a field that contains a default value, and writer’s schema does not have a field with the same name, then the reader should use the default value from its field.
- if the reader’s record schema has a field with no default value, and writer’s schema does not have a field with the same name, an error is signalled.
-
if both are enums:
if the writer’s symbol is not present in the reader’s enum, then an error is signalled.
This resolution algorithm is applied recursively to the reader’s and writer’s array item schemas.
This resolution algorithm is applied recursively to the reader’s and writer’s value schemas.
The first schema in the reader’s union that matches the selected writer’s union schema is recursively resolved against it. if none match, an error is signalled.
- if reader’s is a union, but writer’s is not:
The first schema in the reader’s union that matches the writer’s schema is recursively resolved against it. If none match, an error is signalled.
- if writer’s is a union, but reader’s is not:
If the reader’s schema matches the selected writer’s schema, it is recursively resolved against it. If they do not match, an error is signalled.
A schema’s “doc” fields are ignored for the purposes of schema resolution. Hence, the “doc” portion of a schema may be dropped at serialization.
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
27.2 - IDL Language
Introduction
This document defines Avro IDL, an experimental higher-level language for authoring Avro schemata. Before reading this document, you should have familiarity with the concepts of schemata and protocols, as well as the various primitive and complex types available in Avro.
N.B. This feature is considered experimental in the current version of Avro and the language has not been finalized. Although major changes are unlikely, some syntax may change in future versions of Avro.
Overview
Purpose
The aim of the Avro IDL language is to enable developers to author schemata in a way that feels more similar to common programming languages like Java, C++, or Python. Additionally, the Avro IDL language may feel more familiar for those users who have previously used the interface description languages (IDLs) in other frameworks like Thrift, Protocol Buffers, or CORBA.
Usage
Each Avro IDL file defines a single Avro Protocol, and thus generates as its output a JSON-format Avro Protocol file with extension .avpr.
To convert a .avdl file into a .avpr file, it must be processed by the idl tool. For example:
$ java -jar avroj-tools-1.4.0.jar idl src/test/idl/input/namespaces.avdl /tmp/namespaces.avpr
$ head /tmp/namespaces.avpr
{
"protocol" : "TestNamespace",
"namespace" : "avro.test.protocol",
The idl tool can also process input to and from stdin and stdout. See idl --help for full usage information.
Defining a Protocol in Avro IDL
An Avro IDL file consists of exactly one protocol definition. The minimal protocol is defined by the following code:
This is equivalent to (and generates) the following JSON protocol definition:
{
"protocol" : "MyProtocol",
"types" : [ ],
"messages" : {
}
}
The namespace of the protocol may be changed using the @namespace annotation:
@namespace("mynamespace")
protocol MyProtocol {
}
This notation is used throughout Avro IDL as a way of specifying properties for the annotated element, as will be described later in this document.
Protocols in Avro IDL can contain the following items:
- Imports of external protocol and schema files.
- Definitions of named schemata, including records, errors, enums, and fixeds.
- Definitions of RPC messages
Imports
Files may be imported in one of three formats:
-
An IDL file may be imported with a statement like:
import idl "foo.avdl";
-
A JSON protocol file may be imported with a statement like:
import protocol "foo.avpr";
-
A JSON schema file may be imported with a statement like:
import schema "foo.avsc";
Messages and types in the imported file are added to this file’s protocol.
Imported file names are resolved relative to the current IDL file.
Defining an Enumeration
Enums are defined in Avro IDL using a syntax similar to C or Java:
enum Suit {
SPADES, DIAMONDS, CLUBS, HEARTS
}
Note that, unlike the JSON format, anonymous enums cannot be defined.
Defining a Fixed Length Field
Fixed fields are defined using the following syntax:
fixed MD5(16);
This example defines a fixed-length type called MD5 which contains 16 bytes.
Defining Records and Errors
Records are defined in Avro IDL using a syntax similar to a struct definition in C:
record Employee {
string name;
boolean active = true;
long salary;
}
The above example defines a record with the name “Employee” with three fields.
To define an error, simply use the keyword error instead of record. For example:
error Kaboom {
string explanation;
int result_code = -1;
}
Each field in a record or error consists of a type and a name, optional property annotations and an optional default value.
A type reference in Avro IDL must be one of:
- A primitive type
- A logical type
- A named schema defined prior to this usage in the same Protocol
- A complex type (array, map, or union)
Primitive Types
The primitive types supported by Avro IDL are the same as those supported by Avro’s JSON format. This list includes int, long, string, boolean, float, double, null, and bytes.
References to Named Schemata
If a named schema has already been defined in the same Avro IDL file, it may be referenced by name as if it were a primitive type:
record Card {
Suit suit; // refers to the enum Card defined above
int number;
}
Default Values
Default values for fields may be optionally specified by using an equals sign after the field name followed by a JSON expression indicating the default value. This JSON is interpreted as described in the spec.
Complex Types
Arrays
Array types are written in a manner that will seem familiar to C++ or Java programmers. An array of any type t is denoted array<t>. For example, an array of strings is denoted array<string>, and a multidimensional array of Foo records would be array<array<Foo>>.
Maps
Map types are written similarly to array types. An array that contains values of type t is written map<t>. As in the JSON schema format, all maps contain string-type keys.
Unions
Union types are denoted as union { typeA, typeB, typeC, … }. For example, this record contains a string field that is optional (unioned with null):
record RecordWithUnion {
union { null, string } optionalString;
}
Note that the same restrictions apply to Avro IDL unions as apply to unions defined in the JSON format; namely, a record may not contain multiple elements of the same type.
Defining RPC Messages
The syntax to define an RPC message within a Avro IDL protocol is similar to the syntax for a method declaration within a C header file or a Java interface. To define an RPC message add which takes two arguments named foo and bar, returning an int, simply include the following definition within the protocol:
int add(int foo, int bar = 0);
Message arguments, like record fields, may specify default values.
To define a message with no response, you may use the alias void, equivalent to the Avro null type:
void logMessage(string message);
If you have previously defined an error type within the same protocol, you may declare that a message can throw this error using the syntax:
void goKaboom() throws Kaboom;
To define a one-way message, use the keyword oneway after the parameter list, for example:
void fireAndForget(string message) oneway;
Other Language Features
All Java-style comments are supported within a Avro IDL file. Any text following // on a line is ignored, as is any text between /* and */, possibly spanning multiple lines.
Comments that begin with /** are used as the documentation string for the type or field definition that follows the comment.
Escaping Identifiers
Occasionally, one will need to use a reserved language keyword as an identifier. In order to do so, backticks (`) may be used to escape the identifier. For example, to define a message with the literal name error, you may write:
This syntax is allowed anywhere an identifier is expected.
Annotations for Ordering and Namespaces
Java-style annotations may be used to add additional properties to types and fields throughout Avro IDL.
For example, to specify the sort order of a field within a record, one may use the @order annotation before the field name as follows:
record MyRecord {
string @order("ascending") myAscendingSortField;
string @order("descending") myDescendingField;
string @order("ignore") myIgnoredField;
}
A field’s type may also be preceded by annotations, e.g.:
record MyRecord {
@java-class("java.util.ArrayList") array<string> myStrings;
}
Similarly, a @namespace annotation may be used to modify the namespace when defining a named schema. For example:
@namespace("org.apache.avro.firstNamespace")
protocol MyProto {
@namespace("org.apache.avro.someOtherNamespace")
record Foo {}
record Bar {}
}
will define a protocol in the firstNamespace namespace. The record Foo will be defined in someOtherNamespace and Bar will be defined in firstNamespace as it inherits its default from its container.
Type and field aliases are specified with the @aliases annotation as follows:
@aliases(["org.old.OldRecord", "org.ancient.AncientRecord"])
record MyRecord {
string @aliases(["oldField", "ancientField"]) myNewField;
}
Complete Example
The following is a complete example of a Avro IDL file that shows most of the above features:
/**
* An example protocol in Avro IDL
*/
@namespace("org.apache.avro.test")
protocol Simple {
@aliases(["org.foo.KindOf"])
enum Kind {
FOO,
BAR, // the bar enum value
BAZ
}
fixed MD5(16);
record TestRecord {
@order("ignore")
string name;
@order("descending")
Kind kind;
MD5 hash;
union { MD5, null} @aliases(["hash"]) nullableHash;
array<long> arrayOfLongs;
}
error TestError {
string message;
}
string hello(string greeting);
TestRecord echo(TestRecord `record`);
int add(int arg1, int arg2);
bytes echoBytes(bytes data);
void `error`() throws TestError;
void ping() oneway;
}
Additional examples may be found in the Avro source tree under the src/test/idl/input directory.
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
27.3 - SASL profile
Introduction
SASL (RFC 2222) provides a framework for authentication and security of network protocols. Each protocol that uses SASL is meant to define a SASL profile. This document provides a SASL profile for connection-based Avro RPC.
Overview
SASL negotiation proceeds as a series of message interactions over a connection between a client and server using a selected SASL mechanism. The client starts this negotiation by sending its chosen mechanism name with an initial (possibly empty) message. Negotiation proceeds with the exchange of messages until either side indicates success or failure. The content of the messages is mechanism-specific. If the negotiation succeeds, then the session can proceed over the connection, otherwise it must be abandoned.
Some mechanisms continue to process session data after negotiation (e.g., encrypting it), while some specify that further session data is transmitted unmodified.
Negotiation
Commands
Avro SASL negotiation uses four one-byte commands.
- 0: START Used in a client’s initial message.
- 1: CONTINUE Used while negotiation is ongoing.
- 2: FAIL Terminates negotiation unsuccessfully.
- 3: COMPLETE Terminates negotiation successfully.
The format of a START message is:
| 0 | 4-byte mechanism name length | mechanism name | 4-byte payload length | payload data |
The format of a CONTINUE message is:
| 1 | 4-byte payload length | payload data |
The format of a FAIL message is:
| 2 | 4-byte message length | UTF-8 message |
The format of a COMPLETE message is:
| 3 | 4-byte payload length | payload data |
Process
Negotiation is initiated by a client sending a START command containing the client’s chosen mechanism name and any mechanism-specific payload data.
The server and client then interchange some number (possibly zero) of CONTINUE messages. Each message contains payload data that is processed by the security mechanism to generate the next message.
Once either the client or server send a FAIL message then negotiation has failed. UTF-8-encoded text is included in the failure message. Once either a FAIL message has been sent or received, or any other error occurs in the negotiation, further communication on this connection must cease.
Once either the client or server send a COMPLETE message then negotiation has completed successfully. Session data may now be transmitted over the connection until it is closed by either side.
Session Data
If no SASL QOP (quality of protection) is negotiated, then all subsequent writes to/reads over this connection are written/read unmodified. In particular, messages use Avro framing, and are of the form:
| 4-byte frame length | frame data | ... | 4 zero bytes |
If a SASL QOP is negotiated, then it must be used by the connection for all subsequent messages. This is done by wrapping each non-empty frame written using the security mechanism and unwrapping each non-empty frame read. The length written in each non-empty frame is the length of the wrapped data. Complete frames must be passed to the security mechanism for unwrapping. Unwrapped data is then passed to the application as the content of the frame.
If at any point processing fails due to wrapping, unwrapping or framing errors, then all further communication on this connection must cease.
Anonymous Mechanism
The SASL anonymous mechanism (RFC 2245) is quite simple to implement. In particular, an initial anonymous request may be prefixed by the following static sequence:
| 0 | 0009 | ANONYMOUS | 0000 |
If a server uses the anonymous mechanism, it should check that the mechanism name in the start message prefixing the first request received is ‘ANONYMOUS’, then simply prefix its initial response with a COMPLETE message of:
| 3 | 0000 |
If an anonymous server recieves some other mechanism name, then it may respond with a FAIL message as simple as:
| 2 | 0000 |
Note that the anonymous mechanism need add no additional round-trip messages between client and server. The START message can be piggybacked on the initial request and the COMPLETE or FAIL message can be piggybacked on the initial response.
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
27.4 - Wiki
The Wiki page can be found here.
27.5 - FAQ
What is Avro?
Avro is a data serialization system.
Avro provides:
- Rich data structures.
- A compact, fast, binary data format.
- A container file, to store persistent data.
- Remote procedure call (RPC).
- Simple integration with dynamic languages. Code generation is not required to read or write data files nor to use or implement RPC protocols. Code generation as an optional optimization, only worth implementing for statically typed languages.
How can I get started quickly with Avro?
Check out the Quick Start Guide.
How do I statically compile a schema or protocol into generated code?
In Java
- Add the avro jar, the jackson-mapper-asl.jar and jackson-core-asl.jar to your CLASSPATH.
- Run java org.apache.avro.specific.SpecificCompiler .
This appears to be out of date, the SpecificCompiler requires two arguments, presumably an input and and output file, but it isn’t clear that this does.
Or use the Schema or Protocol Ant tasks. Avro’s build.xml provides examples of how these are used.
Lastly, you can also use the “avro-tools” jar which ships with an Avro release. Just use the “compile (schema|protocol)” command.
How are Strings represented in Java?
They use org.apache.avro.util.Utf8, not java.lang.String.
More generally, how do Avro types map to Java types?
The mappings are documented in the package javadoc for generic, specific and reflect API.
From Doug Cutting:
HDFS splits files into blocks, and mapreduce runs a map task for each block. When the task starts, it needs to be able to seek into the file to the start of the block process through the block’s end. If the file were, e.g., a gzip file, this would not be possible, since gzip files must be decompressed from the start. One cannot seek into the middle of a gzip file and start decompressing. So Hadoop’s SequenceFile places a marker periodically (~64k) in the file at record and compression boundaries, where processing can be sensibly started. Then, when a map task starts processing an HDFS block, it finds the first marker after the block’s start and continues through the first marker in the next block of the file. This requires a bit of non-local access (~0.1%). Avro’s data file uses the same method as SequenceFile.
Why isn’t every value in Avro nullable?
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 of every value stored and
corresponding computational costs to create this bit on write and
interpret it on read. These costs are wasted when values may not in
fact be null, as is the case in many datasets. In Avro such costs are
only paid when values may actually be null.
Also, allowing values to be null is a well-known source of errors. In
Avro, a value declared as non-null will always be non-null and programs
need not test for null values when processing it nor will they ever fail
for lack of such tests.
Tony Hoare calls his invention of null references his “Billion Dollar
Mistake”.
http://qconlondon.com/london-2009/presentation/Null+References:+The+Billion+Dollar+Mistake
Also note that in some programming languages not all values are permitted to be null. For example, in Java, values of type boolean, byte, short, char, int, float, long, and double may not be null.
How can I serialize directly to/from a byte array?
As pointed out in the specification, Avro data should always be stored with its schema. The Avro provided classes DataFileWriter, DataFileReader, and DataFileStream all ensure this by serializing the Schema in a container header. In some special cases, such as when implementing a new storage system or writing unit tests, you may need to write and read directly with the bare Avro serialized values. The following examples use code generated for Java from an Avro Schema, specifically the User example from the Quickstart guide.
Serializing to a byte array
This example takes a User object and returns a newly allocated byte array with the Avro serialization of that user.
ByteArrayOutputStream out = new ByteArrayOutputStream();
BinaryEncoder encoder = EncoderFactory.get().binaryEncoder(out, null);
DatumWriter<User> writer = new SpecificDatumWriter<User>(User.getClassSchema());
writer.write(user, encoder);
encoder.flush();
out.close();
byte[] serializedBytes = out.toByteArray();
Deserializing from a byte array
This example takes a byte array containing the Avro serialization of a user and returns a User object.SpecificDatumReader reader = new SpecificDatumReader(User.getClassSchema());
Decoder decoder = DecoderFactory.get().binaryDecoder(bytes, null);
User user = reader.read(null, decoder);
Note: If you are serializing or deserializing in a loop or as a method, you should be reusing objects, readers and/or writers. Check the JavaDoc to see how to reuse the objects.
28 - Apache Avro™ 1.5.0 Documentation
Introduction
Apache Avro™ is a data serialization system.
Avro provides:
- Rich data structures.
- A compact, fast, binary data format.
- A container file, to store persistent data.
- Remote procedure call (RPC).
- Simple integration with dynamic languages. Code generation is not required to read or write data files nor to use or implement RPC protocols. Code generation as an optional optimization, only worth implementing for statically typed languages.
Schemas
Avro relies on schemas. When Avro data is read, the schema used when writing it is always present. This permits each datum to be written with no per-value overheads, making serialization both fast and small. This also facilitates use with dynamic, scripting languages, since data, together with its schema, is fully self-describing.
When Avro data is stored in a file, its schema is stored with it, so that files may be processed later by any program. If the program reading the data expects a different schema this can be easily resolved, since both schemas are present.
When Avro is used in RPC, the client and server exchange schemas in the connection handshake. (This can be optimized so that, for most calls, no schemas are actually transmitted.) Since both client and server both have the other’s full schema, correspondence between same named fields, missing fields, extra fields, etc. can all be easily resolved.
Avro schemas are defined with JSON . This facilitates implementation in languages that already have JSON libraries.
Comparison with other systems
Avro provides functionality similar to systems such as Thrift, Protocol Buffers, etc. Avro differs from these systems in the following fundamental aspects.
- Dynamic typing: Avro does not require that code be generated. Data is always accompanied by a schema that permits full processing of that data without code generation, static datatypes, etc. This facilitates construction of generic data-processing systems and languages.
- Untagged data: Since the schema is present when data is read, considerably less type information need be encoded with data, resulting in smaller serialization size.
- No manually-assigned field IDs: When a schema changes, both the old and new schema are always present when processing data, so differences may be resolved symbolically, using field names.
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
28.1 - Specification
Introduction
This document defines Apache Avro. It is intended to be the authoritative specification. Implementations of Avro must adhere to this document.
Schema Declaration
A Schema is represented in JSON by one of:
- A JSON string, naming a defined type.
- A JSON object, of the form:
{"type": "typeName" ...attributes...}
where typeName is either a primitive or derived type name, as defined below. Attributes not defined in this document are permitted as metadata, but must not affect the format of serialized data.
- A JSON array, representing a union of embedded types.
Primitive Types
The set of primitive type names is:
- null: no value
- boolean: a binary value
- int: 32-bit signed integer
- long: 64-bit signed integer
- float: single precision (32-bit) IEEE 754 floating-point number
- double: double precision (64-bit) IEEE 754 floating-point number
- bytes: sequence of 8-bit unsigned bytes
- string: unicode character sequence
Primitive types have no specified attributes.
Primitive type names are also defined type names. Thus, for example, the schema “string” is equivalent to:
Complex Types
Avro supports six kinds of complex types: records, enums, arrays, maps, unions and fixed.
Records
Records use the type name “record” and support three attributes:
- name: a JSON string providing the name of the record (required).
- namespace, a JSON string that qualifies the name;
- doc: a JSON string providing documentation to the user of this schema (optional).
- aliases: a JSON array of strings, providing alternate names for this record (optional).
- fields: a JSON array, listing fields (required). Each field is a JSON object with the following attributes:
- name: a JSON string providing the name of the field (required), and
- doc: a JSON string describing this field for users (optional).
- type: A JSON object defining a schema, or a JSON string naming a record definition (required).
- default: A default value for this field, used when reading instances that lack this field (optional). 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. Default values for bytes and fixed fields are JSON strings, where Unicode code points 0-255 are mapped to unsigned 8-bit byte values 0-255.
field default values
| avro type |
json type |
example |
| null |
null |
null |
| boolean |
boolean |
true |
| int,long |
integer |
1 |
| float,double |
number |
1.1 |
| bytes |
string |
"\u00FF" |
| string |
string |
"foo" |
| record |
object |
{"a": 1} |
| enum |
string |
"FOO" |
| array |
array |
[1] |
| map |
object |
{"a": 1} |
| fixed |
string |
"\u00ff" |
- order: specifies how this field impacts sort ordering of this record (optional). Valid values are “ascending” (the default), “descending”, or “ignore”. For more details on how this is used, see the sort order section below.
- aliases: a JSON array of strings, providing alternate names for this field (optional).
For example, a linked-list of 64-bit values may be defined with:
{
"type": "record",
"name": "LongList",
"aliases": ["LinkedLongs"], // old name for this
"fields" : [
{"name": "value", "type": "long"}, // each element has a long
{"name": "next", "type": ["LongList", "null"]} // optional next element
]
}
Enums
Enums use the type name “enum” and support the following attributes:
- name: a JSON string providing the name of the enum (required).
- namespace, a JSON string that qualifies the name ;
- aliases: a JSON array of strings, providing alternate names for this enum (optional).
- doc: a JSON string providing documentation to the user of this schema (optional).
- symbols: a JSON array, listing symbols, as JSON strings (required). All symbols in an enum must be unique; duplicates are prohibited.
For example, playing card suits might be defined with:
{
"type": "enum",
"name": "Suit",
"symbols" : ["SPADES", "HEARTS", "DIAMONDS", "CLUBS"]
}
Arrays
Arrays use the type name “array” and support a single attribute:
- items: the schema of the array’s items.
For example, an array of strings is declared with:
{
"type": "array",
"items" : "string"
}
Maps
Maps use the type name “map” and support one attribute:
- values: the schema of the map’s values.
Map keys are assumed to be strings.
For example, a map from string to long is declared with:
{
"type": "map",
"values" : "long"
}
Unions
Unions, as mentioned above, are represented using JSON arrays. For example, ["string", "null"] declares a schema which may be either a string or null.
Unions may not contain more than one schema with the same type, except for the named types record, fixed and enum. For example, unions containing two array types or two map types are not permitted, but two types with different names are permitted. (Names permit efficient resolution when reading and writing unions.)
Unions may not immediately contain other unions.
Fixed
Fixed uses the type name “fixed” and supports two attributes:
- name: a string naming this fixed (required).
- namespace, a string that qualifies the name;
- aliases: a JSON array of strings, providing alternate names for this enum (optional).
- size: an integer, specifying the number of bytes per value (required).
For example, 16-byte quantity may be declared with:
{"type": "fixed", "size": 16, "name": "md5"}
Names
Record, enums and fixed are named types. Each has a fullname that is composed of two parts; a name and a namespace. Equality of names is defined on the fullname.
The name portion of a fullname, and record field names must:
- start with [A-Za-z_]
- subsequently contain only [A-Za-z0-9_]
A namespace is a dot-separated sequence of such names.
In record, enum and fixed definitions, the fullname is determined in one of the following ways:
A name and namespace are both specified. For example, one might use “name”: “X”, “namespace”: “org.foo” to indicate the fullname org.foo.X.
-
A fullname is specified. If the name specified contains a dot, then it is assumed to be a fullname, and any namespace also specified is ignored. For example, use “name”: “org.foo.X” to indicate the fullname org.foo.X.
-
A name only is specified, i.e., a name that contains no dots. In this case the namespace is taken from the most tightly enclosing schema or protocol. For example, if “name”: “X” is specified, and this occurs within a field of the record definition of org.foo.Y, then the fullname is org.foo.X.
References to previously defined names are as in the latter two cases above: if they contain a dot they are a fullname, if they do not contain a dot, the namespace is the namespace of the enclosing definition.
Primitive type names have no namespace and their names may not be defined in any namespace. A schema may only contain multiple definitions of a fullname if the definitions are equivalent.
Aliases
Named types and fields may have aliases. An implementation may optionally use aliases to map a writer’s schema to the reader’s. This facilitates both schema evolution as well as processing disparate datasets.
Aliases function by re-writing the writer’s schema using aliases from the reader’s schema. For example, if the writer’s schema was named “Foo” and the reader’s schema is named “Bar” and has an alias of “Foo”, then the implementation would act as though “Foo” were named “Bar” when reading. Similarly, if data was written as a record with a field named “x” and is read as a record with a field named “y” with alias “x”, then the implementation would act as though “x” were named “y” when reading.
A type alias may be specified either as a fully namespace-qualified, or relative to the namespace of the name it is an alias for. For example, if a type named “a.b” has aliases of “c” and “x.y”, then the fully qualified names of its aliases are “a.c” and “x.y”.
Data Serialization
Avro data is always serialized with its schema. Files that store Avro data should always also include the schema for that data in the same file. Avro-based remote procedure call (RPC) systems must also guarantee that remote recipients of data have a copy of the schema used to write that data.
Because the schema used to write data is always available when the data is read, Avro data itself is not tagged with type information. The schema is required to parse data.
In general, both serialization and deserialization proceed as a depth-first, left-to-right traversal of the schema, serializing primitive types as they are encountered.
Encodings
Avro specifies two serialization encodings: binary and JSON. Most applications will use the binary encoding, as it is smaller and faster. But, for debugging and web-based applications, the JSON encoding may sometimes be appropriate.
Binary Encoding
Primitive Types
Primitive types are encoded in binary as follows:
- null is written as zero bytes.
- a boolean is written as a single byte whose value is either 0 (false) or 1 (true).
- int and long values are written using variable-length zig-zag coding. Some examples:
| value |
hex |
| 0 |
00 |
| -1 |
01 |
| 1 |
02 |
| -2 |
03 |
| 2 |
04 |
| … |
|
| -64 |
7f |
| 64 |
80 01 |
| … |
|
- a float is written as 4 bytes. The float is converted into a 32-bit integer using a method equivalent to Java’s floatToIntBits and then encoded in little-endian format.
- a double is written as 8 bytes. The double is converted into a 64-bit integer using a method equivalent to Java’s doubleToLongBits and then encoded in little-endian format.
- bytes are encoded as a long followed by that many bytes of data.
- a string is encoded as a long followed by that many bytes of UTF-8 encoded character data.
For example, the three-character string “foo” would be encoded as the long value 3 (encoded as hex 06) followed by the UTF-8 encoding of ‘f’, ‘o’, and ‘o’ (the hex bytes 66 6f 6f):
06 66 6f 6f
Complex Types
Complex types are encoded in binary as follows:
Records
A record is encoded by encoding the values of its fields in the order that they are declared. In other words, a record is encoded as just the concatenation of the encodings of its fields. Field values are encoded per their schema.
For example, the record schema
{
"type": "record",
"name": "test",
"fields" : [
{"name": "a", "type": "long"},
{"name": "b", "type": "string"}
]
}
An instance of this record whose a field has value 27 (encoded as hex 36) and whose b field has value “foo” (encoded as hex bytes 06 66 6f 6f), would be encoded simply as the concatenation of these, namely the hex byte sequence:
36 06 66 6f 6f
Enums
An enum is encoded by a int, representing the zero-based position of the symbol in the schema.
For example, consider the enum:
{"type": "enum", "name": "Foo", "symbols": ["A", "B", "C", "D"] }
This would be encoded by an int between zero and three, with zero indicating “A”, and 3 indicating “D”.
Arrays
Arrays are encoded as a series of blocks. Each block consists of a long count value, followed by that many array items. A block with count zero indicates the end of the array. Each item is encoded per the array’s item schema.
If a block’s count is negative, its absolute value is used, and the count is followed immediately by a long block size indicating the number of bytes in the block. This block size permits fast skipping through data, e.g., when projecting a record to a subset of its fields.
For example, the array schema
{"type": "array", "items": "long"}
an array containing the items 3 and 27 could be encoded as the long value 2 (encoded as hex 04) followed by long values 3 and 27 (encoded as hex 06 36) terminated by zero:
04 06 36 00
The blocked representation permits one to read and write arrays larger than can be buffered in memory, since one can start writing items without knowing the full length of the array.
Maps
Maps are encoded as a series of blocks. Each block consists of a long count value, followed by that many key/value pairs. A block with count zero indicates the end of the map. Each item is encoded per the map’s value schema.
If a block’s count is negative, its absolute value is used, and the count is followed immediately by a long block size indicating the number of bytes in the block. This block size permits fast skipping through data, e.g., when projecting a record to a subset of its fields.
The blocked representation permits one to read and write maps larger than can be buffered in memory, since one can start writing items without knowing the full length of the map.
Unions
A union is encoded by first writing a long value indicating the zero-based position within the union of the schema of its value. The value is then encoded per the indicated schema within the union.
For example, the union schema ["string","null"] would encode:
-
null as the integer 1 (the index of “null” in the union, encoded as hex 02):
02
-
the string “a” as zero (the index of “string” in the union), followed by the serialized string:
02 02 61
Fixed
Fixed instances are encoded using the number of bytes declared in the schema.
JSON Encoding
Except for unions, the JSON encoding is the same as is used to encode field default values.
The value of a union is encoded in JSON as follows:
- if its type is null, then it is encoded as a JSON null;
- otherwise it is encoded as a JSON object with one name/value pair whose name is the type’s name and whose value is the recursively encoded value. For Avro’s named types (record, fixed or enum) the user-specified name is used, for other types the type name is used.
For example, the union schema ["null","string","Foo"], where Foo is a record name, would encode:
- null as null;
- the string “a” as
{"string": "a"} and
- a Foo instance as
{"Foo": {...}}, where {...} indicates the JSON encoding of a Foo instance.
Note that a schema is still required to correctly process JSON-encoded data. For example, the JSON encoding does not distinguish between int and long, float and double, records and maps, enums and strings, etc.
Sort Order
Avro defines a standard sort order for data. This permits data written by one system to be efficiently sorted by another system. This can be an important optimization, as sort order comparisons are sometimes the most frequent per-object operation. Note also that Avro binary-encoded data can be efficiently ordered without deserializing it to objects.
Data items may only be compared if they have identical schemas. Pairwise comparisons are implemented recursively with a depth-first, left-to-right traversal of the schema. The first mismatch encountered determines the order of the items.
Two items with the same schema are compared according to the following rules.
- null data is always equal.
- boolean data is ordered with false before true.
- int, long, float and double data is ordered by ascending numeric value.
- bytes and fixed data are compared lexicographically by unsigned 8-bit values.
- string data is compared lexicographically by Unicode code point. Note that since UTF-8 is used as the binary encoding for strings, sorting of bytes and string binary data is identical.
- array data is compared lexicographically by element.
- enum data is ordered by the symbol’s position in the enum schema. For example, an enum whose symbols are
["z", "a"] would sort “z” values before “a” values.
- union data is first ordered by the branch within the union, and, within that, by the type of the branch. For example, an
["int", "string"] union would order all int values before all string values, with the ints and strings themselves ordered as defined above.
- record data is ordered lexicographically by field. If a field specifies that its order is:
- “ascending”, then the order of its values is unaltered.
- “descending”, then the order of its values is reversed.
- “ignore”, then its values are ignored when sorting.
- map data may not be compared. It is an error to attempt to compare data containing maps unless those maps are in an
"order":"ignore" record field.
Object Container Files
Avro includes a simple object container file format. A file has a schema, and all objects stored in the file must be written according to that schema, using binary encoding. Objects are stored in blocks that may be compressed. Syncronization markers are used between blocks to permit efficient splitting of files for MapReduce processing.
Files may include arbitrary user-specified metadata.
A file consists of:
- A file header, followed by
- one or more file data blocks.
A file header consists of:
- Four bytes, ASCII ‘O’, ‘b’, ‘j’, followed by 1.
- file metadata, including the schema.
- The 16-byte, randomly-generated sync marker for this file.
File metadata consists of:
A long indicating the number of metadata key/value pairs.
For each pair, a string key and bytes value.
All metadata properties that start with “avro.” are reserved. The following file metadata properties are currently used:
- avro.schema contains the schema of objects stored in the file, as JSON data (required).
- avro.codec the name of the compression codec used to compress blocks, as a string. Implementations are required to support the following codecs: “null” and “deflate”. If codec is absent, it is assumed to be “null”. The codecs are described with more detail below.
A file header is thus described by the following schema:
{"type": "record", "name": "org.apache.avro.file.Header",
"fields" : [
{"name": "magic", "type": {"type": "fixed", "name": "Magic", "size": 4}},
{"name": "meta", "type": {"type": "map", "values": "bytes"}},
{"name": "sync", "type": {"type": "fixed", "name": "Sync", "size": 16}}
]
}
A file data block consists of:
- A long indicating the count of objects in this block.
- A long indicating the size in bytes of the serialized objects in the current block, after any codec is applied
- The serialized objects. If a codec is specified, this is compressed by that codec.
- The file’s 16-byte sync marker.
Thus, each block’s binary data can be efficiently extracted or skipped without deserializing the contents. The combination of block size, object counts, and sync markers enable detection of corrupt blocks and help ensure data integrity.
Required Codecs
null
The “null” codec simply passes through data uncompressed.
deflate
The “deflate” codec writes the data block using the deflate algorithm as specified in RFC 1951, and typically implemented using the zlib library. Note that this format (unlike the “zlib format” in RFC 1950) does not have a checksum.
Protocol Declaration
Avro protocols describe RPC interfaces. Like schemas, they are defined with JSON text.
A protocol is a JSON object with the following attributes:
- protocol, a string, the name of the protocol (required);
- namespace, an optional string that qualifies the name;
- doc, an optional string describing this protocol;
- types, an optional list of definitions of named types (records, enums, fixed and errors). An error definition is just like a record definition except it uses “error” instead of “record”. Note that forward references to named types are not permitted.
- messages, an optional JSON object whose keys are message names and whose values are objects whose attributes are described below. No two messages may have the same name.
The name and namespace qualification rules defined for schema objects apply to protocols as well.
Messages
A message has attributes:
- a doc, an optional description of the message,
- a request, a list of named, typed parameter schemas (this has the same form as the fields of a record declaration);
- a response schema;
- an optional union of declared error schemas. The effective union has “string” prepended to the declared union, to permit transmission of undeclared “system” errors. For example, if the declared error union is
["AccessError"], then the effective union is ["string", "AccessError"]. When no errors are declared, the effective error union is ["string"]. Errors are serialized using the effective union; however, a protocol’s JSON declaration contains only the declared union.
- an optional one-way boolean parameter.
A request parameter list is processed equivalently to an anonymous record. Since record field lists may vary between reader and writer, request parameters may also differ between the caller and responder, and such differences are resolved in the same manner as record field differences.
The one-way parameter may only be true when the response type is "null" and no errors are listed.
Sample Protocol
For example, one may define a simple HelloWorld protocol with:
{
"namespace": "com.acme",
"protocol": "HelloWorld",
"doc": "Protocol Greetings",
"types": [
{"name": "Greeting", "type": "record", "fields": [
{"name": "message", "type": "string"}]},
{"name": "Curse", "type": "error", "fields": [
{"name": "message", "type": "string"}]}
],
"messages": {
"hello": {
"doc": "Say hello.",
"request": [{"name": "greeting", "type": "Greeting" }],
"response": "Greeting",
"errors": ["Curse"]
}
}
}
Message Transport
Messages may be transmitted via different transport mechanisms.
To the transport, a message is an opaque byte sequence.
A transport is a system that supports:
- transmission of request messages
- receipt of corresponding response messages
Servers may send a response message back to the client corresponding to a request message. The mechanism of correspondance is transport-specific. For example, in HTTP it is implicit, since HTTP directly supports requests and responses. But a transport that multiplexes many client threads over a single socket would need to tag messages with unique identifiers.
Transports may be either stateless or stateful. In a stateless transport, messaging assumes no established connection state, while stateful transports establish connections that may be used for multiple messages. This distinction is discussed further in the handshake section below.
HTTP as Transport
When HTTP is used as a transport, each Avro message exchange is an HTTP request/response pair. All messages of an Avro protocol should share a single URL at an HTTP server. Other protocols may also use that URL. Both normal and error Avro response messages should use the 200 (OK) response code. The chunked encoding may be used for requests and responses, but, regardless the Avro request and response are the entire content of an HTTP request and response. The HTTP Content-Type of requests and responses should be specified as “avro/binary”. Requests should be made using the POST method.
HTTP is used by Avro as a stateless transport.
Message Framing
Avro messages are framed as a list of buffers.
Framing is a layer between messages and the transport. It exists to optimize certain operations.
The format of framed message data is:
- a series of buffers, where each buffer consists of:
- a four-byte, big-endian buffer length, followed by
- that many bytes of buffer data.
- A message is always terminated by a zero-lenghted buffer.
Framing is transparent to request and response message formats (described below). Any message may be presented as a single or multiple buffers.
Framing can permit readers to more efficiently get different buffers from different sources and for writers to more efficiently store different buffers to different destinations. In particular, it can reduce the number of times large binary objects are copied. For example, if an RPC parameter consists of a megabyte of file data, that data can be copied directly to a socket from a file descriptor, and, on the other end, it could be written directly to a file descriptor, never entering user space.
A simple, recommended, framing policy is for writers to create a new segment whenever a single binary object is written that is larger than a normal output buffer. Small objects are then appended in buffers, while larger objects are written as their own buffers. When a reader then tries to read a large object the runtime can hand it an entire buffer directly, without having to copy it.
Handshake
The purpose of the handshake is to ensure that the client and the server have each other’s protocol definition, so that the client can correctly deserialize responses, and the server can correctly deserialize requests. Both clients and servers should maintain a cache of recently seen protocols, so that, in most cases, a handshake will be completed without extra round-trip network exchanges or the transmission of full protocol text.
RPC requests and responses may not be processed until a handshake has been completed. With a stateless transport, all requests and responses are prefixed by handshakes. With a stateful transport, handshakes are only attached to requests and responses until a successful handshake response has been returned over a connection. After this, request and response payloads are sent without handshakes for the lifetime of that connection.
The handshake process uses the following record schemas:
{
"type": "record",
"name": "HandshakeRequest", "namespace":"org.apache.avro.ipc",
"fields": [
{"name": "clientHash",
"type": {"type": "fixed", "name": "MD5", "size": 16}},
{"name": "clientProtocol", "type": ["null", "string"]},
{"name": "serverHash", "type": "MD5"},
{"name": "meta", "type": ["null", {"type": "map", "values": "bytes"}]}
]
}
{
"type": "record",
"name": "HandshakeResponse", "namespace": "org.apache.avro.ipc",
"fields": [
{"name": "match",
"type": {"type": "enum", "name": "HandshakeMatch",
"symbols": ["BOTH", "CLIENT", "NONE"]}},
{"name": "serverProtocol",
"type": ["null", "string"]},
{"name": "serverHash",
"type": ["null", {"type": "fixed", "name": "MD5", "size": 16}]},
{"name": "meta",
"type": ["null", {"type": "map", "values": "bytes"}]}
]
}
- A client first prefixes each request with a
HandshakeRequest containing just the hash of its protocol and of the server’s protocol (clientHash!=null, clientProtocol=null, serverHash!=null), where the hashes are 128-bit MD5 hashes of the JSON protocol text. If a client has never connected to a given server, it sends its hash as a guess of the server’s hash, otherwise it sends the hash that it previously obtained from this server.
The server responds with a HandshakeResponse containing one of:
-
match=BOTH, serverProtocol=null, serverHash=null if the client sent the valid hash of the server’s protocol and the server knows what protocol corresponds to the client’s hash. In this case, the request is complete and the response data immediately follows the HandshakeResponse.
-
match=CLIENT, serverProtocol!=null, serverHash!=null if the server has previously seen the client’s protocol, but the client sent an incorrect hash of the server’s protocol. The request is complete and the response data immediately follows the HandshakeResponse. The client must use the returned protocol to process the response and should also cache that protocol and its hash for future interactions with this server.
-
match=NONE if the server has not previously seen the client’s protocol. The serverHash and serverProtocol may also be non-null if the server’s protocol hash was incorrect.
In this case the client must then re-submit its request with its protocol text (clientHash!=null, clientProtocol!=null, serverHash!=null) and the server should respond with a successful match (match=BOTH, serverProtocol=null, serverHash=null) as above.
The meta field is reserved for future handshake enhancements.
A call consists of a request message paired with its resulting response or error message. Requests and responses contain extensible metadata, and both kinds of messages are framed as described above.
The format of a call request is:
- request metadata, a map with values of type bytes
- the message name, an Avro string, followed by
- the message parameters. Parameters are serialized according to the message’s request declaration.
When a message is declared one-way and a stateful connection has been established by a successful handshake response, no response data is sent. Otherwise the format of the call response is:
- response metadata, a map with values of type bytes
- a one-byte error flag boolean, followed by either:
- if the error flag is false, the message response, serialized per the message’s response schema.
- if the error flag is true, the error, serialized per the message’s effective error union schema.
Schema Resolution
A reader of Avro data, whether from an RPC or a file, can always parse that data because its schema is provided. But that schema may not be exactly the schema that was expected. For example, if the data was written with a different version of the software than it is read, then records may have had fields added or removed. This section specifies how such schema differences should be resolved.
We refer to the schema used to write the data as the writer’s schema, and the schema that the application expects the reader’s schema. Differences between these should be resolved as follows:
- It is an error if the two schemas do not match.
To match, one of the following must hold:
-
both schemas are arrays whose item types match
-
both schemas are maps whose value types match
-
both schemas are enums whose names match
-
both schemas are fixed whose sizes and names match
-
both schemas are records with the same name
-
either schema is a union
-
both schemas have same primitive type
-
the writer’s schema may be promoted to the reader’s as follows:
- int is promotable to long, float, or double
- long is promotable to float or double
- float is promotable to double
- string is promotable to bytes
- bytes is promotable to string
-
if both are records:
- the ordering of fields may be different: fields are matched by name.
- schemas for fields with the same name in both records are resolved recursively.
- if the writer’s record contains a field with a name not present in the reader’s record, the writer’s value for that field is ignored.
- if the reader’s record schema has a field that contains a default value, and writer’s schema does not have a field with the same name, then the reader should use the default value from its field.
- if the reader’s record schema has a field with no default value, and writer’s schema does not have a field with the same name, an error is signalled.
-
if both are enums:
if the writer’s symbol is not present in the reader’s enum, then an error is signalled.
This resolution algorithm is applied recursively to the reader’s and writer’s array item schemas.
This resolution algorithm is applied recursively to the reader’s and writer’s value schemas.
The first schema in the reader’s union that matches the selected writer’s union schema is recursively resolved against it. if none match, an error is signalled.
- if reader’s is a union, but writer’s is not:
The first schema in the reader’s union that matches the writer’s schema is recursively resolved against it. If none match, an error is signalled.
- if writer’s is a union, but reader’s is not:
If the reader’s schema matches the selected writer’s schema, it is recursively resolved against it. If they do not match, an error is signalled.
A schema’s “doc” fields are ignored for the purposes of schema resolution. Hence, the “doc” portion of a schema may be dropped at serialization.
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
28.2 - IDL Language
Introduction
This document defines Avro IDL, an experimental higher-level language for authoring Avro schemata. Before reading this document, you should have familiarity with the concepts of schemata and protocols, as well as the various primitive and complex types available in Avro.
N.B. This feature is considered experimental in the current version of Avro and the language has not been finalized. Although major changes are unlikely, some syntax may change in future versions of Avro.
Overview
Purpose
The aim of the Avro IDL language is to enable developers to author schemata in a way that feels more similar to common programming languages like Java, C++, or Python. Additionally, the Avro IDL language may feel more familiar for those users who have previously used the interface description languages (IDLs) in other frameworks like Thrift, Protocol Buffers, or CORBA.
Usage
Each Avro IDL file defines a single Avro Protocol, and thus generates as its output a JSON-format Avro Protocol file with extension .avpr.
To convert a .avdl file into a .avpr file, it must be processed by the idl tool. For example:
$ java -jar avroj-tools-1.4.0.jar idl src/test/idl/input/namespaces.avdl /tmp/namespaces.avpr
$ head /tmp/namespaces.avpr
{
"protocol" : "TestNamespace",
"namespace" : "avro.test.protocol",
The idl tool can also process input to and from stdin and stdout. See idl --help for full usage information.
Defining a Protocol in Avro IDL
An Avro IDL file consists of exactly one protocol definition. The minimal protocol is defined by the following code:
This is equivalent to (and generates) the following JSON protocol definition:
{
"protocol" : "MyProtocol",
"types" : [ ],
"messages" : {
}
}
The namespace of the protocol may be changed using the @namespace annotation:
@namespace("mynamespace")
protocol MyProtocol {
}
This notation is used throughout Avro IDL as a way of specifying properties for the annotated element, as will be described later in this document.
Protocols in Avro IDL can contain the following items:
- Imports of external protocol and schema files.
- Definitions of named schemata, including records, errors, enums, and fixeds.
- Definitions of RPC messages
Imports
Files may be imported in one of three formats:
-
An IDL file may be imported with a statement like:
import idl "foo.avdl";
-
A JSON protocol file may be imported with a statement like:
import protocol "foo.avpr";
-
A JSON schema file may be imported with a statement like:
import schema "foo.avsc";
Messages and types in the imported file are added to this file’s protocol.
Imported file names are resolved relative to the current IDL file.
Defining an Enumeration
Enums are defined in Avro IDL using a syntax similar to C or Java:
enum Suit {
SPADES, DIAMONDS, CLUBS, HEARTS
}
Note that, unlike the JSON format, anonymous enums cannot be defined.
Defining a Fixed Length Field
Fixed fields are defined using the following syntax:
fixed MD5(16);
This example defines a fixed-length type called MD5 which contains 16 bytes.
Defining Records and Errors
Records are defined in Avro IDL using a syntax similar to a struct definition in C:
record Employee {
string name;
boolean active = true;
long salary;
}
The above example defines a record with the name “Employee” with three fields.
To define an error, simply use the keyword error instead of record. For example:
error Kaboom {
string explanation;
int result_code = -1;
}
Each field in a record or error consists of a type and a name, optional property annotations and an optional default value.
A type reference in Avro IDL must be one of:
- A primitive type
- A logical type
- A named schema defined prior to this usage in the same Protocol
- A complex type (array, map, or union)
Primitive Types
The primitive types supported by Avro IDL are the same as those supported by Avro’s JSON format. This list includes int, long, string, boolean, float, double, null, and bytes.
References to Named Schemata
If a named schema has already been defined in the same Avro IDL file, it may be referenced by name as if it were a primitive type:
record Card {
Suit suit; // refers to the enum Card defined above
int number;
}
Default Values
Default values for fields may be optionally specified by using an equals sign after the field name followed by a JSON expression indicating the default value. This JSON is interpreted as described in the spec.
Complex Types
Arrays
Array types are written in a manner that will seem familiar to C++ or Java programmers. An array of any type t is denoted array<t>. For example, an array of strings is denoted array<string>, and a multidimensional array of Foo records would be array<array<Foo>>.
Maps
Map types are written similarly to array types. An array that contains values of type t is written map<t>. As in the JSON schema format, all maps contain string-type keys.
Unions
Union types are denoted as union { typeA, typeB, typeC, … }. For example, this record contains a string field that is optional (unioned with null):
record RecordWithUnion {
union { null, string } optionalString;
}
Note that the same restrictions apply to Avro IDL unions as apply to unions defined in the JSON format; namely, a record may not contain multiple elements of the same type.
Defining RPC Messages
The syntax to define an RPC message within a Avro IDL protocol is similar to the syntax for a method declaration within a C header file or a Java interface. To define an RPC message add which takes two arguments named foo and bar, returning an int, simply include the following definition within the protocol:
int add(int foo, int bar = 0);
Message arguments, like record fields, may specify default values.
To define a message with no response, you may use the alias void, equivalent to the Avro null type:
void logMessage(string message);
If you have previously defined an error type within the same protocol, you may declare that a message can throw this error using the syntax:
void goKaboom() throws Kaboom;
To define a one-way message, use the keyword oneway after the parameter list, for example:
void fireAndForget(string message) oneway;
Other Language Features
All Java-style comments are supported within a Avro IDL file. Any text following // on a line is ignored, as is any text between /* and */, possibly spanning multiple lines.
Escaping Identifiers
Occasionally, one will need to use a reserved language keyword as an identifier. In order to do so, backticks (`) may be used to escape the identifier. For example, to define a message with the literal name error, you may write:
This syntax is allowed anywhere an identifier is expected.
Annotations for Ordering and Namespaces
Java-style annotations may be used to add additional properties to types and fields throughout Avro IDL.
For example, to specify the sort order of a field within a record, one may use the @order annotation before the field name as follows:
record MyRecord {
string @order("ascending") myAscendingSortField;
string @order("descending") myDescendingField;
string @order("ignore") myIgnoredField;
}
A field’s type may also be preceded by annotations, e.g.:
record MyRecord {
@java-class("java.util.ArrayList") array<string> myStrings;
}
Similarly, a @namespace annotation may be used to modify the namespace when defining a named schema. For example:
@namespace("org.apache.avro.firstNamespace")
protocol MyProto {
@namespace("org.apache.avro.someOtherNamespace")
record Foo {}
record Bar {}
}
will define a protocol in the firstNamespace namespace. The record Foo will be defined in someOtherNamespace and Bar will be defined in firstNamespace as it inherits its default from its container.
Type and field aliases are specified with the @aliases annotation as follows:
@aliases(["org.old.OldRecord", "org.ancient.AncientRecord"])
record MyRecord {
string @aliases(["oldField", "ancientField"]) myNewField;
}
Complete Example
The following is a complete example of a Avro IDL file that shows most of the above features:
/**
* An example protocol in Avro IDL
*/
@namespace("org.apache.avro.test")
protocol Simple {
@aliases(["org.foo.KindOf"])
enum Kind {
FOO,
BAR, // the bar enum value
BAZ
}
fixed MD5(16);
record TestRecord {
@order("ignore")
string name;
@order("descending")
Kind kind;
MD5 hash;
union { MD5, null} @aliases(["hash"]) nullableHash;
array<long> arrayOfLongs;
}
error TestError {
string message;
}
string hello(string greeting);
TestRecord echo(TestRecord `record`);
int add(int arg1, int arg2);
bytes echoBytes(bytes data);
void `error`() throws TestError;
void ping() oneway;
}
Additional examples may be found in the Avro source tree under the src/test/idl/input directory.
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
28.3 - SASL profile
Introduction
SASL (RFC 2222) provides a framework for authentication and security of network protocols. Each protocol that uses SASL is meant to define a SASL profile. This document provides a SASL profile for connection-based Avro RPC.
Overview
SASL negotiation proceeds as a series of message interactions over a connection between a client and server using a selected SASL mechanism. The client starts this negotiation by sending its chosen mechanism name with an initial (possibly empty) message. Negotiation proceeds with the exchange of messages until either side indicates success or failure. The content of the messages is mechanism-specific. If the negotiation succeeds, then the session can proceed over the connection, otherwise it must be abandoned.
Some mechanisms continue to process session data after negotiation (e.g., encrypting it), while some specify that further session data is transmitted unmodified.
Negotiation
Commands
Avro SASL negotiation uses four one-byte commands.
- 0: START Used in a client’s initial message.
- 1: CONTINUE Used while negotiation is ongoing.
- 2: FAIL Terminates negotiation unsuccessfully.
- 3: COMPLETE Terminates negotiation successfully.
The format of a START message is:
| 0 | 4-byte mechanism name length | mechanism name | 4-byte payload length | payload data |
The format of a CONTINUE message is:
| 1 | 4-byte payload length | payload data |
The format of a FAIL message is:
| 2 | 4-byte message length | UTF-8 message |
The format of a COMPLETE message is:
| 3 | 4-byte payload length | payload data |
Process
Negotiation is initiated by a client sending a START command containing the client’s chosen mechanism name and any mechanism-specific payload data.
The server and client then interchange some number (possibly zero) of CONTINUE messages. Each message contains payload data that is processed by the security mechanism to generate the next message.
Once either the client or server send a FAIL message then negotiation has failed. UTF-8-encoded text is included in the failure message. Once either a FAIL message has been sent or received, or any other error occurs in the negotiation, further communication on this connection must cease.
Once either the client or server send a COMPLETE message then negotiation has completed successfully. Session data may now be transmitted over the connection until it is closed by either side.
Session Data
If no SASL QOP (quality of protection) is negotiated, then all subsequent writes to/reads over this connection are written/read unmodified. In particular, messages use Avro framing, and are of the form:
| 4-byte frame length | frame data | ... | 4 zero bytes |
If a SASL QOP is negotiated, then it must be used by the connection for all subsequent messages. This is done by wrapping each non-empty frame written using the security mechanism and unwrapping each non-empty frame read. The length written in each non-empty frame is the length of the wrapped data. Complete frames must be passed to the security mechanism for unwrapping. Unwrapped data is then passed to the application as the content of the frame.
If at any point processing fails due to wrapping, unwrapping or framing errors, then all further communication on this connection must cease.
Anonymous Mechanism
The SASL anonymous mechanism (RFC 2245) is quite simple to implement. In particular, an initial anonymous request may be prefixed by the following static sequence:
| 0 | 0009 | ANONYMOUS | 0000 |
If a server uses the anonymous mechanism, it should check that the mechanism name in the start message prefixing the first request received is ‘ANONYMOUS’, then simply prefix its initial response with a COMPLETE message of:
| 3 | 0000 |
If an anonymous server recieves some other mechanism name, then it may respond with a FAIL message as simple as:
| 2 | 0000 |
Note that the anonymous mechanism need add no additional round-trip messages between client and server. The START message can be piggybacked on the initial request and the COMPLETE or FAIL message can be piggybacked on the initial response.
Apache Avro, Avro, Apache, and the Avro and Apache logos are trademarks of The Apache Software Foundation.
28.4 - Wiki
The Wiki page can be found here.
28.5 - FAQ
What is Avro?
Avro is a data serialization system.
Avro provides:
- Rich data structures.
- A compact, fast, binary data format.
- A container file, to store persistent data.
- Remote procedure call (RPC).
- Simple integration with dynamic languages. Code generation is not required to read or write data files nor to use or implement RPC protocols. Code generation as an optional optimization, only worth implementing for statically typed languages.
How can I get started quickly with Avro?
Check out the Quick Start Guide.
How do I statically compile a schema or protocol into generated code?
In Java
- Add the avro jar, the jackson-mapper-asl.jar and jackson-core-asl.jar to your CLASSPATH.
- Run java org.apache.avro.specific.SpecificCompiler .
This appears to be out of date, the SpecificCompiler requires two arguments, presumably an input and and output file, but it isn’t clear that this does.
Or use the Schema or Protocol Ant tasks. Avro’s build.xml provides examples of how these are used.
Lastly, you can also use the “avro-tools” jar which ships with an Avro release. Just use the “compile (schema|protocol)” command.
How are Strings represented in Java?
They use org.apache.avro.util.Utf8, not java.lang.String.
More generally, how do Avro types map to Java types?
The mappings are documented in the package javadoc for generic, specific and reflect API.
From Doug Cutting:
HDFS splits files into blocks, and mapreduce runs a map task for each block. When the task starts, it needs to be able to seek into the file to the start of the block process through the block’s end. If the file were, e.g., a gzip file, this would not be possible, since gzip files must be decompressed from the start. One cannot seek into the middle of a gzip file and start decompressing. So Hadoop’s SequenceFile places a marker periodically (~64k) in the file at record and compression boundaries, where processing can be sensibly started. Then, when a map task starts processing an HDFS block, it finds the first marker after the block’s start and continues through the first marker in the next block of the file. This requires a bit of non-local access (~0.1%). Avro’s data file uses the same method as SequenceFile.
Why isn’t every value in Avro nullable?
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 of every value stored and
corresponding computational costs to create this bit on write and
interpret it on read. These costs are wasted when values may not in
fact be null, as is the case in many datasets. In Avro such costs are
only paid when values may actually be null.
Also, allowing values to be null is a well-known source of errors. In
Avro, a value declared as non-null will always be non-null and programs
need not test for null values when processing it nor will they ever fail
for lack of such tests.
Tony Hoare calls his invention of null references his “Billion Dollar
Mistake”.
http://qconlondon.com/london-2009/presentation/Null+References:+The+Billion+Dollar+Mistake
Also note that in some programming languages not all values are permitted to be null. For example, in Java, values of type boolean, byte, short, char, int, float, long, and double may not be null.
How can I serialize directly to/from a byte array?
As pointed out in the specification, Avro data should always be stored with its schema. The Avro provided classes DataFileWriter, DataFileReader, and DataFileStream all ensure this by serializing the Schema in a container header. In some special cases, such as when implementing a new storage system or writing unit tests, you may need to write and read directly with the bare Avro serialized values. The following examples use code generated for Java from an Avro Schema, specifically the User example from the Quickstart guide.
Serializing to a byte array
This example takes a User object and returns a newly allocated byte array with the Avro serialization of that user.
ByteArrayOutputStream out = new ByteArrayOutputStream();
BinaryEncoder encoder = EncoderFactory.get().binaryEncoder(out, null);
DatumWriter<User> writer = new SpecificDatumWriter<User>(User.getClassSchema());
writer.write(user, encoder);
encoder.flush();
out.close();
byte[] serializedBytes = out.toByteArray();
Deserializing from a byte array
This example takes a byte array containing the Avro serialization of a user and returns a User object.SpecificDatumReader reader = new SpecificDatumReader(User.getClassSchema());
Decoder decoder = DecoderFactory.get().binaryDecoder(bytes, null);
User user = reader.read(null, decoder);
Note: If you are serializing or deserializing in a loop or as a method, you should be reusing objects, readers and/or writers. Check the JavaDoc to see how to reuse the objects.
29 - Apache Avro™ 1.4.1 Documentation
Introduction
Apache Avro™ is a data serialization system.
Avro provides:
- Rich data structures.
- A compact, fast, binary data format.
- A container file, to store persistent data.
- Remote procedure call (RPC).
- Simple integration with dynamic languages. Code generation is not required to read or write data files nor to use or implement RPC protocols. Code generation as an optional optimization, only worth implementing for statically typed languages.
Schemas
Avro relies on schemas. When Avro data is read, the schema used when writing it is always present. This permits each datum to be written with no per-value overheads, making serialization both fast and small. This also facilitates use with dynamic, scripting languages, since data, together with its schema, is fully self-describing.
When Avro data is stored in a file, its schema is stored with it, so that files may be processed later by any program. If the program reading the data expects a different schema this can be easily resolved, since both schemas are present.
When Avro is used in RPC, the client and server exchange schemas in the connection handshake. (This can be optimized so that, for most calls, no schemas are actually transmitted.) Since both client and server both have the other’s full schema, correspondence between same named fields, missing fields, extra fields, etc. can all be easily resolved.
Avro schemas are defined with JSON . This facilitates implementation in languages that already have JSON libraries.
Comparison with other systems
Avro provides functionality similar to systems such as Thrift, Protocol Buffers, etc. Avro differs from these systems in the following fundamental aspects.
- Dynamic typing: Avro does not require that code be generated. Data is always accompanied by a schema that permits full processing of that data without code generation, static datatypes, etc. This facilitates construction of generic data-processing systems and languages.
- Untagged data: Since the schema is present when data is read, considerably less type information need be encoded with data, resulting in smaller serialization size.
- No manually-assigned field IDs: When a schema changes, both the old and new schema are always present when processing data, so differences may be resolved symbolically, using field names.
29.1 - Specification
Introduction
This document defines Avro. It is intended to be the authoritative specification. Implementations of Avro must adhere to this document.
Schema Declaration
A Schema is represented in JSON by one of:
- A JSON string, naming a defined type.
- A JSON object, of the form:
{"type": "typeName" ...attributes...}
where typeName is either a primitive or derived type name, as defined below. Attributes not defined in this document are permitted as metadata, but must not affect the format of serialized data.
- A JSON array, representing a union of embedded types.
Primitive Types
The set of primitive type names is:
- null: no value
- boolean: a binary value
- int: 32-bit signed integer
- long: 64-bit signed integer
- float: single precision (32-bit) IEEE 754 floating-point number
- double: double precision (64-bit) IEEE 754 floating-point number
- bytes: sequence of 8-bit unsigned bytes
- string: unicode character sequence
Primitive types have no specified attributes.
Primitive type names are also defined type names. Thus, for example, the schema “string” is equivalent to:
Complex Types
Avro supports six kinds of complex types: records, enums, arrays, maps, unions and fixed.
Records
Records use the type name “record” and support three attributes:
- name: a JSON string providing the name of the record (required).
- namespace, a JSON string that qualifies the name;
- doc: a JSON string providing documentation to the user of this schema (optional).
- aliases: a JSON array of strings, providing alternate names for this record (optional).
- fields: a JSON array, listing fields (required). Each field is a JSON object with the following attributes:
- name: a JSON string providing the name of the field (required), and
- doc: a JSON string describing this field for users (optional).
- type: A JSON object defining a schema, or a JSON string naming a record definition (required).
- default: A default value for this field, used when reading instances that lack this field (optional). 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. Default values for bytes and fixed fields are JSON strings, where Unicode code points 0-255 are mapped to unsigned 8-bit byte values 0-255.
field default values
| avro type |
json type |
example |
| null |
null |
null |
| boolean |
boolean |
true |
| int,long |
integer |
1 |
| float,double |
number |
1.1 |
| bytes |
string |
"\u00FF" |
| string |
string |
"foo" |
| record |
object |
{"a": 1} |
| enum |
string |
"FOO" |
| array |
array |
[1] |
| map |
object |
{"a": 1} |
| fixed |
string |
"\u00ff" |
- order: specifies how this field impacts sort ordering of this record (optional). Valid values are “ascending” (the default), “descending”, or “ignore”. For more details on how this is used, see the sort order section below.
- aliases: a JSON array of strings, providing alternate names for this field (optional).
For example, a linked-list of 64-bit values may be defined with:
{
"type": "record",
"name": "LongList",
"aliases": ["LinkedLongs"], // old name for this
"fields" : [
{"name": "value", "type": "long"}, // each element has a long
{"name": "next", "type": ["LongList", "null"]} // optional next element
]
}
Enums
Enums use the type name “enum” and support the following attributes:
- name: a JSON string providing the name of the enum (required).
- namespace, a JSON string that qualifies the name ;
- aliases: a JSON array of strings, providing alternate names for this enum (optional).
- doc: a JSON string providing documentation to the user of this schema (optional).
- symbols: a JSON array, listing symbols, as JSON strings (required). All symbols in an enum must be unique; duplicates are prohibited.
For example, playing card suits might be defined with:
{
"type": "enum",
"name": "Suit",
"symbols" : ["SPADES", "HEARTS", "DIAMONDS", "CLUBS"]
}
Arrays
Arrays use the type name “array” and support a single attribute:
- items: the schema of the array’s items.
For example, an array of strings is declared with:
{
"type": "array",
"items" : "string"
}
Maps
Maps use the type name “map” and support one attribute:
- values: the schema of the map’s values.
Map keys are assumed to be strings.
For example, a map from string to long is declared with:
{
"type": "map",
"values" : "long"
}
Unions
Unions, as mentioned above, are represented using JSON arrays. For example, ["string", "null"] declares a schema which may be either a string or null.
Unions may not contain more than one schema with the same type, except for the named types record, fixed and enum. For example, unions containing two array types or two map types are not permitted, but two types with different names are permitted. (Names permit efficient resolution when reading and writing unions.)
Unions may not immediately contain other unions.
Fixed
Fixed uses the type name “fixed” and supports two attributes:
- name: a string naming this fixed (required).
- namespace, a string that qualifies the name;
- aliases: a JSON array of strings, providing alternate names for this enum (optional).
- size: an integer, specifying the number of bytes per value (required).
For example, 16-byte quantity may be declared with:
{"type": "fixed", "size": 16, "name": "md5"}
Names
Record, enums and fixed are named types. Each has a fullname that is composed of two parts; a name and a namespace. Equality of names is defined on the fullname.
The name portion of a fullname, and record field names must:
- start with [A-Za-z_]
- subsequently contain only [A-Za-z0-9_]
A namespace is a dot-separated sequence of such names.
In record, enum and fixed definitions, the fullname is determined in one of the following ways:
A name and namespace are both specified. For example, one might use “name”: “X”, “namespace”: “org.foo” to indicate the fullname org.foo.X.
-
A fullname is specified. If the name specified contains a dot, then it is assumed to be a fullname, and any namespace also specified is ignored. For example, use “name”: “org.foo.X” to indicate the fullname org.foo.X.
-
A name only is specified, i.e., a name that contains no dots. In this case the namespace is taken from the most tightly enclosing schema or protocol. For example, if “name”: “X” is specified, and this occurs within a field of the record definition of org.foo.Y, then the fullname is org.foo.X.
References to previously defined names are as in the latter two cases above: if they contain a dot they are a fullname, if they do not contain a dot, the namespace is the namespace of the enclosing definition.
Primitive type names have no namespace and their names may not be defined in any namespace. A schema may only contain multiple definitions of a fullname if the definitions are equivalent.
Aliases
Named types and fields may have aliases. An implementation may optionally use aliases to map a writer’s schema to the reader’s. This facilitates both schema evolution as well as processing disparate datasets.
Aliases function by re-writing the writer’s schema using aliases from the reader’s schema. For example, if the writer’s schema was named “Foo” and the reader’s schema is named “Bar” and has an alias of “Foo”, then the implementation would act as though “Foo” were named “Bar” when reading. Similarly, if data was written as a record with a field named “x” and is read as a record with a field named “y” with alias “x”, then the implementation would act as though “x” were named “y” when reading.
A type alias may be specified either as a fully namespace-qualified, or relative to the namespace of the name it is an alias for. For example, if a type named “a.b” has aliases of “c” and “x.y”, then the fully qualified names of its aliases are “a.c” and “x.y”.
Data Serialization
Avro data is always serialized with its schema. Files that store Avro data should always also include the schema for that data in the same file. Avro-based remote procedure call (RPC) systems must also guarantee that remote recipients of data have a copy of the schema used to write that data.
Because the schema used to write data is always available when the data is read, Avro data itself is not tagged with type information. The schema is required to parse data.
In general, both serialization and deserialization proceed as a depth-first, left-to-right traversal of the schema, serializing primitive types as they are encountered.
Encodings
Avro specifies two serialization encodings: binary and JSON. Most applications will use the binary encoding, as it is smaller and faster. But, for debugging and web-based applications, the JSON encoding may sometimes be appropriate.
Binary Encoding
Primitive Types
Primitive types are encoded in binary as follows:
- null is written as zero bytes.
- a boolean is written as a single byte whose value is either 0 (false) or 1 (true).
- int and long values are written using variable-length zig-zag coding. Some examples:
| value |
hex |
| 0 |
00 |
| -1 |
01 |
| 1 |
02 |
| -2 |
03 |
| 2 |
04 |
| … |
|
| -64 |
7f |
| 64 |
80 01 |
| … |
|
- a float is written as 4 bytes. The float is converted into a 32-bit integer using a method equivalent to Java’s floatToIntBits and then encoded in little-endian format.
- a double is written as 8 bytes. The double is converted into a 64-bit integer using a method equivalent to Java’s doubleToLongBits and then encoded in little-endian format.
- bytes are encoded as a long followed by that many bytes of data.
- a string is encoded as a long followed by that many bytes of UTF-8 encoded character data.
For example, the three-character string “foo” would be encoded as the long value 3 (encoded as hex 06) followed by the UTF-8 encoding of ‘f’, ‘o’, and ‘o’ (the hex bytes 66 6f 6f):
06 66 6f 6f
Complex Types
Complex types are encoded in binary as follows:
Records
A record is encoded by encoding the values of its fields in the order that they are declared. In other words, a record is encoded as just the concatenation of the encodings of its fields. Field values are encoded per their schema.
For example, the record schema
{
"type": "record",
"name": "test",
"fields" : [
{"name": "a", "type": "long"},
{"name": "b", "type": "string"}
]
}
An instance of this record whose a field has value 27 (encoded as hex 36) and whose b field has value “foo” (encoded as hex bytes 06 66 6f 6f), would be encoded simply as the concatenation of these, namely the hex byte sequence:
36 06 66 6f 6f
Enums
An enum is encoded by a int, representing the zero-based position of the symbol in the schema.
For example, consider the enum:
{"type": "enum", "name": "Foo", "symbols": ["A", "B", "C", "D"] }
This would be encoded by an int between zero and three, with zero indicating “A”, and 3 indicating “D”.
Arrays
Arrays are encoded as a series of blocks. Each block consists of a long count value, followed by that many array items. A block with count zero indicates the end of the array. Each item is encoded per the array’s item schema.
If a block’s count is negative, its absolute value is used, and the count is followed immediately by a long block size indicating the number of bytes in the block. This block size permits fast skipping through data, e.g., when projecting a record to a subset of its fields.
For example, the array schema
{"type": "array", "items": "long"}
an array containing the items 3 and 27 could be encoded as the long value 2 (encoded as hex 04) followed by long values 3 and 27 (encoded as hex 06 36) terminated by zero:
04 06 36 00
The blocked representation permits one to read and write arrays larger than can be buffered in memory, since one can start writing items without knowing the full length of the array.
Maps
Maps are encoded as a series of blocks. Each block consists of a long count value, followed by that many key/value pairs. A block with count zero indicates the end of the map. Each item is encoded per the map’s value schema.
If a block’s count is negative, its absolute value is used, and the count is followed immediately by a long block size indicating the number of bytes in the block. This block size permits fast skipping through data, e.g., when projecting a record to a subset of its fields.
The blocked representation permits one to read and write maps larger than can be buffered in memory, since one can start writing items without knowing the full length of the map.
Unions
A union is encoded by first writing a long value indicating the zero-based position within the union of the schema of its value. The value is then encoded per the indicated schema within the union.
For example, the union schema ["string","null"] would encode:
-
null as the integer 1 (the index of “null” in the union, encoded as hex 02):
02
-
the string “a” as zero (the index of “string” in the union), followed by the serialized string:
02 02 61
Fixed
Fixed instances are encoded using the number of bytes declared in the schema.
JSON Encoding
Except for unions, the JSON encoding is the same as is used to encode field default values.
The value of a union is encoded in JSON as follows:
- if its type is null, then it is encoded as a JSON null;
- otherwise it is encoded as a JSON object with one name/value pair whose name is the type’s name and whose value is the recursively encoded value. For Avro’s named types (record, fixed or enum) the user-specified name is used, for other types the type name is used.
For example, the union schema ["null","string","Foo"], where Foo is a record name, would encode:
- null as null;
- the string “a” as
{"string": "a"} and
- a Foo instance as
{"Foo": {...}}, where {...} indicates the JSON encoding of a Foo instance.
Note that a schema is still required to correctly process JSON-encoded data. For example, the JSON encoding does not distinguish between int and long, float and double, records and maps, enums and strings, etc.
Sort Order
Avro defines a standard sort order for data. This permits data written by one system to be efficiently sorted by another system. This can be an important optimization, as sort order comparisons are sometimes the most frequent per-object operation. Note also that Avro binary-encoded data can be efficiently ordered without deserializing it to objects.
Data items may only be compared if they have identical schemas. Pairwise comparisons are implemented recursively with a depth-first, left-to-right traversal of the schema. The first mismatch encountered determines the order of the items.
Two items with the same schema are compared according to the following rules.
- null data is always equal.
- boolean data is ordered with false before true.
- int, long, float and double data is ordered by ascending numeric value.
- bytes and fixed data are compared lexicographically by unsigned 8-bit values.
- string data is compared lexicographically by Unicode code point. Note that since UTF-8 is used as the binary encoding for strings, sorting of bytes and string binary data is identical.
- array data is compared lexicographically by element.
- enum data is ordered by the symbol’s position in the enum schema. For example, an enum whose symbols are
["z", "a"] would sort “z” values before “a” values.
- union data is first ordered by the branch within the union, and, within that, by the type of the branch. For example, an
["int", "string"] union would order all int values before all string values, with the ints and strings themselves ordered as defined above.
- record data is ordered lexicographically by field. If a field specifies that its order is:
- “ascending”, then the order of its values is unaltered.
- “descending”, then the order of its values is reversed.
- “ignore”, then its values are ignored when sorting.
- map data may not be compared. It is an error to attempt to compare data containing maps unless those maps are in an
"order":"ignore" record field.
Object Container Files
Avro includes a simple object container file format. A file has a schema, and all objects stored in the file must be written according to that schema, using binary encoding. Objects are stored in blocks that may be compressed. Syncronization markers are used between blocks to permit efficient splitting of files for MapReduce processing.
Files may include arbitrary user-specified metadata.
A file consists of:
- A file header, followed by
- one or more file data blocks.
A file header consists of:
- Four bytes, ASCII ‘O’, ‘b’, ‘j’, followed by 1.
- file metadata, including the schema.
- The 16-byte, randomly-generated sync marker for this file.
File metadata consists of:
A long indicating the number of metadata key/value pairs.
For each pair, a string key and bytes value.
All metadata properties that start with “avro.” are reserved. The following file metadata properties are currently used:
- avro.schema contains the schema of objects stored in the file, as JSON data (required).
- avro.codec the name of the compression codec used to compress blocks, as a string. Implementations are required to support the following codecs: “null” and “deflate”. If codec is absent, it is assumed to be “null”. The codecs are described with more detail below.
A file header is thus described by the following schema:
{"type": "record", "name": "org.apache.avro.file.Header",
"fields" : [
{"name": "magic", "type": {"type": "fixed", "name": "Magic", "size": 4}},
{"name": "meta", "type": {"type": "map", "values": "bytes"}},
{"name": "sync", "type": {"type": "fixed", "name": "Sync", "size": 16}}
]
}
A file data block consists of:
- A long indicating the count of objects in this block.
- A long indicating the size in bytes of the serialized objects in the current block, after any codec is applied
- The serialized objects. If a codec is specified, this is compressed by that codec.
- The file’s 16-byte sync marker.
Thus, each block’s binary data can be efficiently extracted or skipped without deserializing the contents. The combination of block size, object counts, and sync markers enable detection of corrupt blocks and help ensure data integrity.
Required Codecs
null
The “null” codec simply passes through data uncompressed.
deflate
The “deflate” codec writes the data block using the deflate algorithm as specified in RFC 1951, and typically implemented using the zlib library. Note that this format (unlike the “zlib format” in RFC 1950) does not have a checksum.
Protocol Declaration
Avro protocols describe RPC interfaces. Like schemas, they are defined with JSON text.
A protocol is a JSON object with the following attributes:
- protocol, a string, the name of the protocol (required);
- namespace, an optional string that qualifies the name;
- doc, an optional string describing this protocol;
- types, an optional list of definitions of named types (records, enums, fixed and errors). An error definition is just like a record definition except it uses “error” instead of “record”. Note that forward references to named types are not permitted.
- messages, an optional JSON object whose keys are message names and whose values are objects whose attributes are described below. No two messages may have the same name.
The name and namespace qualification rules defined for schema objects apply to protocols as well.
Messages
A message has attributes:
- a doc, an optional description of the message,
- a request, a list of named, typed parameter schemas (this has the same form as the fields of a record declaration);
- a response schema;
- an optional union of declared error schemas. The effective union has “string” prepended to the declared union, to permit transmission of undeclared “system” errors. For example, if the declared error union is
["AccessError"], then the effective union is ["string", "AccessError"]. When no errors are declared, the effective error union is ["string"]. Errors are serialized using the effective union; however, a protocol’s JSON declaration contains only the declared union.
- an optional one-way boolean parameter.
A request parameter list is processed equivalently to an anonymous record. Since record field lists may vary between reader and writer, request parameters may also differ between the caller and responder, and such differences are resolved in the same manner as record field differences.
The one-way parameter may only be true when the response type is "null" and no errors are listed.
Sample Protocol
For example, one may define a simple HelloWorld protocol with:
{
"namespace": "com.acme",
"protocol": "HelloWorld",
"doc": "Protocol Greetings",
"types": [
{"name": "Greeting", "type": "record", "fields": [
{"name": "message", "type": "string"}]},
{"name": "Curse", "type": "error", "fields": [
{"name": "message", "type": "string"}]}
],
"messages": {
"hello": {
"doc": "Say hello.",
"request": [{"name": "greeting", "type": "Greeting" }],
"response": "Greeting",
"errors": ["Curse"]
}
}
}
Message Transport
Messages may be transmitted via different transport mechanisms.
To the transport, a message is an opaque byte sequence.
A transport is a system that supports:
- transmission of request messages
- receipt of corresponding response messages
Servers may send a response message back to the client corresponding to a request message. The mechanism of correspondance is transport-specific. For example, in HTTP it is implicit, since HTTP directly supports requests and responses. But a transport that multiplexes many client threads over a single socket would need to tag messages with unique identifiers.
Transports may be either stateless or stateful. In a stateless transport, messaging assumes no established connection state, while stateful transports establish connections that may be used for multiple messages. This distinction is discussed further in the handshake section below.
HTTP as Transport
When HTTP is used as a transport, each Avro message exchange is an HTTP request/response pair. All messages of an Avro protocol should share a single URL at an HTTP server. Other protocols may also use that URL. Both normal and error Avro response messages should use the 200 (OK) response code. The chunked encoding may be used for requests and responses, but, regardless the Avro request and response are the entire content of an HTTP request and response. The HTTP Content-Type of requests and responses should be specified as “avro/binary”. Requests should be made using the POST method.
HTTP is used by Avro as a stateless transport.
Message Framing
Avro messages are framed as a list of buffers.
Framing is a layer between messages and the transport. It exists to optimize certain operations.
The format of framed message data is:
- a series of buffers, where each buffer consists of:
- a four-byte, big-endian buffer length, followed by
- that many bytes of buffer data.
- A message is always terminated by a zero-lenghted buffer.
Framing is transparent to request and response message formats (described below). Any message may be presented as a single or multiple buffers.
Framing can permit readers to more efficiently get different buffers from different sources and for writers to more efficiently store different buffers to different destinations. In particular, it can reduce the number of times large binary objects are copied. For example, if an RPC parameter consists of a megabyte of file data, that data can be copied directly to a socket from a file descriptor, and, on the other end, it could be written directly to a file descriptor, never entering user space.
A simple, recommended, framing policy is for writers to create a new segment whenever a single binary object is written that is larger than a normal output buffer. Small objects are then appended in buffers, while larger objects are written as their own buffers. When a reader then tries to read a large object the runtime can hand it an entire buffer directly, without having to copy it.
Handshake
The purpose of the handshake is to ensure that the client and the server have each other’s protocol definition, so that the client can correctly deserialize responses, and the server can correctly deserialize requests. Both clients and servers should maintain a cache of recently seen protocols, so that, in most cases, a handshake will be completed without extra round-trip network exchanges or the transmission of full protocol text.
RPC requests and responses may not be processed until a handshake has been completed. With a stateless transport, all requests and responses are prefixed by handshakes. With a stateful transport, handshakes are only attached to requests and responses until a successful handshake response has been returned over a connection. After this, request and response payloads are sent without handshakes for the lifetime of that connection.
The handshake process uses the following record schemas:
{
"type": "record",
"name": "HandshakeRequest", "namespace":"org.apache.avro.ipc",
"fields": [
{"name": "clientHash",
"type": {"type": "fixed", "name": "MD5", "size": 16}},
{"name": "clientProtocol", "type": ["null", "string"]},
{"name": "serverHash", "type": "MD5"},
{"name": "meta", "type": ["null", {"type": "map", "values": "bytes"}]}
]
}
{
"type": "record",
"name": "HandshakeResponse", "namespace": "org.apache.avro.ipc",
"fields": [
{"name": "match",
"type": {"type": "enum", "name": "HandshakeMatch",
"symbols": ["BOTH", "CLIENT", "NONE"]}},
{"name": "serverProtocol",
"type": ["null", "string"]},
{"name": "serverHash",
"type": ["null", {"type": "fixed", "name": "MD5", "size": 16}]},
{"name": "meta",
"type": ["null", {"type": "map", "values": "bytes"}]}
]
}
- A client first prefixes each request with a
HandshakeRequest containing just the hash of its protocol and of the server’s protocol (clientHash!=null, clientProtocol=null, serverHash!=null), where the hashes are 128-bit MD5 hashes of the JSON protocol text. If a client has never connected to a given server, it sends its hash as a guess of the server’s hash, otherwise it sends the hash that it previously obtained from this server.
The server responds with a HandshakeResponse containing one of:
-
match=BOTH, serverProtocol=null, serverHash=null if the client sent the valid hash of the server’s protocol and the server knows what protocol corresponds to the client’s hash. In this case, the request is complete and the response data immediately follows the HandshakeResponse.
-
match=CLIENT, serverProtocol!=null, serverHash!=null if the server has previously seen the client’s protocol, but the client sent an incorrect hash of the server’s protocol. The request is complete and the response data immediately follows the HandshakeResponse. The client must use the returned protocol to process the response and should also cache that protocol and its hash for future interactions with this server.
-
match=NONE if the server has not previously seen the client’s protocol. The serverHash and serverProtocol may also be non-null if the server’s protocol hash was incorrect.
In this case the client must then re-submit its request with its protocol text (clientHash!=null, clientProtocol!=null, serverHash!=null) and the server should respond with a successful match (match=BOTH, serverProtocol=null, serverHash=null) as above.
The meta field is reserved for future handshake enhancements.
A call consists of a request message paired with its resulting response or error message. Requests and responses contain extensible metadata, and both kinds of messages are framed as described above.
The format of a call request is:
- request metadata, a map with values of type bytes
- the message name, an Avro string, followed by
- the message parameters. Parameters are serialized according to the message’s request declaration.
When a message is declared one-way and a stateful connection has been established by a successful handshake response, no response data is sent. Otherwise the format of the call response is:
- response metadata, a map with values of type bytes
- a one-byte error flag boolean, followed by either:
- if the error flag is false, the message response, serialized per the message’s response schema.
- if the error flag is true, the error, serialized per the message’s effective error union schema.
Schema Resolution
A reader of Avro data, whether from an RPC or a file, can always parse that data because its schema is provided. But that schema may not be exactly the schema that was expected. For example, if the data was written with a different version of the software than it is read, then records may have had fields added or removed. This section specifies how such schema differences should be resolved.
We call the schema used to write the data as the writer’s schema, and the schema that the application expects the reader’s schema. Differences between these should be resolved as follows:
- It is an error if the two schemas do not match.
To match, one of the following must hold:
-
both schemas are arrays whose item types match
-
both schemas are maps whose value types match
-
both schemas are enums whose names match
-
both schemas are fixed whose sizes and names match
-
both schemas are records with the same name
-
either schema is a union
-
both schemas have same primitive type
-
the writer’s schema may be promoted to the reader’s as follows:
- int is promotable to long, float, or double
- long is promotable to float or double
- float is promotable to double
- string is promotable to bytes
- bytes is promotable to string
-
if both are records:
- the ordering of fields may be different: fields are matched by name.
- schemas for fields with the same name in both records are resolved recursively.
- if the writer’s record contains a field with a name not present in the reader’s record, the writer’s value for that field is ignored.
- if the reader’s record schema has a field that contains a default value, and writer’s schema does not have a field with the same name, then the reader should use the default value from its field.
- if the reader’s record schema has a field with no default value, and writer’s schema does not have a field with the same name, an error is signalled.
-
if both are enums:
if the writer’s symbol is not present in the reader’s enum, then an error is signalled.
This resolution algorithm is applied recursively to the reader’s and writer’s array item schemas.
This resolution algorithm is applied recursively to the reader’s and writer’s value schemas.
The first schema in the reader’s union that matches the selected writer’s union schema is recursively resolved against it. if none match, an error is signalled.
- if reader’s is a union, but writer’s is not:
The first schema in the reader’s union that matches the writer’s schema is recursively resolved against it. If none match, an error is signalled.
- if writer’s is a union, but reader’s is not:
If the reader’s schema matches the selected writer’s schema, it is recursively resolved against it. If they do not match, an error is signalled.
A schema’s “doc” fields are ignored for the purposes of schema resolution. Hence, the “doc” portion of a schema may be dropped at serialization.
29.2 - IDL Language
Introduction
This document defines Avro IDL, an experimental higher-level language for authoring Avro schemata. Before reading this document, you should have familiarity with the concepts of schemata and protocols, as well as the various primitive and complex types available in Avro.
N.B. This feature is considered experimental in the current version of Avro and the language has not been finalized. Although major changes are unlikely, some syntax may change in future versions of Avro.
Overview
Purpose
The aim of the Avro IDL language is to enable developers to author schemata in a way that feels more similar to common programming languages like Java, C++, or Python. Additionally, the Avro IDL language may feel more familiar for those users who have previously used the interface description languages (IDLs) in other frameworks like Thrift, Protocol Buffers, or CORBA.
Usage
Each Avro IDL file defines a single Avro Protocol, and thus generates as its output a JSON-format Avro Protocol file with extension .avpr.
To convert a .avdl file into a .avpr file, it must be processed by the idl tool. For example:
$ java -jar avroj-tools-1.4.0.jar idl src/test/idl/input/namespaces.avdl /tmp/namespaces.avpr
$ head /tmp/namespaces.avpr
{
"protocol" : "TestNamespace",
"namespace" : "avro.test.protocol",
The idl tool can also process input to and from stdin and stdout. See idl --help for full usage information.
Defining a Protocol in Avro IDL
An Avro IDL file consists of exactly one protocol definition. The minimal protocol is defined by the following code:
This is equivalent to (and generates) the following JSON protocol definition:
{
"protocol" : "MyProtocol",
"types" : [ ],
"messages" : {
}
}
The namespace of the protocol may be changed using the @namespace annotation:
@namespace("mynamespace")
protocol MyProtocol {
}
This notation is used throughout Avro IDL as a way of specifying properties for the annotated element, as will be described later in this document.
Protocols in Avro IDL can contain the following items:
- Imports of external protocol and schema files.
- Definitions of named schemata, including records, errors, enums, and fixeds.
- Definitions of RPC messages
Imports
Files may be imported in one of three formats:
-
An IDL file may be imported with a statement like:
import idl "foo.avdl";
-
A JSON protocol file may be imported with a statement like:
import protocol "foo.avpr";
-
A JSON schema file may be imported with a statement like:
import schema "foo.avsc";
Messages and types in the imported file are added to this file’s protocol.
Imported file names are resolved relative to the current IDL file.
Defining an Enumeration
Enums are defined in Avro IDL using a syntax similar to C or Java:
enum Suit {
SPADES, DIAMONDS, CLUBS, HEARTS
}
Note that, unlike the JSON format, anonymous enums cannot be defined.
Defining a Fixed Length Field
Fixed fields are defined using the following syntax:
fixed MD5(16);
This example defines a fixed-length type called MD5 which contains 16 bytes.
Defining Records and Errors
Records are defined in Avro IDL using a syntax similar to a struct definition in C:
record Employee {
string name;
boolean active = true;
long salary;
}
The above example defines a record with the name “Employee” with three fields.
To define an error, simply use the keyword error instead of record. For example:
error Kaboom {
string explanation;
int result_code = -1;
}
Each field in a record or error consists of a type and a name, optional property annotations and an optional default value.
A type reference in Avro IDL must be one of:
- A primitive type
- A logical type
- A named schema defined prior to this usage in the same Protocol
- A complex type (array, map, or union)
Primitive Types
The primitive types supported by Avro IDL are the same as those supported by Avro’s JSON format. This list includes int, long, string, boolean, float, double, null, and bytes.
References to Named Schemata
If a named schema has already been defined in the same Avro IDL file, it may be referenced by name as if it were a primitive type:
record Card {
Suit suit; // refers to the enum Card defined above
int number;
}
Default Values
Default values for fields may be optionally specified by using an equals sign after the field name followed by a JSON expression indicating the default value. This JSON is interpreted as described in the spec.
Complex Types
Arrays
Array types are written in a manner that will seem familiar to C++ or Java programmers. An array of any type t is denoted array<t>. For example, an array of strings is denoted array<string>, and a multidimensional array of Foo records would be array<array<Foo>>.
Maps
Map types are written similarly to array types. An array that contains values of type t is written map<t>. As in the JSON schema format, all maps contain string-type keys.
Unions
Union types are denoted as union { typeA, typeB, typeC, … }. For example, this record contains a string field that is optional (unioned with null):
record RecordWithUnion {
union { null, string } optionalString;
}
Note that the same restrictions apply to Avro IDL unions as apply to unions defined in the JSON format; namely, a record may not contain multiple elements of the same type.
Defining RPC Messages
The syntax to define an RPC message within a Avro IDL protocol is similar to the syntax for a method declaration within a C header file or a Java interface. To define an RPC message add which takes two arguments named foo and bar, returning an int, simply include the following definition within the protocol:
int add(int foo, int bar = 0);
Message arguments, like record fields, may specify default values.
To define a message with no response, you may use the alias void, equivalent to the Avro null type:
void logMessage(string message);
If you have previously defined an error type within the same protocol, you may declare that a message can throw this error using the syntax:
void goKaboom() throws Kaboom;
To define a one-way message, use the keyword oneway after the parameter list, for example:
void fireAndForget(string message) oneway;
Other Language Features
All Java-style comments are supported within a Avro IDL file. Any text following // on a line is ignored, as is any text between /* and */, possibly spanning multiple lines.
Escaping Identifiers
Occasionally, one will need to use a reserved language keyword as an identifier. In order to do so, backticks (`) may be used to escape the identifier. For example, to define a message with the literal name error, you may write:
This syntax is allowed anywhere an identifier is expected.
Annotations for Ordering and Namespaces
Java-style annotations may be used to add additional properties to types and fields throughout Avro IDL.
For example, to specify the sort order of a field within a record, one may use the @order annotation before the field name as follows:
record MyRecord {
string @order("ascending") myAscendingSortField;
string @order("descending") myDescendingField;
string @order("ignore") myIgnoredField;
}
A field’s type may also be preceded by annotations, e.g.:
record MyRecord {
@java-class("java.util.ArrayList") array string myStrings;
}
Similarly, a @namespace annotation may be used to modify the namespace when defining a named schema. For example:
@namespace("org.apache.avro.firstNamespace")
protocol MyProto {
@namespace("org.apache.avro.someOtherNamespace")
record Foo {}
record Bar {}
}
will define a protocol in the firstNamespace namespace. The record Foo will be defined in someOtherNamespace and Bar will be defined in firstNamespace as it inherits its default from its container.
Type and field aliases are specified with the @aliases annotation as follows:
@aliases(["org.old.OldRecord", "org.ancient.AncientRecord"])
record MyRecord {
string @aliases(["oldField", "ancientField"]) myNewField;
}
Complete Example
The following is a complete example of a Avro IDL file that shows most of the above features:
/**
* An example protocol in Avro IDL
*/
@namespace("org.apache.avro.test")
protocol Simple {
@aliases(["org.foo.KindOf"])
enum Kind {
FOO,
BAR, // the bar enum value
BAZ
}
fixed MD5(16);
record TestRecord {
@order("ignore")
string name;
@order("descending")
Kind kind;
MD5 hash;
union { MD5, null} @aliases(["hash"]) nullableHash;
array<long> arrayOfLongs;
}
error TestError {
string message;
}
string hello(string greeting);
TestRecord echo(TestRecord `record`);
int add(int arg1, int arg2);
bytes echoBytes(bytes data);
void `error`() throws TestError;
void ping() oneway;
}
Additional examples may be found in the Avro source tree under the src/test/idl/input directory.
29.3 - SASL profile
Introduction
SASL (RFC 2222) provides a framework for authentication and security of network protocols. Each protocol that uses SASL is meant to define a SASL profile. This document provides a SASL profile for connection-based Avro RPC.
Overview
SASL negotiation proceeds as a series of message interactions over a connection between a client and server using a selected SASL mechanism. The client starts this negotiation by sending its chosen mechanism name with an initial (possibly empty) message. Negotiation proceeds with the exchange of messages until either side indicates success or failure. The content of the messages is mechanism-specific. If the negotiation succeeds, then the session can proceed over the connection, otherwise it must be abandoned.
Some mechanisms continue to process session data after negotiation (e.g., encrypting it), while some specify that further session data is transmitted unmodified.
Negotiation
Commands
Avro SASL negotiation uses four one-byte commands.
- 0: START Used in a client’s initial message.
- 1: CONTINUE Used while negotiation is ongoing.
- 2: FAIL Terminates negotiation unsuccessfully.
- 3: COMPLETE Terminates negotiation successfully.
The format of a START message is:
| 0 | 4-byte mechanism name length | mechanism name | 4-byte payload length | payload data |
The format of a CONTINUE message is:
| 1 | 4-byte payload length | payload data |
The format of a FAIL message is:
| 2 | 4-byte message length | UTF-8 message |
The format of a COMPLETE message is:
| 3 | 4-byte payload length | payload data |
Process
Negotiation is initiated by a client sending a START command containing the client’s chosen mechanism name and any mechanism-specific payload data.
The server and client then interchange some number (possibly zero) of CONTINUE messages. Each message contains payload data that is processed by the security mechanism to generate the next message.
Once either the client or server send a FAIL message then negotiation has failed. UTF-8-encoded text is included in the failure message. Once either a FAIL message has been sent or received, or any other error occurs in the negotiation, further communication on this connection must cease.
Once either the client or server send a COMPLETE message then negotiation has completed successfully. Session data may now be transmitted over the connection until it is closed by either side.
Session Data
If no SASL QOP (quality of protection) is negotiated, then all subsequent writes to/reads over this connection are written/read unmodified. In particular, messages use Avro framing, and are of the form:
| 4-byte frame length | frame data | ... | 4 zero bytes |
If a SASL QOP is negotiated, then it must be used by the connection for all subsequent messages. This is done by wrapping each non-empty frame written using the security mechanism and unwrapping each non-empty frame read. The length written in each non-empty frame is the length of the wrapped data. Complete frames must be passed to the security mechanism for unwrapping. Unwrapped data is then passed to the application as the content of the frame.
If at any point processing fails due to wrapping, unwrapping or framing errors, then all further communication on this connection must cease.
Anonymous Mechanism
The SASL anonymous mechanism (RFC 2245) is quite simple to implement. In particular, an initial anonymous request may be prefixed by the following static sequence:
| 0 | 0009 | ANONYMOUS | 0000 |
If a server uses the anonymous mechanism, it should check that the mechanism name in the start message prefixing the first request received is ‘ANONYMOUS’, then simply prefix its initial response with a COMPLETE message of:
| 3 | 0000 |
If an anonymous server recieves some other mechanism name, then it may respond with a FAIL message as simple as:
| 2 | 0000 |
Note that the anonymous mechanism need add no additional round-trip messages between client and server. The START message can be piggybacked on the initial request and the COMPLETE or FAIL message can be piggybacked on the initial response.
29.4 - Wiki
The Wiki page can be found here.
29.5 - FAQ
What is Avro?
Avro is a data serialization system.
Avro provides:
- Rich data structures.
- A compact, fast, binary data format.
- A container file, to store persistent data.
- Remote procedure call (RPC).
- Simple integration with dynamic languages. Code generation is not required to read or write data files nor to use or implement RPC protocols. Code generation as an optional optimization, only worth implementing for statically typed languages.
How can I get started quickly with Avro?
Check out the Quick Start Guide.
How do I statically compile a schema or protocol into generated code?
In Java
- Add the avro jar, the jackson-mapper-asl.jar and jackson-core-asl.jar to your CLASSPATH.
- Run java org.apache.avro.specific.SpecificCompiler .
This appears to be out of date, the SpecificCompiler requires two arguments, presumably an input and and output file, but it isn’t clear that this does.
Or use the Schema or Protocol Ant tasks. Avro’s build.xml provides examples of how these are used.
Lastly, you can also use the “avro-tools” jar which ships with an Avro release. Just use the “compile (schema|protocol)” command.
How are Strings represented in Java?
They use org.apache.avro.util.Utf8, not java.lang.String.
More generally, how do Avro types map to Java types?
The mappings are documented in the package javadoc for generic, specific and reflect API.
From Doug Cutting:
HDFS splits files into blocks, and mapreduce runs a map task for each block. When the task starts, it needs to be able to seek into the file to the start of the block process through the block’s end. If the file were, e.g., a gzip file, this would not be possible, since gzip files must be decompressed from the start. One cannot seek into the middle of a gzip file and start decompressing. So Hadoop’s SequenceFile places a marker periodically (~64k) in the file at record and compression boundaries, where processing can be sensibly started. Then, when a map task starts processing an HDFS block, it finds the first marker after the block’s start and continues through the first marker in the next block of the file. This requires a bit of non-local access (~0.1%). Avro’s data file uses the same method as SequenceFile.
Why isn’t every value in Avro nullable?
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 of every value stored and
corresponding computational costs to create this bit on write and
interpret it on read. These costs are wasted when values may not in
fact be null, as is the case in many datasets. In Avro such costs are
only paid when values may actually be null.
Also, allowing values to be null is a well-known source of errors. In
Avro, a value declared as non-null will always be non-null and programs
need not test for null values when processing it nor will they ever fail
for lack of such tests.
Tony Hoare calls his invention of null references his “Billion Dollar
Mistake”.
http://qconlondon.com/london-2009/presentation/Null+References:+The+Billion+Dollar+Mistake
Also note that in some programming languages not all values are permitted to be null. For example, in Java, values of type boolean, byte, short, char, int, float, long, and double may not be null.
How can I serialize directly to/from a byte array?
As pointed out in the specification, Avro data should always be stored with its schema. The Avro provided classes DataFileWriter, DataFileReader, and DataFileStream all ensure this by serializing the Schema in a container header. In some special cases, such as when implementing a new storage system or writing unit tests, you may need to write and read directly with the bare Avro serialized values. The following examples use code generated for Java from an Avro Schema, specifically the User example from the Quickstart guide.
Serializing to a byte array
This example takes a User object and returns a newly allocated byte array with the Avro serialization of that user.
ByteArrayOutputStream out = new ByteArrayOutputStream();
BinaryEncoder encoder = EncoderFactory.get().binaryEncoder(out, null);
DatumWriter<User> writer = new SpecificDatumWriter<User>(User.getClassSchema());
writer.write(user, encoder);
encoder.flush();
out.close();
byte[] serializedBytes = out.toByteArray();
Deserializing from a byte array
This example takes a byte array containing the Avro serialization of a user and returns a User object.SpecificDatumReader reader = new SpecificDatumReader(User.getClassSchema());
Decoder decoder = DecoderFactory.get().binaryDecoder(bytes, null);
User user = reader.read(null, decoder);
Note: If you are serializing or deserializing in a loop or as a method, you should be reusing objects, readers and/or writers. Check the JavaDoc to see how to reuse the objects.
30 - Apache Avro™ 1.4.0 Documentation
Introduction
Avro™ is a data serialization system.
Avro provides:
- Rich data structures.
- A compact, fast, binary data format.
- A container file, to store persistent data.
- Remote procedure call (RPC).
- Simple integration with dynamic languages. Code generation is not required to read or write data files nor to use or implement RPC protocols. Code generation as an optional optimization, only worth implementing for statically typed languages.
Schemas
Avro relies on schemas. When Avro data is read, the schema used when writing it is always present. This permits each datum to be written with no per-value overheads, making serialization both fast and small. This also facilitates use with dynamic, scripting languages, since data, together with its schema, is fully self-describing.
When Avro data is stored in a file, its schema is stored with it, so that files may be processed later by any program. If the program reading the data expects a different schema this can be easily resolved, since both schemas are present.
When Avro is used in RPC, the client and server exchange schemas in the connection handshake. (This can be optimized so that, for most calls, no schemas are actually transmitted.) Since both client and server both have the other’s full schema, correspondence between same named fields, missing fields, extra fields, etc. can all be easily resolved.
Avro schemas are defined with JSON . This facilitates implementation in languages that already have JSON libraries.
Comparison with other systems
Avro provides functionality similar to systems such as Thrift, Protocol Buffers, etc. Avro differs from these systems in the following fundamental aspects.
- Dynamic typing: Avro does not require that code be generated. Data is always accompanied by a schema that permits full processing of that data without code generation, static datatypes, etc. This facilitates construction of generic data-processing systems and languages.
- Untagged data: Since the schema is present when data is read, considerably less type information need be encoded with data, resulting in smaller serialization size.
- No manually-assigned field IDs: When a schema changes, both the old and new schema are always present when processing data, so differences may be resolved symbolically, using field names.
30.1 - Specification
Introduction
This document defines Avro. It is intended to be the authoritative specification. Implementations of Avro must adhere to this document.
Schema Declaration
A Schema is represented in JSON by one of:
- A JSON string, naming a defined type.
- A JSON object, of the form:
{"type": "typeName" ...attributes...}
where typeName is either a primitive or derived type name, as defined below. Attributes not defined in this document are permitted as metadata, but must not affect the format of serialized data.
- A JSON array, representing a union of embedded types.
Primitive Types
The set of primitive type names is:
- null: no value
- boolean: a binary value
- int: 32-bit signed integer
- long: 64-bit signed integer
- float: single precision (32-bit) IEEE 754 floating-point number
- double: double precision (64-bit) IEEE 754 floating-point number
- bytes: sequence of 8-bit unsigned bytes
- string: unicode character sequence
Primitive types have no specified attributes.
Primitive type names are also defined type names. Thus, for example, the schema “string” is equivalent to:
Complex Types
Avro supports six kinds of complex types: records, enums, arrays, maps, unions and fixed.
Records
Records use the type name “record” and support three attributes:
- name: a JSON string providing the name of the record (required).
- namespace, a JSON string that qualifies the name;
- doc: a JSON string providing documentation to the user of this schema (optional).
- aliases: a JSON array of strings, providing alternate names for this record (optional).
- fields: a JSON array, listing fields (required). Each field is a JSON object with the following attributes:
- name: a JSON string providing the name of the field (required), and
- doc: a JSON string describing this field for users (optional).
- type: A JSON object defining a schema, or a JSON string naming a record definition (required).
- default: A default value for this field, used when reading instances that lack this field (optional). 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. Default values for bytes and fixed fields are JSON strings, where Unicode code points 0-255 are mapped to unsigned 8-bit byte values 0-255.
field default values
| avro type |
json type |
example |
| null |
null |
null |
| boolean |
boolean |
true |
| int,long |
integer |
1 |
| float,double |
number |
1.1 |
| bytes |
string |
"\u00FF" |
| string |
string |
"foo" |
| record |
object |
{"a": 1} |
| enum |
string |
"FOO" |
| array |
array |
[1] |
| map |
object |
{"a": 1} |
| fixed |
string |
"\u00ff" |
- order: specifies how this field impacts sort ordering of this record (optional). Valid values are “ascending” (the default), “descending”, or “ignore”. For more details on how this is used, see the sort order section below.
- aliases: a JSON array of strings, providing alternate names for this field (optional).
For example, a linked-list of 64-bit values may be defined with:
{
"type": "record",
"name": "LongList",
"aliases": ["LinkedLongs"], // old name for this
"fields" : [
{"name": "value", "type": "long"}, // each element has a long
{"name": "next", "type": ["LongList", "null"]} // optional next element
]
}
Enums
Enums use the type name “enum” and support the following attributes:
- name: a JSON string providing the name of the enum (required).
- namespace, a JSON string that qualifies the name ;
- aliases: a JSON array of strings, providing alternate names for this enum (optional).
- doc: a JSON string providing documentation to the user of this schema (optional).
- symbols: a JSON array, listing symbols, as JSON strings (required). All symbols in an enum must be unique; duplicates are prohibited.
For example, playing card suits might be defined with:
{
"type": "enum",
"name": "Suit",
"symbols" : ["SPADES", "HEARTS", "DIAMONDS", "CLUBS"]
}
Arrays
Arrays use the type name “array” and support a single attribute:
- items: the schema of the array’s items.
For example, an array of strings is declared with:
{
"type": "array",
"items" : "string"
}
Maps
Maps use the type name “map” and support one attribute:
- values: the schema of the map’s values.
Map keys are assumed to be strings.
For example, a map from string to long is declared with:
{
"type": "map",
"values" : "long"
}
Unions
Unions, as mentioned above, are represented using JSON arrays. For example, ["string", "null"] declares a schema which may be either a string or null.
Unions may not contain more than one schema with the same type, except for the named types record, fixed and enum. For example, unions containing two array types or two map types are not permitted, but two types with different names are permitted. (Names permit efficient resolution when reading and writing unions.)
Unions may not immediately contain other unions.
Fixed
Fixed uses the type name “fixed” and supports two attributes:
- name: a string naming this fixed (required).
- namespace, a string that qualifies the name;
- aliases: a JSON array of strings, providing alternate names for this enum (optional).
- size: an integer, specifying the number of bytes per value (required).
For example, 16-byte quantity may be declared with:
{"type": "fixed", "size": 16, "name": "md5"}
Names
Record, enums and fixed are named types. Each has a fullname that is composed of two parts; a name and a namespace. Equality of names is defined on the fullname.
The name portion of a fullname, and record field names must:
- start with [A-Za-z_]
- subsequently contain only [A-Za-z0-9_]
A namespace is a dot-separated sequence of such names.
In record, enum and fixed definitions, the fullname is determined in one of the following ways:
A name and namespace are both specified. For example, one might use “name”: “X”, “namespace”: “org.foo” to indicate the fullname org.foo.X.
-
A fullname is specified. If the name specified contains a dot, then it is assumed to be a fullname, and any namespace also specified is ignored. For example, use “name”: “org.foo.X” to indicate the fullname org.foo.X.
-
A name only is specified, i.e., a name that contains no dots. In this case the namespace is taken from the most tightly enclosing schema or protocol. For example, if “name”: “X” is specified, and this occurs within a field of the record definition of org.foo.Y, then the fullname is org.foo.X.
References to previously defined names are as in the latter two cases above: if they contain a dot they are a fullname, if they do not contain a dot, the namespace is the namespace of the enclosing definition.
Primitive type names have no namespace and their names may not be defined in any namespace. A schema may only contain multiple definitions of a fullname if the definitions are equivalent.
Aliases
Named types and fields may have aliases. An implementation may optionally use aliases to map a writer’s schema to the reader’s. This facilitates both schema evolution as well as processing disparate datasets.
Aliases function by re-writing the writer’s schema using aliases from the reader’s schema. For example, if the writer’s schema was named “Foo” and the reader’s schema is named “Bar” and has an alias of “Foo”, then the implementation would act as though “Foo” were named “Bar” when reading. Similarly, if data was written as a record with a field named “x” and is read as a record with a field named “y” with alias “x”, then the implementation would act as though “x” were named “y” when reading.
A type alias may be specified either as a fully namespace-qualified, or relative to the namespace of the name it is an alias for. For example, if a type named “a.b” has aliases of “c” and “x.y”, then the fully qualified names of its aliases are “a.c” and “x.y”.
Data Serialization
Avro data is always serialized with its schema. Files that store Avro data should always also include the schema for that data in the same file. Avro-based remote procedure call (RPC) systems must also guarantee that remote recipients of data have a copy of the schema used to write that data.
Because the schema used to write data is always available when the data is read, Avro data itself is not tagged with type information. The schema is required to parse data.
In general, both serialization and deserialization proceed as a depth-first, left-to-right traversal of the schema, serializing primitive types as they are encountered.
Encodings
Avro specifies two serialization encodings: binary and JSON. Most applications will use the binary encoding, as it is smaller and faster. But, for debugging and web-based applications, the JSON encoding may sometimes be appropriate.
Binary Encoding
Primitive Types
Primitive types are encoded in binary as follows:
- null is written as zero bytes.
- a boolean is written as a single byte whose value is either 0 (false) or 1 (true).
- int and long values are written using variable-length zig-zag coding. Some examples:
| value |
hex |
| 0 |
00 |
| -1 |
01 |
| 1 |
02 |
| -2 |
03 |
| 2 |
04 |
| … |
|
| -64 |
7f |
| 64 |
80 01 |
| … |
|
- a float is written as 4 bytes. The float is converted into a 32-bit integer using a method equivalent to Java’s floatToIntBits and then encoded in little-endian format.
- a double is written as 8 bytes. The double is converted into a 64-bit integer using a method equivalent to Java’s doubleToLongBits and then encoded in little-endian format.
- bytes are encoded as a long followed by that many bytes of data.
- a string is encoded as a long followed by that many bytes of UTF-8 encoded character data.
For example, the three-character string “foo” would be encoded as the long value 3 (encoded as hex 06) followed by the UTF-8 encoding of ‘f’, ‘o’, and ‘o’ (the hex bytes 66 6f 6f):
06 66 6f 6f
Complex Types
Complex types are encoded in binary as follows:
Records
A record is encoded by encoding the values of its fields in the order that they are declared. In other words, a record is encoded as just the concatenation of the encodings of its fields. Field values are encoded per their schema.
For example, the record schema
{
"type": "record",
"name": "test",
"fields" : [
{"name": "a", "type": "long"},
{"name": "b", "type": "string"}
]
}
An instance of this record whose a field has value 27 (encoded as hex 36) and whose b field has value “foo” (encoded as hex bytes 06 66 6f 6f), would be encoded simply as the concatenation of these, namely the hex byte sequence:
36 06 66 6f 6f
Enums
An enum is encoded by a int, representing the zero-based position of the symbol in the schema.
For example, consider the enum:
{"type": "enum", "name": "Foo", "symbols": ["A", "B", "C", "D"] }
This would be encoded by an int between zero and three, with zero indicating “A”, and 3 indicating “D”.
Arrays
Arrays are encoded as a series of blocks. Each block consists of a long count value, followed by that many array items. A block with count zero indicates the end of the array. Each item is encoded per the array’s item schema.
If a block’s count is negative, its absolute value is used, and the count is followed immediately by a long block size indicating the number of bytes in the block. This block size permits fast skipping through data, e.g., when projecting a record to a subset of its fields.
For example, the array schema
{"type": "array", "items": "long"}
an array containing the items 3 and 27 could be encoded as the long value 2 (encoded as hex 04) followed by long values 3 and 27 (encoded as hex 06 36) terminated by zero:
04 06 36 00
The blocked representation permits one to read and write arrays larger than can be buffered in memory, since one can start writing items without knowing the full length of the array.
Maps
Maps are encoded as a series of blocks. Each block consists of a long count value, followed by that many key/value pairs. A block with count zero indicates the end of the map. Each item is encoded per the map’s value schema.
If a block’s count is negative, its absolute value is used, and the count is followed immediately by a long block size indicating the number of bytes in the block. This block size permits fast skipping through data, e.g., when projecting a record to a subset of its fields.
The blocked representation permits one to read and write maps larger than can be buffered in memory, since one can start writing items without knowing the full length of the map.
Unions
A union is encoded by first writing a long value indicating the zero-based position within the union of the schema of its value. The value is then encoded per the indicated schema within the union.
For example, the union schema ["string","null"] would encode:
-
null as the integer 1 (the index of “null” in the union, encoded as hex 02):
02
-
the string “a” as zero (the index of “string” in the union), followed by the serialized string:
02 02 61
Fixed
Fixed instances are encoded using the number of bytes declared in the schema.
JSON Encoding
Except for unions, the JSON encoding is the same as is used to encode field default values.
The value of a union is encoded in JSON as follows:
- if its type is null, then it is encoded as a JSON null;
- otherwise it is encoded as a JSON object with one name/value pair whose name is the type’s name and whose value is the recursively encoded value. For Avro’s named types (record, fixed or enum) the user-specified name is used, for other types the type name is used.
For example, the union schema ["null","string","Foo"], where Foo is a record name, would encode:
- null as null;
- the string “a” as
{"string": "a"} and
- a Foo instance as
{"Foo": {...}}, where {...} indicates the JSON encoding of a Foo instance.
Note that a schema is still required to correctly process JSON-encoded data. For example, the JSON encoding does not distinguish between int and long, float and double, records and maps, enums and strings, etc.
Sort Order
Avro defines a standard sort order for data. This permits data written by one system to be efficiently sorted by another system. This can be an important optimization, as sort order comparisons are sometimes the most frequent per-object operation. Note also that Avro binary-encoded data can be efficiently ordered without deserializing it to objects.
Data items may only be compared if they have identical schemas. Pairwise comparisons are implemented recursively with a depth-first, left-to-right traversal of the schema. The first mismatch encountered determines the order of the items.
Two items with the same schema are compared according to the following rules.
- null data is always equal.
- boolean data is ordered with false before true.
- int, long, float and double data is ordered by ascending numeric value.
- bytes and fixed data are compared lexicographically by unsigned 8-bit values.
- string data is compared lexicographically by Unicode code point. Note that since UTF-8 is used as the binary encoding for strings, sorting of bytes and string binary data is identical.
- array data is compared lexicographically by element.
- enum data is ordered by the symbol’s position in the enum schema. For example, an enum whose symbols are
["z", "a"] would sort “z” values before “a” values.
- union data is first ordered by the branch within the union, and, within that, by the type of the branch. For example, an
["int", "string"] union would order all int values before all string values, with the ints and strings themselves ordered as defined above.
- record data is ordered lexicographically by field. If a field specifies that its order is:
- “ascending”, then the order of its values is unaltered.
- “descending”, then the order of its values is reversed.
- “ignore”, then its values are ignored when sorting.
- map data may not be compared. It is an error to attempt to compare data containing maps unless those maps are in an
"order":"ignore" record field.
Object Container Files
Avro includes a simple object container file format. A file has a schema, and all objects stored in the file must be written according to that schema, using binary encoding. Objects are stored in blocks that may be compressed. Syncronization markers are used between blocks to permit efficient splitting of files for MapReduce processing.
Files may include arbitrary user-specified metadata.
A file consists of:
- A file header, followed by
- one or more file data blocks.
A file header consists of:
- Four bytes, ASCII ‘O’, ‘b’, ‘j’, followed by 1.
- file metadata, including the schema.
- The 16-byte, randomly-generated sync marker for this file.
File metadata consists of:
A long indicating the number of metadata key/value pairs.
For each pair, a string key and bytes value.
All metadata properties that start with “avro.” are reserved. The following file metadata properties are currently used:
- avro.schema contains the schema of objects stored in the file, as JSON data (required).
- avro.codec the name of the compression codec used to compress blocks, as a string. Implementations are required to support the following codecs: “null” and “deflate”. If codec is absent, it is assumed to be “null”. The codecs are described with more detail below.
A file header is thus described by the following schema:
{"type": "record", "name": "org.apache.avro.file.Header",
"fields" : [
{"name": "magic", "type": {"type": "fixed", "name": "Magic", "size": 4}},
{"name": "meta", "type": {"type": "map", "values": "bytes"}},
{"name": "sync", "type": {"type": "fixed", "name": "Sync", "size": 16}}
]
}
A file data block consists of:
- A long indicating the count of objects in this block.
- A long indicating the size in bytes of the serialized objects in the current block, after any codec is applied
- The serialized objects. If a codec is specified, this is compressed by that codec.
- The file’s 16-byte sync marker.
Thus, each block’s binary data can be efficiently extracted or skipped without deserializing the contents. The combination of block size, object counts, and sync markers enable detection of corrupt blocks and help ensure data integrity.
Required Codecs
null
The “null” codec simply passes through data uncompressed.
deflate
The “deflate” codec writes the data block using the deflate algorithm as specified in RFC 1951, and typically implemented using the zlib library. Note that this format (unlike the “zlib format” in RFC 1950) does not have a checksum.
Protocol Declaration
Avro protocols describe RPC interfaces. Like schemas, they are defined with JSON text.
A protocol is a JSON object with the following attributes:
- protocol, a string, the name of the protocol (required);
- namespace, an optional string that qualifies the name;
- doc, an optional string describing this protocol;
- types, an optional list of definitions of named types (records, enums, fixed and errors). An error definition is just like a record definition except it uses “error” instead of “record”. Note that forward references to named types are not permitted.
- messages, an optional JSON object whose keys are message names and whose values are objects whose attributes are described below. No two messages may have the same name.
The name and namespace qualification rules defined for schema objects apply to protocols as well.
Messages
A message has attributes:
- a doc, an optional description of the message,
- a request, a list of named, typed parameter schemas (this has the same form as the fields of a record declaration);
- a response schema;
- an optional union of declared error schemas. The effective union has “string” prepended to the declared union, to permit transmission of undeclared “system” errors. For example, if the declared error union is
["AccessError"], then the effective union is ["string", "AccessError"]. When no errors are declared, the effective error union is ["string"]. Errors are serialized using the effective union; however, a protocol’s JSON declaration contains only the declared union.
- an optional one-way boolean parameter.
A request parameter list is processed equivalently to an anonymous record. Since record field lists may vary between reader and writer, request parameters may also differ between the caller and responder, and such differences are resolved in the same manner as record field differences.
The one-way parameter may only be true when the response type is "null" and no errors are listed.
Sample Protocol
For example, one may define a simple HelloWorld protocol with:
{
"namespace": "com.acme",
"protocol": "HelloWorld",
"doc": "Protocol Greetings",
"types": [
{"name": "Greeting", "type": "record", "fields": [
{"name": "message", "type": "string"}]},
{"name": "Curse", "type": "error", "fields": [
{"name": "message", "type": "string"}]}
],
"messages": {
"hello": {
"doc": "Say hello.",
"request": [{"name": "greeting", "type": "Greeting" }],
"response": "Greeting",
"errors": ["Curse"]
}
}
}
Message Transport
Messages may be transmitted via different transport mechanisms.
To the transport, a message is an opaque byte sequence.
A transport is a system that supports:
- transmission of request messages
- receipt of corresponding response messages
Servers may send a response message back to the client corresponding to a request message. The mechanism of correspondance is transport-specific. For example, in HTTP it is implicit, since HTTP directly supports requests and responses. But a transport that multiplexes many client threads over a single socket would need to tag messages with unique identifiers.
Transports may be either stateless or stateful. In a stateless transport, messaging assumes no established connection state, while stateful transports establish connections that may be used for multiple messages. This distinction is discussed further in the handshake section below.
HTTP as Transport
When HTTP is used as a transport, each Avro message exchange is an HTTP request/response pair. All messages of an Avro protocol should share a single URL at an HTTP server. Other protocols may also use that URL. Both normal and error Avro response messages should use the 200 (OK) response code. The chunked encoding may be used for requests and responses, but, regardless the Avro request and response are the entire content of an HTTP request and response. The HTTP Content-Type of requests and responses should be specified as “avro/binary”. Requests should be made using the POST method.
HTTP is used by Avro as a stateless transport.
Message Framing
Avro messages are framed as a list of buffers.
Framing is a layer between messages and the transport. It exists to optimize certain operations.
The format of framed message data is:
- a series of buffers, where each buffer consists of:
- a four-byte, big-endian buffer length, followed by
- that many bytes of buffer data.
- A message is always terminated by a zero-lenghted buffer.
Framing is transparent to request and response message formats (described below). Any message may be presented as a single or multiple buffers.
Framing can permit readers to more efficiently get different buffers from different sources and for writers to more efficiently store different buffers to different destinations. In particular, it can reduce the number of times large binary objects are copied. For example, if an RPC parameter consists of a megabyte of file data, that data can be copied directly to a socket from a file descriptor, and, on the other end, it could be written directly to a file descriptor, never entering user space.
A simple, recommended, framing policy is for writers to create a new segment whenever a single binary object is written that is larger than a normal output buffer. Small objects are then appended in buffers, while larger objects are written as their own buffers. When a reader then tries to read a large object the runtime can hand it an entire buffer directly, without having to copy it.
Handshake
The purpose of the handshake is to ensure that the client and the server have each other’s protocol definition, so that the client can correctly deserialize responses, and the server can correctly deserialize requests. Both clients and servers should maintain a cache of recently seen protocols, so that, in most cases, a handshake will be completed without extra round-trip network exchanges or the transmission of full protocol text.
RPC requests and responses may not be processed until a handshake has been completed. With a stateless transport, all requests and responses are prefixed by handshakes. With a stateful transport, handshakes are only attached to requests and responses until a successful handshake response has been returned over a connection. After this, request and response payloads are sent without handshakes for the lifetime of that connection.
The handshake process uses the following record schemas:
{
"type": "record",
"name": "HandshakeRequest", "namespace":"org.apache.avro.ipc",
"fields": [
{"name": "clientHash",
"type": {"type": "fixed", "name": "MD5", "size": 16}},
{"name": "clientProtocol", "type": ["null", "string"]},
{"name": "serverHash", "type": "MD5"},
{"name": "meta", "type": ["null", {"type": "map", "values": "bytes"}]}
]
}
{
"type": "record",
"name": "HandshakeResponse", "namespace": "org.apache.avro.ipc",
"fields": [
{"name": "match",
"type": {"type": "enum", "name": "HandshakeMatch",
"symbols": ["BOTH", "CLIENT", "NONE"]}},
{"name": "serverProtocol",
"type": ["null", "string"]},
{"name": "serverHash",
"type": ["null", {"type": "fixed", "name": "MD5", "size": 16}]},
{"name": "meta",
"type": ["null", {"type": "map", "values": "bytes"}]}
]
}
- A client first prefixes each request with a
HandshakeRequest containing just the hash of its protocol and of the server’s protocol (clientHash!=null, clientProtocol=null, serverHash!=null), where the hashes are 128-bit MD5 hashes of the JSON protocol text. If a client has never connected to a given server, it sends its hash as a guess of the server’s hash, otherwise it sends the hash that it previously obtained from this server.
The server responds with a HandshakeResponse containing one of:
-
match=BOTH, serverProtocol=null, serverHash=null if the client sent the valid hash of the server’s protocol and the server knows what protocol corresponds to the client’s hash. In this case, the request is complete and the response data immediately follows the HandshakeResponse.
-
match=CLIENT, serverProtocol!=null, serverHash!=null if the server has previously seen the client’s protocol, but the client sent an incorrect hash of the server’s protocol. The request is complete and the response data immediately follows the HandshakeResponse. The client must use the returned protocol to process the response and should also cache that protocol and its hash for future interactions with this server.
-
match=NONE if the server has not previously seen the client’s protocol. The serverHash and serverProtocol may also be non-null if the server’s protocol hash was incorrect.
In this case the client must then re-submit its request with its protocol text (clientHash!=null, clientProtocol!=null, serverHash!=null) and the server should respond with a successful match (match=BOTH, serverProtocol=null, serverHash=null) as above.
The meta field is reserved for future handshake enhancements.
A call consists of a request message paired with its resulting response or error message. Requests and responses contain extensible metadata, and both kinds of messages are framed as described above.
The format of a call request is:
- request metadata, a map with values of type bytes
- the message name, an Avro string, followed by
- the message parameters. Parameters are serialized according to the message’s request declaration.
When a message is declared one-way and a stateful connection has been established by a successful handshake response, no response data is sent. Otherwise the format of the call response is:
- response metadata, a map with values of type bytes
- a one-byte error flag boolean, followed by either:
- if the error flag is false, the message response, serialized per the message’s response schema.
- if the error flag is true, the error, serialized per the message’s effective error union schema.
Schema Resolution
A reader of Avro data, whether from an RPC or a file, can always parse that data because its schema is provided. But that schema may not be exactly the schema that was expected. For example, if the data was written with a different version of the software than it is read, then records may have had fields added or removed. This section specifies how such schema differences should be resolved.
We call the schema used to write the data as the writer’s schema, and the schema that the application expects the reader’s schema. Differences between these should be resolved as follows:
- It is an error if the two schemas do not match.
To match, one of the following must hold:
-
both schemas are arrays whose item types match
-
both schemas are maps whose value types match
-
both schemas are enums whose names match
-
both schemas are fixed whose sizes and names match
-
both schemas are records with the same name
-
either schema is a union
-
both schemas have same primitive type
-
the writer’s schema may be promoted to the reader’s as follows:
- int is promotable to long, float, or double
- long is promotable to float or double
- float is promotable to double
- string is promotable to bytes
- bytes is promotable to string
-
if both are records:
- the ordering of fields may be different: fields are matched by name.
- schemas for fields with the same name in both records are resolved recursively.
- if the writer’s record contains a field with a name not present in the reader’s record, the writer’s value for that field is ignored.
- if the reader’s record schema has a field that contains a default value, and writer’s schema does not have a field with the same name, then the reader should use the default value from its field.
- if the reader’s record schema has a field with no default value, and writer’s schema does not have a field with the same name, an error is signalled.
-
if both are enums:
if the writer’s symbol is not present in the reader’s enum, then an error is signalled.
This resolution algorithm is applied recursively to the reader’s and writer’s array item schemas.
This resolution algorithm is applied recursively to the reader’s and writer’s value schemas.
The first schema in the reader’s union that matches the selected writer’s union schema is recursively resolved against it. if none match, an error is signalled.
- if reader’s is a union, but writer’s is not:
The first schema in the reader’s union that matches the writer’s schema is recursively resolved against it. If none match, an error is signalled.
- if writer’s is a union, but reader’s is not:
If the reader’s schema matches the selected writer’s schema, it is recursively resolved against it. If they do not match, an error is signalled.
A schema’s “doc” fields are ignored for the purposes of schema resolution. Hence, the “doc” portion of a schema may be dropped at serialization.
30.2 - IDL Language
Introduction
This document defines Avro IDL, an experimental higher-level language for authoring Avro schemata. Before reading this document, you should have familiarity with the concepts of schemata and protocols, as well as the various primitive and complex types available in Avro.
N.B. This feature is considered experimental in the current version of Avro and the language has not been finalized. Although major changes are unlikely, some syntax may change in future versions of Avro.
Overview
Purpose
The aim of the Avro IDL language is to enable developers to author schemata in a way that feels more similar to common programming languages like Java, C++, or Python. Additionally, the Avro IDL language may feel more familiar for those users who have previously used the interface description languages (IDLs) in other frameworks like Thrift, Protocol Buffers, or CORBA.
Usage
Each Avro IDL file defines a single Avro Protocol, and thus generates as its output a JSON-format Avro Protocol file with extension .avpr.
To convert a .avdl file into a .avpr file, it must be processed by the idl tool. For example:
$ java -jar avroj-tools-1.4.0.jar idl src/test/idl/input/namespaces.avdl /tmp/namespaces.avpr
$ head /tmp/namespaces.avpr
{
"protocol" : "TestNamespace",
"namespace" : "avro.test.protocol",
The idl tool can also process input to and from stdin and stdout. See idl --help for full usage information.
Defining a Protocol in Avro IDL
An Avro IDL file consists of exactly one protocol definition. The minimal protocol is defined by the following code:
This is equivalent to (and generates) the following JSON protocol definition:
{
"protocol" : "MyProtocol",
"types" : [ ],
"messages" : {
}
}
The namespace of the protocol may be changed using the @namespace annotation:
@namespace("mynamespace")
protocol MyProtocol {
}
This notation is used throughout Avro IDL as a way of specifying properties for the annotated element, as will be described later in this document.
Protocols in Avro IDL can contain the following items:
- Imports of external protocol and schema files.
- Definitions of named schemata, including records, errors, enums, and fixeds.
- Definitions of RPC messages
Imports
Files may be imported in one of three formats:
-
An IDL file may be imported with a statement like:
import idl "foo.avdl";
-
A JSON protocol file may be imported with a statement like:
import protocol "foo.avpr";
-
A JSON schema file may be imported with a statement like:
import schema "foo.avsc";
Messages and types in the imported file are added to this file’s protocol.
Imported file names are resolved relative to the current IDL file.
Defining an Enumeration
Enums are defined in Avro IDL using a syntax similar to C or Java:
enum Suit {
SPADES, DIAMONDS, CLUBS, HEARTS
}
Note that, unlike the JSON format, anonymous enums cannot be defined.
Defining a Fixed Length Field
Fixed fields are defined using the following syntax:
fixed MD5(16);
This example defines a fixed-length type called MD5 which contains 16 bytes.
Defining Records and Errors
Records are defined in Avro IDL using a syntax similar to a struct definition in C:
record Employee {
string name;
boolean active = true;
long salary;
}
The above example defines a record with the name “Employee” with three fields.
To define an error, simply use the keyword error instead of record. For example:
error Kaboom {
string explanation;
int result_code = -1;
}
Each field in a record or error consists of a type and a name, optional property annotations and an optional default value.
A type reference in Avro IDL must be one of:
- A primitive type
- A logical type
- A named schema defined prior to this usage in the same Protocol
- A complex type (array, map, or union)
Primitive Types
The primitive types supported by Avro IDL are the same as those supported by Avro’s JSON format. This list includes int, long, string, boolean, float, double, null, and bytes.
References to Named Schemata
If a named schema has already been defined in the same Avro IDL file, it may be referenced by name as if it were a primitive type:
record Card {
Suit suit; // refers to the enum Card defined above
int number;
}
Default Values
Default values for fields may be optionally specified by using an equals sign after the field name followed by a JSON expression indicating the default value. This JSON is interpreted as described in the spec.
Complex Types
Arrays
Array types are written in a manner that will seem familiar to C++ or Java programmers. An array of any type t is denoted array<t>. For example, an array of strings is denoted array<string>, and a multidimensional array of Foo records would be array<array<Foo>>.
Maps
Map types are written similarly to array types. An array that contains values of type t is written map<t>. As in the JSON schema format, all maps contain string-type keys.
Unions
Union types are denoted as union { typeA, typeB, typeC, … }. For example, this record contains a string field that is optional (unioned with null):
record RecordWithUnion {
union { null, string } optionalString;
}
Note that the same restrictions apply to Avro IDL unions as apply to unions defined in the JSON format; namely, a record may not contain multiple elements of the same type.
Defining RPC Messages
The syntax to define an RPC message within a Avro IDL protocol is similar to the syntax for a method declaration within a C header file or a Java interface. To define an RPC message add which takes two arguments named foo and bar, returning an int, simply include the following definition within the protocol:
int add(int foo, int bar = 0);
Message arguments, like record fields, may specify default values.
To define a message with no response, you may use the alias void, equivalent to the Avro null type:
void logMessage(string message);
If you have previously defined an error type within the same protocol, you may declare that a message can throw this error using the syntax:
void goKaboom() throws Kaboom;
To define a one-way message, use the keyword oneway after the parameter list, for example:
void fireAndForget(string message) oneway;
Other Language Features
All Java-style comments are supported within a Avro IDL file. Any text following // on a line is ignored, as is any text between /* and */, possibly spanning multiple lines.
Escaping Identifiers
Occasionally, one will need to use a reserved language keyword as an identifier. In order to do so, backticks (`) may be used to escape the identifier. For example, to define a message with the literal name error, you may write:
This syntax is allowed anywhere an identifier is expected.
Annotations for Ordering and Namespaces
Java-style annotations may be used to add additional properties to types and fields throughout Avro IDL.
For example, to specify the sort order of a field within a record, one may use the @order annotation before the field name as follows:
record MyRecord {
string @order("ascending") myAscendingSortField;
string @order("descending") myDescendingField;
string @order("ignore") myIgnoredField;
}
A field’s type may also be preceded by annotations, e.g.:
record MyRecord {
@java-class("java.util.ArrayList") array string myStrings;
}
Similarly, a @namespace annotation may be used to modify the namespace when defining a named schema. For example:
@namespace("org.apache.avro.firstNamespace")
protocol MyProto {
@namespace("org.apache.avro.someOtherNamespace")
record Foo {}
record Bar {}
}
will define a protocol in the firstNamespace namespace. The record Foo will be defined in someOtherNamespace and Bar will be defined in firstNamespace as it inherits its default from its container.
Complete Example
The following is a complete example of a Avro IDL file that shows most of the above features:
/**
* An example protocol in Avro IDL
*/
@namespace("org.apache.avro.test")
protocol Simple {
@aliases(["org.foo.KindOf"])
enum Kind {
FOO,
BAR, // the bar enum value
BAZ
}
fixed MD5(16);
record TestRecord {
@order("ignore")
string name;
@order("descending")
Kind kind;
MD5 hash;
union { MD5, null} nullableHash;
array<long> arrayOfLongs;
}
error TestError {
string message;
}
string hello(string greeting);
TestRecord echo(TestRecord `record`);
int add(int arg1, int arg2);
bytes echoBytes(bytes data);
void `error`() throws TestError;
void ping() oneway;
}
Additional examples may be found in the Avro source tree under the src/test/idl/input directory.
30.3 - Wiki
The Wiki page can be found here.
30.4 - FAQ
What is Avro?
Avro is a data serialization system.
Avro provides:
- Rich data structures.
- A compact, fast, binary data format.
- A container file, to store persistent data.
- Remote procedure call (RPC).
- Simple integration with dynamic languages. Code generation is not required to read or write data files nor to use or implement RPC protocols. Code generation as an optional optimization, only worth implementing for statically typed languages.
How can I get started quickly with Avro?
Check out the Quick Start Guide.
How do I statically compile a schema or protocol into generated code?
In Java
- Add the avro jar, the jackson-mapper-asl.jar and jackson-core-asl.jar to your CLASSPATH.
- Run java org.apache.avro.specific.SpecificCompiler .
This appears to be out of date, the SpecificCompiler requires two arguments, presumably an input and and output file, but it isn’t clear that this does.
Or use the Schema or Protocol Ant tasks. Avro’s build.xml provides examples of how these are used.
Lastly, you can also use the “avro-tools” jar which ships with an Avro release. Just use the “compile (schema|protocol)” command.
How are Strings represented in Java?
They use org.apache.avro.util.Utf8, not java.lang.String.
More generally, how do Avro types map to Java types?
The mappings are documented in the package javadoc for generic, specific and reflect API.
From Doug Cutting:
HDFS splits files into blocks, and mapreduce runs a map task for each block. When the task starts, it needs to be able to seek into the file to the start of the block process through the block’s end. If the file were, e.g., a gzip file, this would not be possible, since gzip files must be decompressed from the start. One cannot seek into the middle of a gzip file and start decompressing. So Hadoop’s SequenceFile places a marker periodically (~64k) in the file at record and compression boundaries, where processing can be sensibly started. Then, when a map task starts processing an HDFS block, it finds the first marker after the block’s start and continues through the first marker in the next block of the file. This requires a bit of non-local access (~0.1%). Avro’s data file uses the same method as SequenceFile.
Why isn’t every value in Avro nullable?
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 of every value stored and
corresponding computational costs to create this bit on write and
interpret it on read. These costs are wasted when values may not in
fact be null, as is the case in many datasets. In Avro such costs are
only paid when values may actually be null.
Also, allowing values to be null is a well-known source of errors. In
Avro, a value declared as non-null will always be non-null and programs
need not test for null values when processing it nor will they ever fail
for lack of such tests.
Tony Hoare calls his invention of null references his “Billion Dollar
Mistake”.
http://qconlondon.com/london-2009/presentation/Null+References:+The+Billion+Dollar+Mistake
Also note that in some programming languages not all values are permitted to be null. For example, in Java, values of type boolean, byte, short, char, int, float, long, and double may not be null.
How can I serialize directly to/from a byte array?
As pointed out in the specification, Avro data should always be stored with its schema. The Avro provided classes DataFileWriter, DataFileReader, and DataFileStream all ensure this by serializing the Schema in a container header. In some special cases, such as when implementing a new storage system or writing unit tests, you may need to write and read directly with the bare Avro serialized values. The following examples use code generated for Java from an Avro Schema, specifically the User example from the Quickstart guide.
Serializing to a byte array
This example takes a User object and returns a newly allocated byte array with the Avro serialization of that user.
ByteArrayOutputStream out = new ByteArrayOutputStream();
BinaryEncoder encoder = EncoderFactory.get().binaryEncoder(out, null);
DatumWriter<User> writer = new SpecificDatumWriter<User>(User.getClassSchema());
writer.write(user, encoder);
encoder.flush();
out.close();
byte[] serializedBytes = out.toByteArray();
Deserializing from a byte array
This example takes a byte array containing the Avro serialization of a user and returns a User object.SpecificDatumReader reader = new SpecificDatumReader(User.getClassSchema());
Decoder decoder = DecoderFactory.get().binaryDecoder(bytes, null);
User user = reader.read(null, decoder);
Note: If you are serializing or deserializing in a loop or as a method, you should be reusing objects, readers and/or writers. Check the JavaDoc to see how to reuse the objects.
31 - Apache Avro™ 1.3.3 Documentation
Introduction
Avro™ is a data serialization system.
Avro provides:
- Rich data structures.
- A compact, fast, binary data format.
- A container file, to store persistent data.
- Remote procedure call (RPC).
- Simple integration with dynamic languages. Code generation is not required to read or write data files nor to use or implement RPC protocols. Code generation as an optional optimization, only worth implementing for statically typed languages.
Schemas
Avro relies on schemas. When Avro data is read, the schema used when writing it is always present. This permits each datum to be written with no per-value overheads, making serialization both fast and small. This also facilitates use with dynamic, scripting languages, since data, together with its schema, is fully self-describing.
When Avro data is stored in a file, its schema is stored with it, so that files may be processed later by any program. If the program reading the data expects a different schema this can be easily resolved, since both schemas are present.
When Avro is used in RPC, the client and server exchange schemas in the connection handshake. (This can be optimized so that, for most calls, no schemas are actually transmitted.) Since both client and server both have the other’s full schema, correspondence between same named fields, missing fields, extra fields, etc. can all be easily resolved.
Avro schemas are defined with JSON . This facilitates implementation in languages that already have JSON libraries.
Comparison with other systems
Avro provides functionality similar to systems such as Thrift, Protocol Buffers, etc. Avro differs from these systems in the following fundamental aspects.
- Dynamic typing: Avro does not require that code be generated. Data is always accompanied by a schema that permits full processing of that data without code generation, static datatypes, etc. This facilitates construction of generic data-processing systems and languages.
- Untagged data: Since the schema is present when data is read, considerably less type information need be encoded with data, resulting in smaller serialization size.
- No manually-assigned field IDs: When a schema changes, both the old and new schema are always present when processing data, so differences may be resolved symbolically, using field names.
31.1 - Specification
Introduction
This document defines Avro. It is intended to be the authoritative specification. Implementations of Avro must adhere to this document.
Schema Declaration
A Schema is represented in JSON by one of:
- A JSON string, naming a defined type.
- A JSON object, of the form:
{"type": "typeName" ...attributes...}
where typeName is either a primitive or derived type name, as defined below. Attributes not defined in this document are permitted as metadata, but must not affect the format of serialized data.
- A JSON array, representing a union of embedded types.
Primitive Types
The set of primitive type names is:
- null: no value
- boolean: a binary value
- int: 32-bit signed integer
- long: 64-bit signed integer
- float: single precision (32-bit) IEEE 754 floating-point number
- double: double precision (64-bit) IEEE 754 floating-point number
- bytes: sequence of 8-bit unsigned bytes
- string: unicode character sequence
Primitive types have no specified attributes.
Primitive type names are also defined type names. Thus, for example, the schema “string” is equivalent to:
Complex Types
Avro supports six kinds of complex types: records, enums, arrays, maps, unions and fixed.
Records
Records use the type name “record” and support three attributes:
- name: a JSON string providing the name of the record (required).
- namespace, a JSON string that qualifies the name;
- doc: a JSON string providing documentation to the user of this schema (optional).
- fields: a JSON array, listing fields (required). Each field is a JSON object with the following attributes:
- name: a JSON string providing the name of the field (required), and
- doc: a JSON string describing this field for users (optional).
- type: A JSON object defining a schema, or a JSON string naming a record definition (required).
- default: A default value for this field, used when reading instances that lack this field (optional). 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. Default values for bytes and fixed fields are JSON strings, where Unicode code points 0-255 are mapped to unsigned 8-bit byte values 0-255.
field default values
| avro type |
json type |
example |
| null |
null |
null |
| boolean |
boolean |
true |
| int,long |
integer |
1 |
| float,double |
number |
1.1 |
| bytes |
string |
"\u00FF" |
| string |
string |
"foo" |
| record |
object |
{"a": 1} |
| enum |
string |
"FOO" |
| array |
array |
[1] |
| map |
object |
{"a": 1} |
| fixed |
string |
"\u00ff" |
- order: specifies how this field impacts sort ordering of this record (optional). Valid values are “ascending” (the default), “descending”, or “ignore”. For more details on how this is used, see the sort order section below.
For example, a linked-list of 64-bit values may be defined with:
{
"type": "record",
"name": "LongList",
"fields" : [
{"name": "value", "type": "long"}, // each element has a long
{"name": "next", "type": ["LongList", "null"]} // optional next element
]
}
Enums
Enums use the type name “enum” and support the following attributes:
- name: a JSON string providing the name of the enum (required).
- namespace, a JSON string that qualifies the name ;
- doc: a JSON string providing documentation to the user of this schema (optional).
- symbols: a JSON array, listing symbols, as JSON strings (required). All symbols in an enum must be unique; duplicates are prohibited.
For example, playing card suits might be defined with:
{
"type": "enum",
"name": "Suit",
"symbols" : ["SPADES", "HEARTS", "DIAMONDS", "CLUBS"]
}
Arrays
Arrays use the type name “array” and support a single attribute:
- items: the schema of the array’s items.
For example, an array of strings is declared with:
{
"type": "array",
"items" : "string"
}
Maps
Maps use the type name “map” and support one attribute:
- values: the schema of the map’s values.
Map keys are assumed to be strings.
For example, a map from string to long is declared with:
{
"type": "map",
"values" : "long"
}
Unions
Unions, as mentioned above, are represented using JSON arrays. For example, ["string", "null"] declares a schema which may be either a string or null.
Unions may not contain more than one schema with the same type, except for the named types record, fixed and enum. For example, unions containing two array types or two map types are not permitted, but two types with different names are permitted. (Names permit efficient resolution when reading and writing unions.)
Unions may not immediately contain other unions.
Fixed
Fixed uses the type name “fixed” and supports two attributes:
- name: a string naming this fixed (required).
- namespace, a string that qualifies the name;
- size: an integer, specifying the number of bytes per value (required).
For example, 16-byte quantity may be declared with:
{"type": "fixed", "size": 16, "name": "md5"}
Names
Record, enums and fixed are named types. Each has a fullname that is composed of two parts; a name and a namespace. Equality of names is defined on the fullname.
The name portion of a fullname, and record field names must:
- start with [A-Za-z_]
- subsequently contain only [A-Za-z0-9_]
A namespace is a dot-separated sequence of such names.
In record, enum and fixed definitions, the fullname is determined in one of the following ways:
A name and namespace are both specified. For example, one might use “name”: “X”, “namespace”: “org.foo” to indicate the fullname org.foo.X.
-
A fullname is specified. If the name specified contains a dot, then it is assumed to be a fullname, and any namespace also specified is ignored. For example, use “name”: “org.foo.X” to indicate the fullname org.foo.X.
-
A name only is specified, i.e., a name that contains no dots. In this case the namespace is taken from the most tightly enclosing schema or protocol. For example, if “name”: “X” is specified, and this occurs within a field of the record definition of org.foo.Y, then the fullname is org.foo.X.
References to previously defined names are as in the latter two cases above: if they contain a dot they are a fullname, if they do not contain a dot, the namespace is the namespace of the enclosing definition.
Primitive type names have no namespace and their names may not be defined in any namespace. A schema may only contain multiple definitions of a fullname if the definitions are equivalent.
Data Serialization
Avro data is always serialized with its schema. Files that store Avro data should always also include the schema for that data in the same file. Avro-based remote procedure call (RPC) systems must also guarantee that remote recipients of data have a copy of the schema used to write that data.
Because the schema used to write data is always available when the data is read, Avro data itself is not tagged with type information. The schema is required to parse data.
In general, both serialization and deserialization proceed as a depth-first, left-to-right traversal of the schema, serializing primitive types as they are encountered.
Encodings
Avro specifies two serialization encodings: binary and JSON. Most applications will use the binary encoding, as it is smaller and faster. But, for debugging and web-based applications, the JSON encoding may sometimes be appropriate.
Binary Encoding
Primitive Types
Primitive types are encoded in binary as follows:
- null is written as zero bytes.
- a boolean is written as a single byte whose value is either 0 (false) or 1 (true).
- int and long values are written using variable-length zig-zag coding. Some examples:
| value |
hex |
| 0 |
00 |
| -1 |
01 |
| 1 |
02 |
| -2 |
03 |
| 2 |
04 |
| … |
|
| -64 |
7f |
| 64 |
80 01 |
| … |
|
- a float is written as 4 bytes. The float is converted into a 32-bit integer using a method equivalent to Java’s floatToIntBits and then encoded in little-endian format.
- a double is written as 8 bytes. The double is converted into a 64-bit integer using a method equivalent to Java’s doubleToLongBits and then encoded in little-endian format.
- bytes are encoded as a long followed by that many bytes of data.
- a string is encoded as a long followed by that many bytes of UTF-8 encoded character data.
For example, the three-character string “foo” would be encoded as the long value 3 (encoded as hex 06) followed by the UTF-8 encoding of ‘f’, ‘o’, and ‘o’ (the hex bytes 66 6f 6f):
06 66 6f 6f
Complex Types
Complex types are encoded in binary as follows:
Records
A record is encoded by encoding the values of its fields in the order that they are declared. In other words, a record is encoded as just the concatenation of the encodings of its fields. Field values are encoded per their schema.
For example, the record schema
{
"type": "record",
"name": "test",
"fields" : [
{"name": "a", "type": "long"},
{"name": "b", "type": "string"}
]
}
An instance of this record whose a field has value 27 (encoded as hex 36) and whose b field has value “foo” (encoded as hex bytes 06 66 6f 6f), would be encoded simply as the concatenation of these, namely the hex byte sequence:
36 06 66 6f 6f
Enums
An enum is encoded by a int, representing the zero-based position of the symbol in the schema.
For example, consider the enum:
{"type": "enum", "name": "Foo", "symbols": ["A", "B", "C", "D"] }
This would be encoded by an int between zero and three, with zero indicating “A”, and 3 indicating “D”.
Arrays
Arrays are encoded as a series of blocks. Each block consists of a long count value, followed by that many array items. A block with count zero indicates the end of the array. Each item is encoded per the array’s item schema.
If a block’s count is negative, its absolute value is used, and the count is followed immediately by a long block size indicating the number of bytes in the block. This block size permits fast skipping through data, e.g., when projecting a record to a subset of its fields.
For example, the array schema
{"type": "array", "items": "long"}
an array containing the items 3 and 27 could be encoded as the long value 2 (encoded as hex 04) followed by long values 3 and 27 (encoded as hex 06 36) terminated by zero:
04 06 36 00
The blocked representation permits one to read and write arrays larger than can be buffered in memory, since one can start writing items without knowing the full length of the array.
Maps
Maps are encoded as a series of blocks. Each block consists of a long count value, followed by that many key/value pairs. A block with count zero indicates the end of the map. Each item is encoded per the map’s value schema.
If a block’s count is negative, its absolute value is used, and the count is followed immediately by a long block size indicating the number of bytes in the block. This block size permits fast skipping through data, e.g., when projecting a record to a subset of its fields.
The blocked representation permits one to read and write maps larger than can be buffered in memory, since one can start writing items without knowing the full length of the map.
Unions
A union is encoded by first writing a long value indicating the zero-based position within the union of the schema of its value. The value is then encoded per the indicated schema within the union.
For example, the union schema ["string","null"] would encode:
-
null as the integer 1 (the index of “null” in the union, encoded as hex 02):
02
-
the string “a” as zero (the index of “string” in the union), followed by the serialized string:
02 02 61
Fixed
Fixed instances are encoded using the number of bytes declared in the schema.
JSON Encoding
Except for unions, the JSON encoding is the same as is used to encode field default values.
The value of a union is encoded in JSON as follows:
- if its type is null, then it is encoded as a JSON null;
- otherwise it is encoded as a JSON object with one name/value pair whose name is the type’s name and whose value is the recursively encoded value. For Avro’s named types (record, fixed or enum) the user-specified name is used, for other types the type name is used.
For example, the union schema ["null","string","Foo"], where Foo is a record name, would encode:
- null as null;
- the string “a” as
{"string": "a"} and
- a Foo instance as
{"Foo": {...}}, where {...} indicates the JSON encoding of a Foo instance.
Note that a schema is still required to correctly process JSON-encoded data. For example, the JSON encoding does not distinguish between int and long, float and double, records and maps, enums and strings, etc.
Sort Order
Avro defines a standard sort order for data. This permits data written by one system to be efficiently sorted by another system. This can be an important optimization, as sort order comparisons are sometimes the most frequent per-object operation. Note also that Avro binary-encoded data can be efficiently ordered without deserializing it to objects.
Data items may only be compared if they have identical schemas. Pairwise comparisons are implemented recursively with a depth-first, left-to-right traversal of the schema. The first mismatch encountered determines the order of the items.
Two items with the same schema are compared according to the following rules.
- null data is always equal.
- boolean data is ordered with false before true.
- int, long, float and double data is ordered by ascending numeric value.
- bytes and fixed data are compared lexicographically by unsigned 8-bit values.
- string data is compared lexicographically by Unicode code point. Note that since UTF-8 is used as the binary encoding for strings, sorting of bytes and string binary data is identical.
- array data is compared lexicographically by element.
- enum data is ordered by the symbol’s position in the enum schema. For example, an enum whose symbols are
["z", "a"] would sort “z” values before “a” values.
- union data is first ordered by the branch within the union, and, within that, by the type of the branch. For example, an
["int", "string"] union would order all int values before all string values, with the ints and strings themselves ordered as defined above.
- record data is ordered lexicographically by field. If a field specifies that its order is:
- “ascending”, then the order of its values is unaltered.
- “descending”, then the order of its values is reversed.
- “ignore”, then its values are ignored when sorting.
- map data may not be compared. It is an error to attempt to compare data containing maps unless those maps are in an
"order":"ignore" record field.
Object Container Files
Avro includes a simple object container file format. A file has a schema, and all objects stored in the file must be written according to that schema, using binary encoding. Objects are stored in blocks that may be compressed. Syncronization markers are used between blocks to permit efficient splitting of files for MapReduce processing.
Files may include arbitrary user-specified metadata.
A file consists of:
- A file header, followed by
- one or more file data blocks.
A file header consists of:
- Four bytes, ASCII ‘O’, ‘b’, ‘j’, followed by 1.
- file metadata, including the schema.
- The 16-byte, randomly-generated sync marker for this file.
File metadata consists of:
A long indicating the number of metadata key/value pairs.
For each pair, a string key and bytes value.
All metadata properties that start with “avro.” are reserved. The following file metadata properties are currently used:
- avro.schema contains the schema of objects stored in the file, as JSON data (required).
- avro.codec the name of the compression codec used to compress blocks, as a string. Implementations are required to support the following codecs: “null” and “deflate”. If codec is absent, it is assumed to be “null”. The codecs are described with more detail below.
A file header is thus described by the following schema:
{"type": "record", "name": "org.apache.avro.file.Header",
"fields" : [
{"name": "magic", "type": {"type": "fixed", "name": "Magic", "size": 4}},
{"name": "meta", "type": {"type": "map", "values": "bytes"}},
{"name": "sync", "type": {"type": "fixed", "name": "Sync", "size": 16}}
]
}
A file data block consists of:
- A long indicating the count of objects in this block.
- A long indicating the size in bytes of the serialized objects in the current block, after any codec is applied
- The serialized objects. If a codec is specified, this is compressed by that codec.
- The file’s 16-byte sync marker.
Thus, each block’s binary data can be efficiently extracted or skipped without deserializing the contents. The combination of block size, object counts, and sync markers enable detection of corrupt blocks and help ensure data integrity.
Required Codecs
null
The “null” codec simply passes through data uncompressed.
deflate
The “deflate” codec writes the data block using the deflate algorithm as specified in RFC 1951, and typically implemented using the zlib library. Note that this format (unlike the “zlib format” in RFC 1950) does not have a checksum.
Protocol Declaration
Avro protocols describe RPC interfaces. Like schemas, they are defined with JSON text.
A protocol is a JSON object with the following attributes:
- protocol, a string, the name of the protocol (required);
- namespace, an optional string that qualifies the name;
- doc, an optional string describing this protocol;
- types, an optional list of definitions of named types (records, enums, fixed and errors). An error definition is just like a record definition except it uses “error” instead of “record”. Note that forward references to named types are not permitted.
- messages, an optional JSON object whose keys are message names and whose values are objects whose attributes are described below. No two messages may have the same name.
The name and namespace qualification rules defined for schema objects apply to protocols as well.
Messages
A message has attributes:
- a doc, an optional description of the message,
- a request, a list of named, typed parameter schemas (this has the same form as the fields of a record declaration);
- a response schema;
- an optional union of error schemas.* an optional one-way boolean parameter.
A request parameter list is processed equivalently to an anonymous record. Since record field lists may vary between reader and writer, request parameters may also differ between the caller and responder, and such differences are resolved in the same manner as record field differences.
Sample Protocol
For example, one may define a simple HelloWorld protocol with:
{
"namespace": "com.acme",
"protocol": "HelloWorld",
"doc": "Protocol Greetings",
"types": [
{"name": "Greeting", "type": "record", "fields": [
{"name": "message", "type": "string"}]},
{"name": "Curse", "type": "error", "fields": [
{"name": "message", "type": "string"}]}
],
"messages": {
"hello": {
"doc": "Say hello.",
"request": [{"name": "greeting", "type": "Greeting" }],
"response": "Greeting",
"errors": ["Curse"]
}
}
}
Message Transport
Messages may be transmitted via different transport mechanisms.
To the transport, a message is an opaque byte sequence.
A transport is a system that supports:
- transmission of request messages
- receipt of corresponding response messages
Servers may send a response message back to the client corresponding to a request message. The mechanism of correspondance is transport-specific. For example, in HTTP it is implicit, since HTTP directly supports requests and responses. But a transport that multiplexes many client threads over a single socket would need to tag messages with unique identifiers.
HTTP as Transport
When HTTP is used as a transport, each Avro message exchange is an HTTP request/response pair. All messages of an Avro protocol should share a single URL at an HTTP server. Other protocols may also use that URL. Both normal and error Avro response messages should use the 200 (OK) response code. The chunked encoding may be used for requests and responses, but, regardless the Avro request and response are the entire content of an HTTP request and response. The HTTP Content-Type of requests and responses should be specified as “avro/binary”. Requests should be made using the POST method.
Message Framing
Avro messages are framed as a list of buffers.
Framing is a layer between messages and the transport. It exists to optimize certain operations.
The format of framed message data is:
- a series of buffers, where each buffer consists of:
- a four-byte, big-endian buffer length, followed by
- that many bytes of buffer data.
- A message is always terminated by a zero-lenghted buffer.
Framing is transparent to request and response message formats (described below). Any message may be presented as a single or multiple buffers.
Framing can permit readers to more efficiently get different buffers from different sources and for writers to more efficiently store different buffers to different destinations. In particular, it can reduce the number of times large binary objects are copied. For example, if an RPC parameter consists of a megabyte of file data, that data can be copied directly to a socket from a file descriptor, and, on the other end, it could be written directly to a file descriptor, never entering user space.
A simple, recommended, framing policy is for writers to create a new segment whenever a single binary object is written that is larger than a normal output buffer. Small objects are then appended in buffers, while larger objects are written as their own buffers. When a reader then tries to read a large object the runtime can hand it an entire buffer directly, without having to copy it.
Handshake
The purpose of the handshake is to ensure that the client and the server have each other’s protocol definition, so that the client can correctly deserialize responses, and the server can correctly deserialize requests. Both clients and servers should maintain a cache of recently seen protocols, so that, in most cases, a handshake will be completed without extra round-trip network exchanges or the transmission of full protocol text.
The handshake process uses the following record schemas:
{
"type": "record",
"name": "HandshakeRequest", "namespace":"org.apache.avro.ipc",
"fields": [
{"name": "clientHash",
"type": {"type": "fixed", "name": "MD5", "size": 16}},
{"name": "clientProtocol", "type": ["null", "string"]},
{"name": "serverHash", "type": "MD5"},
{"name": "meta", "type": ["null", {"type": "map", "values": "bytes"}]}
]
}
{
"type": "record",
"name": "HandshakeResponse", "namespace": "org.apache.avro.ipc",
"fields": [
{"name": "match",
"type": {"type": "enum", "name": "HandshakeMatch",
"symbols": ["BOTH", "CLIENT", "NONE"]}},
{"name": "serverProtocol",
"type": ["null", "string"]},
{"name": "serverHash",
"type": ["null", {"type": "fixed", "name": "MD5", "size": 16}]},
{"name": "meta",
"type": ["null", {"type": "map", "values": "bytes"}]}
]
}
- A client first prefixes each request with a
HandshakeRequest containing just the hash of its protocol and of the server’s protocol (clientHash!=null, clientProtocol=null, serverHash!=null), where the hashes are 128-bit MD5 hashes of the JSON protocol text. If a client has never connected to a given server, it sends its hash as a guess of the server’s hash, otherwise it sends the hash that it previously obtained from this server.
The server responds with a HandshakeResponse containing one of:
-
match=BOTH, serverProtocol=null, serverHash=null if the client sent the valid hash of the server’s protocol and the server knows what protocol corresponds to the client’s hash. In this case, the request is complete and the response data immediately follows the HandshakeResponse.
-
match=CLIENT, serverProtocol!=null, serverHash!=null if the server has previously seen the client’s protocol, but the client sent an incorrect hash of the server’s protocol. The request is complete and the response data immediately follows the HandshakeResponse. The client must use the returned protocol to process the response and should also cache that protocol and its hash for future interactions with this server.
-
match=NONE if the server has not previously seen the client’s protocol. The serverHash and serverProtocol may also be non-null if the server’s protocol hash was incorrect.
In this case the client must then re-submit its request with its protocol text (clientHash!=null, clientProtocol!=null, serverHash!=null) and the server should respond with a successful match (match=BOTH, serverProtocol=null, serverHash=null) as above.
The meta field is reserved for future handshake enhancements.
A call consists of a request message paired with its resulting response or error message. Requests and responses contain extensible metadata, and both kinds of messages are framed as described above.
The format of a call request is:
- request metadata, a map with values of type bytes
- the message name, an Avro string, followed by
- the message parameters. Parameters are serialized according to the message’s request declaration.
The format of a call response is:
- response metadata, a map with values of type bytes
- a one-byte error flag boolean, followed by either:
- if the error flag is false, the message response, serialized per the message’s response schema.
- if the error flag is true, the error, serialized per the message’s effective error union schema.
Schema Resolution
A reader of Avro data, whether from an RPC or a file, can always parse that data because its schema is provided. But that schema may not be exactly the schema that was expected. For example, if the data was written with a different version of the software than it is read, then records may have had fields added or removed. This section specifies how such schema differences should be resolved.
We call the schema used to write the data as the writer’s schema, and the schema that the application expects the reader’s schema. Differences between these should be resolved as follows:
- It is an error if the two schemas do not match.
To match, one of the following must hold:
-
both schemas are arrays whose item types match
-
both schemas are maps whose value types match
-
both schemas are enums whose names match
-
both schemas are fixed whose sizes and names match
-
both schemas are records with the same name
-
either schema is a union
-
both schemas have same primitive type
-
the writer’s schema may be promoted to the reader’s as follows:
- int is promotable to long, float, or double
- long is promotable to float or double
- float is promotable to double
- string is promotable to bytes
- bytes is promotable to string
-
if both are records:
- the ordering of fields may be different: fields are matched by name.
- schemas for fields with the same name in both records are resolved recursively.
- if the writer’s record contains a field with a name not present in the reader’s record, the writer’s value for that field is ignored.
- if the reader’s record schema has a field that contains a default value, and writer’s schema does not have a field with the same name, then the reader should use the default value from its field.
- if the reader’s record schema has a field with no default value, and writer’s schema does not have a field with the same name, an error is signalled.
-
if both are enums:
if the writer’s symbol is not present in the reader’s enum, then an error is signalled.
This resolution algorithm is applied recursively to the reader’s and writer’s array item schemas.
This resolution algorithm is applied recursively to the reader’s and writer’s value schemas.
The first schema in the reader’s union that matches the selected writer’s union schema is recursively resolved against it. if none match, an error is signalled.
- if reader’s is a union, but writer’s is not:
The first schema in the reader’s union that matches the writer’s schema is recursively resolved against it. If none match, an error is signalled.
- if writer’s is a union, but reader’s is not:
If the reader’s schema matches the selected writer’s schema, it is recursively resolved against it. If they do not match, an error is signalled.
A schema’s “doc” fields are ignored for the purposes of schema resolution. Hence, the “doc” portion of a schema may be dropped at serialization.
31.2 - GenAvro Language
Introduction
This document defines GenAvro, an experimental higher-level language for authoring Avro schemata. Before reading this document, you should have familiarity with the concepts of schemata and protocols, as well as the various primitive and complex types available in Avro.
N.B. This feature is considered experimental in the current version of Avro and the language has not been finalized. Although major changes are unlikely, some syntax may change in future versions of Avro.
Overview
Purpose
The aim of the GenAvro language is to enable developers to author schemata in a way that feels more similar to common programming languages like Java, C++, or Python. Additionally, the GenAvro language may feel more familiar for those users who have previously used the interface description languages (IDLs) in other frameworks like Thrift, Protocol Buffers, or CORBA.
Usage
Each GenAvro file defines a single Avro Protocol, and thus generates as its output a JSON-format Avro Protocol file with extension .avpr.
To convert a .genavro file into a .avpr file, it must be processed by the avroj genavro tool. For example:
$ java -jar avroj-1.3.0.jar genavro src/test/genavro/input/namespaces.genavro /tmp/namespaces.avpr
$ head /tmp/namespaces.avpr{
"protocol" : "TestNamespace",
"namespace" : "avro.test.protocol",
The avroj genavro tool can also process input to and from stdin and stdout. See avroj genavro –help for full usage information.
Defining a Protocol in GenAvro
A GenAvro file consists of exactly one protocol definition. The minimal protocol is defined by the following code:
This is equivalent to (and generates) the following JSON protocol definition:
{
"protocol" : "MyProtocol",
"types" : [ ],
"messages" : {
}
}
The namespace of the protocol may be changed using the @namespace annotation:
@namespace("mynamespace")
protocol MyProtocol {
}
3 This notation is used throughout GenAvro as a way of specifying properties for the annotated element, as will be described later in this document.
Protocols in GenAvro can contain the following items:
- Definitions of named schemata, including records, errors, enums, and fixeds.
- Definitions of RPC messages
Defining an Enumeration
Enums are defined in GenAvro using a syntax similar to C or Java:
enum Suit {
SPADES, DIAMONDS, CLUBS, HEARTS
}
Note that, unlike the JSON format, anonymous enums cannot be defined.
Defining a Fixed Length Field
Fixed fields are defined using the following syntax:
fixed MD5(16);
This example defines a fixed-length type called MD5 which contains 16 bytes.
Defining Records and Errors
Records are defined in GenAvro using a syntax similar to a struct definition in C:
record Employee {
string name;
boolean active;
long salary;
}
The above example defines a record with the name “Employee” with three fields.
To define an error, simply use the keyword error instead of record. For example:
error Kaboom {
string explanation;
int result_code;
}
Each field in a record or error consists of a type and a name, along with optional property annotations.
A type reference in GenAvro must be one of:
- A primitive type
- A named schema defined prior to this usage in the same Protocol
- A complex type (array, map, or union)
Primitive Types
The primitive types supported by GenAvro are the same as those supported by Avro’s JSON format. This list includes int, long, string, boolean, float, double, null, and bytes.
References to Named Schemata
If a named schema has already been defined in the same GenAvro file, it may be referenced by name as if it were a primitive type:
record Card {
Suit suit; // refers to the enum Card defined above
int number;
}
Complex Types
Arrays
Array types are written in a manner that will seem familiar to C++ or Java programmers. An array of any type t is denoted array<t>. For example, an array of strings is denoted array<string>, and a multidimensional array of Foo records would be array<array<Foo>>.
Maps
Map types are written similarly to array types. An array that contains values of type t is written map<t>. As in the JSON schema format, all maps contain string-type keys.
Unions
Union types are denoted as union { typeA, typeB, typeC, … }. For example, this record contains a string field that is optional (unioned with null):
record RecordWithUnion {
union { null, string } optionalString;
}
Note that the same restrictions apply to GenAvro unions as apply to unions defined in the JSON format; namely, a record may not contain multiple elements of the same type.
Defining RPC Messages
The syntax to define an RPC message within a GenAvro protocol is similar to the syntax for a method declaration within a C header file or a Java interface. To define an RPC message add which takes two arguments named foo and bar, returning an int, simply include the following definition within the protocol:
int add(int foo, int bar = 0);
To define a message with no response, you may use the alias void, equivalent to the Avro null type:
void logMessage(string message);
If you have previously defined an error type within the same protocol, you may declare that a message can throw this error using the syntax:
void goKaboom() throws Kaboom;
Other Language Features
All Java-style comments are supported within a GenAvro file. Any text following // on a line is ignored, as is any text between /* and */, possibly spanning multiple lines.
Escaping Identifiers
Occasionally, one will need to use a reserved language keyword as an identifier. In order to do so, backticks (`) may be used to escape the identifier. For example, to define a message with the literal name error, you may write:
This syntax is allowed anywhere an identifier is expected.
Annotations for Ordering and Namespaces
Java-style annotations may be used to add additional properties to types throughout GenAvro. For example, to specify the sort order of a field within a record, one may use the @order annotation as follows:
record MyRecord {
@order("ascending")
string myAscendingSortField;
@order("descending")
string myDescendingField;
@order("ignore")
string myIgnoredField;
}
Similarly, a @namespace annotation may be used to modify the namespace when defining a named schema. For example:
@namespace("org.apache.avro.firstNamespace")
protocol MyProto {
@namespace("org.apache.avro.someOtherNamespace")
record Foo {}
record Bar {}
}
will define a protocol in the firstNamespace namespace. The record Foo will be defined in someOtherNamespace and Bar will be defined in firstNamespace as it inherits its default from its container.
Complete Example
The following is a complete example of a GenAvro file that shows most of the above features:
/**
* An example protocol in GenAvro
*/
@namespace("org.apache.avro.test")
protocol Simple {
@aliases(["org.foo.KindOf"])
enum Kind {
FOO,
BAR, // the bar enum value
BAZ
}
fixed MD5(16);
record TestRecord {
@order("ignore")
string name;
@order("descending")
Kind kind;
MD5 hash;
union { MD5, null} nullableHash;
array<long> arrayOfLongs;
}
error TestError {
string message;
}
string hello(string greeting);
TestRecord echo(TestRecord `record`);
int add(int arg1, int arg2);
bytes echoBytes(bytes data);
void `error`() throws TestError;
void ping() oneway;
}
Additional examples may be found in the Avro source tree under the src/test/genavro/input directory.
31.3 - Wiki
The Wiki page can be found here.
31.4 - FAQ
What is Avro?
Avro is a data serialization system.
Avro provides:
- Rich data structures.
- A compact, fast, binary data format.
- A container file, to store persistent data.
- Remote procedure call (RPC).
- Simple integration with dynamic languages. Code generation is not required to read or write data files nor to use or implement RPC protocols. Code generation as an optional optimization, only worth implementing for statically typed languages.
How can I get started quickly with Avro?
Check out the Quick Start Guide.
How do I statically compile a schema or protocol into generated code?
In Java
- Add the avro jar, the jackson-mapper-asl.jar and jackson-core-asl.jar to your CLASSPATH.
- Run java org.apache.avro.specific.SpecificCompiler .
This appears to be out of date, the SpecificCompiler requires two arguments, presumably an input and and output file, but it isn’t clear that this does.
Or use the Schema or Protocol Ant tasks. Avro’s build.xml provides examples of how these are used.
Lastly, you can also use the “avro-tools” jar which ships with an Avro release. Just use the “compile (schema|protocol)” command.
How are Strings represented in Java?
They use org.apache.avro.util.Utf8, not java.lang.String.
More generally, how do Avro types map to Java types?
The mappings are documented in the package javadoc for generic, specific and reflect API.
From Doug Cutting:
HDFS splits files into blocks, and mapreduce runs a map task for each block. When the task starts, it needs to be able to seek into the file to the start of the block process through the block’s end. If the file were, e.g., a gzip file, this would not be possible, since gzip files must be decompressed from the start. One cannot seek into the middle of a gzip file and start decompressing. So Hadoop’s SequenceFile places a marker periodically (~64k) in the file at record and compression boundaries, where processing can be sensibly started. Then, when a map task starts processing an HDFS block, it finds the first marker after the block’s start and continues through the first marker in the next block of the file. This requires a bit of non-local access (~0.1%). Avro’s data file uses the same method as SequenceFile.
Why isn’t every value in Avro nullable?
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 of every value stored and
corresponding computational costs to create this bit on write and
interpret it on read. These costs are wasted when values may not in
fact be null, as is the case in many datasets. In Avro such costs are
only paid when values may actually be null.
Also, allowing values to be null is a well-known source of errors. In
Avro, a value declared as non-null will always be non-null and programs
need not test for null values when processing it nor will they ever fail
for lack of such tests.
Tony Hoare calls his invention of null references his “Billion Dollar
Mistake”.
http://qconlondon.com/london-2009/presentation/Null+References:+The+Billion+Dollar+Mistake
Also note that in some programming languages not all values are permitted to be null. For example, in Java, values of type boolean, byte, short, char, int, float, long, and double may not be null.
How can I serialize directly to/from a byte array?
As pointed out in the specification, Avro data should always be stored with its schema. The Avro provided classes DataFileWriter, DataFileReader, and DataFileStream all ensure this by serializing the Schema in a container header. In some special cases, such as when implementing a new storage system or writing unit tests, you may need to write and read directly with the bare Avro serialized values. The following examples use code generated for Java from an Avro Schema, specifically the User example from the Quickstart guide.
Serializing to a byte array
This example takes a User object and returns a newly allocated byte array with the Avro serialization of that user.
ByteArrayOutputStream out = new ByteArrayOutputStream();
BinaryEncoder encoder = EncoderFactory.get().binaryEncoder(out, null);
DatumWriter<User> writer = new SpecificDatumWriter<User>(User.getClassSchema());
writer.write(user, encoder);
encoder.flush();
out.close();
byte[] serializedBytes = out.toByteArray();
Deserializing from a byte array
This example takes a byte array containing the Avro serialization of a user and returns a User object.SpecificDatumReader reader = new SpecificDatumReader(User.getClassSchema());
Decoder decoder = DecoderFactory.get().binaryDecoder(bytes, null);
User user = reader.read(null, decoder);
Note: If you are serializing or deserializing in a loop or as a method, you should be reusing objects, readers and/or writers. Check the JavaDoc to see how to reuse the objects.
32 - Apache Avro™ 1.3.2 Documentation
Introduction
Avro is a data serialization system.
Avro provides:
- Rich data structures.
- A compact, fast, binary data format.
- A container file, to store persistent data.
- Remote procedure call (RPC).
- Simple integration with dynamic languages. Code generation is not required to read or write data files nor to use or implement RPC protocols. Code generation as an optional optimization, only worth implementing for statically typed languages.
Schemas
Avro relies on schemas. When Avro data is read, the schema used when writing it is always present. This permits each datum to be written with no per-value overheads, making serialization both fast and small. This also facilitates use with dynamic, scripting languages, since data, together with its schema, is fully self-describing.
When Avro data is stored in a file, its schema is stored with it, so that files may be processed later by any program. If the program reading the data expects a different schema this can be easily resolved, since both schemas are present.
When Avro is used in RPC, the client and server exchange schemas in the connection handshake. (This can be optimized so that, for most calls, no schemas are actually transmitted.) Since both client and server both have the other’s full schema, correspondence between same named fields, missing fields, extra fields, etc. can all be easily resolved.
Avro schemas are defined with JSON . This facilitates implementation in languages that already have JSON libraries.
Comparison with other systems
Avro provides functionality similar to systems such as Thrift, Protocol Buffers, etc. Avro differs from these systems in the following fundamental aspects.
- Dynamic typing: Avro does not require that code be generated. Data is always accompanied by a schema that permits full processing of that data without code generation, static datatypes, etc. This facilitates construction of generic data-processing systems and languages.
- Untagged data: Since the schema is present when data is read, considerably less type information need be encoded with data, resulting in smaller serialization size.
- No manually-assigned field IDs: When a schema changes, both the old and new schema are always present when processing data, so differences may be resolved symbolically, using field names.
32.1 - Specification
Introduction
This document defines Avro. It is intended to be the authoritative specification. Implementations of Avro must adhere to this document.
Schema Declaration
A Schema is represented in JSON by one of:
- A JSON string, naming a defined type.
- A JSON object, of the form:
{"type": "typeName" ...attributes...}
where typeName is either a primitive or derived type name, as defined below. Attributes not defined in this document are permitted as metadata, but must not affect the format of serialized data.
- A JSON array, representing a union of embedded types.
Primitive Types
The set of primitive type names is:
- null: no value
- boolean: a binary value
- int: 32-bit signed integer
- long: 64-bit signed integer
- float: single precision (32-bit) IEEE 754 floating-point number
- double: double precision (64-bit) IEEE 754 floating-point number
- bytes: sequence of 8-bit unsigned bytes
- string: unicode character sequence
Primitive types have no specified attributes.
Primitive type names are also defined type names. Thus, for example, the schema “string” is equivalent to:
Complex Types
Avro supports six kinds of complex types: records, enums, arrays, maps, unions and fixed.
Records
Records use the type name “record” and support three attributes:
- name: a JSON string providing the name of the record (required).
- namespace, a JSON string that qualifies the name;
- doc: a JSON string providing documentation to the user of this schema (optional).
- fields: a JSON array, listing fields (required). Each field is a JSON object with the following attributes:
- name: a JSON string providing the name of the field (required), and
- doc: a JSON string describing this field for users (optional).
- type: A JSON object defining a schema, or a JSON string naming a record definition (required).
- default: A default value for this field, used when reading instances that lack this field (optional). 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. Default values for bytes and fixed fields are JSON strings, where Unicode code points 0-255 are mapped to unsigned 8-bit byte values 0-255.
field default values
| avro type |
json type |
example |
| null |
null |
null |
| boolean |
boolean |
true |
| int,long |
integer |
1 |
| float,double |
number |
1.1 |
| bytes |
string |
"\u00FF" |
| string |
string |
"foo" |
| record |
object |
{"a": 1} |
| enum |
string |
"FOO" |
| array |
array |
[1] |
| map |
object |
{"a": 1} |
| fixed |
string |
"\u00ff" |
- order: specifies how this field impacts sort ordering of this record (optional). Valid values are “ascending” (the default), “descending”, or “ignore”. For more details on how this is used, see the sort order section below.
For example, a linked-list of 64-bit values may be defined with:
{
"type": "record",
"name": "LongList",
"fields" : [
{"name": "value", "type": "long"}, // each element has a long
{"name": "next", "type": ["LongList", "null"]} // optional next element
]
}
Enums
Enums use the type name “enum” and support the following attributes:
- name: a JSON string providing the name of the enum (required).
- namespace, a JSON string that qualifies the name ;
- doc: a JSON string providing documentation to the user of this schema (optional).
- symbols: a JSON array, listing symbols, as JSON strings (required). All symbols in an enum must be unique; duplicates are prohibited.
For example, playing card suits might be defined with:
{
"type": "enum",
"name": "Suit",
"symbols" : ["SPADES", "HEARTS", "DIAMONDS", "CLUBS"]
}
Arrays
Arrays use the type name “array” and support a single attribute:
- items: the schema of the array’s items.
For example, an array of strings is declared with:
{
"type": "array",
"items" : "string"
}
Maps
Maps use the type name “map” and support one attribute:
- values: the schema of the map’s values.
Map keys are assumed to be strings.
For example, a map from string to long is declared with:
{
"type": "map",
"values" : "long"
}
Unions
Unions, as mentioned above, are represented using JSON arrays. For example, ["string", "null"] declares a schema which may be either a string or null.
Unions may not contain more than one schema with the same type, except for the named types record, fixed and enum. For example, unions containing two array types or two map types are not permitted, but two types with different names are permitted. (Names permit efficient resolution when reading and writing unions.)
Unions may not immediately contain other unions.
Fixed
Fixed uses the type name “fixed” and supports two attributes:
- name: a string naming this fixed (required).
- namespace, a string that qualifies the name;
- size: an integer, specifying the number of bytes per value (required).
For example, 16-byte quantity may be declared with:
{"type": "fixed", "size": 16, "name": "md5"}
Names
Record, enums and fixed are named types. Each has a fullname that is composed of two parts; a name and a namespace. Equality of names is defined on the fullname.
The name portion of a fullname, and record field names must:
- start with [A-Za-z_]
- subsequently contain only [A-Za-z0-9_]
A namespace is a dot-separated sequence of such names.
In record, enum and fixed definitions, the fullname is determined in one of the following ways:
A name and namespace are both specified. For example, one might use “name”: “X”, “namespace”: “org.foo” to indicate the fullname org.foo.X.
-
A fullname is specified. If the name specified contains a dot, then it is assumed to be a fullname, and any namespace also specified is ignored. For example, use “name”: “org.foo.X” to indicate the fullname org.foo.X.
-
A name only is specified, i.e., a name that contains no dots. In this case the namespace is taken from the most tightly enclosing schema or protocol. For example, if “name”: “X” is specified, and this occurs within a field of the record definition of org.foo.Y, then the fullname is org.foo.X.
References to previously defined names are as in the latter two cases above: if they contain a dot they are a fullname, if they do not contain a dot, the namespace is the namespace of the enclosing definition.
Primitive type names have no namespace and their names may not be defined in any namespace. A schema may only contain multiple definitions of a fullname if the definitions are equivalent.
Data Serialization
Avro data is always serialized with its schema. Files that store Avro data should always also include the schema for that data in the same file. Avro-based remote procedure call (RPC) systems must also guarantee that remote recipients of data have a copy of the schema used to write that data.
Because the schema used to write data is always available when the data is read, Avro data itself is not tagged with type information. The schema is required to parse data.
In general, both serialization and deserialization proceed as a depth-first, left-to-right traversal of the schema, serializing primitive types as they are encountered.
Encodings
Avro specifies two serialization encodings: binary and JSON. Most applications will use the binary encoding, as it is smaller and faster. But, for debugging and web-based applications, the JSON encoding may sometimes be appropriate.
Binary Encoding
Primitive Types
Primitive types are encoded in binary as follows:
- null is written as zero bytes.
- a boolean is written as a single byte whose value is either 0 (false) or 1 (true).
- int and long values are written using variable-length zig-zag coding. Some examples:
| value |
hex |
| 0 |
00 |
| -1 |
01 |
| 1 |
02 |
| -2 |
03 |
| 2 |
04 |
| … |
|
| -64 |
7f |
| 64 |
80 01 |
| … |
|
- a float is written as 4 bytes. The float is converted into a 32-bit integer using a method equivalent to Java’s floatToIntBits and then encoded in little-endian format.
- a double is written as 8 bytes. The double is converted into a 64-bit integer using a method equivalent to Java’s doubleToLongBits and then encoded in little-endian format.
- bytes are encoded as a long followed by that many bytes of data.
- a string is encoded as a long followed by that many bytes of UTF-8 encoded character data.
For example, the three-character string “foo” would be encoded as the long value 3 (encoded as hex 06) followed by the UTF-8 encoding of ‘f’, ‘o’, and ‘o’ (the hex bytes 66 6f 6f):
06 66 6f 6f
Complex Types
Complex types are encoded in binary as follows:
Records
A record is encoded by encoding the values of its fields in the order that they are declared. In other words, a record is encoded as just the concatenation of the encodings of its fields. Field values are encoded per their schema.
For example, the record schema
{
"type": "record",
"name": "test",
"fields" : [
{"name": "a", "type": "long"},
{"name": "b", "type": "string"}
]
}
An instance of this record whose a field has value 27 (encoded as hex 36) and whose b field has value “foo” (encoded as hex bytes 06 66 6f 6f), would be encoded simply as the concatenation of these, namely the hex byte sequence:
36 06 66 6f 6f
Enums
An enum is encoded by a int, representing the zero-based position of the symbol in the schema.
For example, consider the enum:
{"type": "enum", "name": "Foo", "symbols": ["A", "B", "C", "D"] }
This would be encoded by an int between zero and three, with zero indicating “A”, and 3 indicating “D”.
Arrays
Arrays are encoded as a series of blocks. Each block consists of a long count value, followed by that many array items. A block with count zero indicates the end of the array. Each item is encoded per the array’s item schema.
If a block’s count is negative, its absolute value is used, and the count is followed immediately by a long block size indicating the number of bytes in the block. This block size permits fast skipping through data, e.g., when projecting a record to a subset of its fields.
For example, the array schema
{"type": "array", "items": "long"}
an array containing the items 3 and 27 could be encoded as the long value 2 (encoded as hex 04) followed by long values 3 and 27 (encoded as hex 06 36) terminated by zero:
04 06 36 00
The blocked representation permits one to read and write arrays larger than can be buffered in memory, since one can start writing items without knowing the full length of the array.
Maps
Maps are encoded as a series of blocks. Each block consists of a long count value, followed by that many key/value pairs. A block with count zero indicates the end of the map. Each item is encoded per the map’s value schema.
If a block’s count is negative, its absolute value is used, and the count is followed immediately by a long block size indicating the number of bytes in the block. This block size permits fast skipping through data, e.g., when projecting a record to a subset of its fields.
The blocked representation permits one to read and write maps larger than can be buffered in memory, since one can start writing items without knowing the full length of the map.
Unions
A union is encoded by first writing a long value indicating the zero-based position within the union of the schema of its value. The value is then encoded per the indicated schema within the union.
For example, the union schema ["string","null"] would encode:
-
null as the integer 1 (the index of “null” in the union, encoded as hex 02):
02
-
the string “a” as zero (the index of “string” in the union), followed by the serialized string:
02 02 61
Fixed
Fixed instances are encoded using the number of bytes declared in the schema.
JSON Encoding
Except for unions, the JSON encoding is the same as is used to encode field default values.
The value of a union is encoded in JSON as follows:
- if its type is null, then it is encoded as a JSON null;
- otherwise it is encoded as a JSON object with one name/value pair whose name is the type’s name and whose value is the recursively encoded value. For Avro’s named types (record, fixed or enum) the user-specified name is used, for other types the type name is used.
For example, the union schema ["null","string","Foo"], where Foo is a record name, would encode:
- null as null;
- the string “a” as
{"string": "a"} and
- a Foo instance as
{"Foo": {...}}, where {...} indicates the JSON encoding of a Foo instance.
Note that a schema is still required to correctly process JSON-encoded data. For example, the JSON encoding does not distinguish between int and long, float and double, records and maps, enums and strings, etc.
Sort Order
Avro defines a standard sort order for data. This permits data written by one system to be efficiently sorted by another system. This can be an important optimization, as sort order comparisons are sometimes the most frequent per-object operation. Note also that Avro binary-encoded data can be efficiently ordered without deserializing it to objects.
Data items may only be compared if they have identical schemas. Pairwise comparisons are implemented recursively with a depth-first, left-to-right traversal of the schema. The first mismatch encountered determines the order of the items.
Two items with the same schema are compared according to the following rules.
- null data is always equal.
- boolean data is ordered with false before true.
- int, long, float and double data is ordered by ascending numeric value.
- bytes and fixed data are compared lexicographically by unsigned 8-bit values.
- string data is compared lexicographically by Unicode code point. Note that since UTF-8 is used as the binary encoding for strings, sorting of bytes and string binary data is identical.
- array data is compared lexicographically by element.
- enum data is ordered by the symbol’s position in the enum schema. For example, an enum whose symbols are
["z", "a"] would sort “z” values before “a” values.
- union data is first ordered by the branch within the union, and, within that, by the type of the branch. For example, an
["int", "string"] union would order all int values before all string values, with the ints and strings themselves ordered as defined above.
- record data is ordered lexicographically by field. If a field specifies that its order is:
- “ascending”, then the order of its values is unaltered.
- “descending”, then the order of its values is reversed.
- “ignore”, then its values are ignored when sorting.
- map data may not be compared. It is an error to attempt to compare data containing maps unless those maps are in an
"order":"ignore" record field.
Object Container Files
Avro includes a simple object container file format. A file has a schema, and all objects stored in the file must be written according to that schema, using binary encoding. Objects are stored in blocks that may be compressed. Syncronization markers are used between blocks to permit efficient splitting of files for MapReduce processing.
Files may include arbitrary user-specified metadata.
A file consists of:
- A file header, followed by
- one or more file data blocks.
A file header consists of:
- Four bytes, ASCII ‘O’, ‘b’, ‘j’, followed by 1.
- file metadata, including the schema.
- The 16-byte, randomly-generated sync marker for this file.
File metadata consists of:
A long indicating the number of metadata key/value pairs.
For each pair, a string key and bytes value.
All metadata properties that start with “avro.” are reserved. The following file metadata properties are currently used:
- avro.schema contains the schema of objects stored in the file, as JSON data (required).
- avro.codec the name of the compression codec used to compress blocks, as a string. Implementations are required to support the following codecs: “null” and “deflate”. If codec is absent, it is assumed to be “null”. The codecs are described with more detail below.
A file header is thus described by the following schema:
{"type": "record", "name": "org.apache.avro.file.Header",
"fields" : [
{"name": "magic", "type": {"type": "fixed", "name": "Magic", "size": 4}},
{"name": "meta", "type": {"type": "map", "values": "bytes"}},
{"name": "sync", "type": {"type": "fixed", "name": "Sync", "size": 16}}
]
}
A file data block consists of:
- A long indicating the count of objects in this block.
- A long indicating the size in bytes of the serialized objects in the current block, after any codec is applied
- The serialized objects. If a codec is specified, this is compressed by that codec.
- The file’s 16-byte sync marker.
Thus, each block’s binary data can be efficiently extracted or skipped without deserializing the contents. The combination of block size, object counts, and sync markers enable detection of corrupt blocks and help ensure data integrity.
Required Codecs
null
The “null” codec simply passes through data uncompressed.
deflate
The “deflate” codec writes the data block using the deflate algorithm as specified in RFC 1951, and typically implemented using the zlib library. Note that this format (unlike the “zlib format” in RFC 1950) does not have a checksum.
Protocol Declaration
Avro protocols describe RPC interfaces. Like schemas, they are defined with JSON text.
A protocol is a JSON object with the following attributes:
- protocol, a string, the name of the protocol (required);
- namespace, an optional string that qualifies the name;
- doc, an optional string describing this protocol;
- types, an optional list of definitions of named types (records, enums, fixed and errors). An error definition is just like a record definition except it uses “error” instead of “record”. Note that forward references to named types are not permitted.
- messages, an optional JSON object whose keys are message names and whose values are objects whose attributes are described below. No two messages may have the same name.
The name and namespace qualification rules defined for schema objects apply to protocols as well.
Messages
A message has attributes:
- a doc, an optional description of the message,
- a request, a list of named, typed parameter schemas (this has the same form as the fields of a record declaration);
- a response schema; and
- an optional union of error schemas.
A request parameter list is processed equivalently to an anonymous record. Since record field lists may vary between reader and writer, request parameters may also differ between the caller and responder, and such differences are resolved in the same manner as record field differences.
Sample Protocol
For example, one may define a simple HelloWorld protocol with:
{
"namespace": "com.acme",
"protocol": "HelloWorld",
"doc": "Protocol Greetings",
"types": [
{"name": "Greeting", "type": "record", "fields": [
{"name": "message", "type": "string"}]},
{"name": "Curse", "type": "error", "fields": [
{"name": "message", "type": "string"}]}
],
"messages": {
"hello": {
"doc": "Say hello.",
"request": [{"name": "greeting", "type": "Greeting" }],
"response": "Greeting",
"errors": ["Curse"]
}
}
}
Message Transport
Messages may be transmitted via different transport mechanisms.
To the transport, a message is an opaque byte sequence.
A transport is a system that supports:
- transmission of request messages
- receipt of corresponding response messages
Servers will send a response message back to the client corresponding to each request message. The mechanism of that correspondance is transport-specific. For example, in HTTP it might be implicit, since HTTP directly supports requests and responses. But a transport that multiplexes many client threads over a single socket would need to tag messages with unique identifiers.
HTTP as Transport
When HTTP is used as a transport, each Avro message exchange is an HTTP request/response pair. All messages of an Avro protocol should share a single URL at an HTTP server. Other protocols may also use that URL. Both normal and error Avro response messages should use the 200 (OK) response code. The chunked encoding may be used for requests and responses, but, regardless the Avro request and response are the entire content of an HTTP request and response. The HTTP Content-Type of requests and responses should be specified as “avro/binary”. Requests should be made using the POST method.
Message Framing
Avro messages are framed as a list of buffers.
Framing is a layer between messages and the transport. It exists to optimize certain operations.
The format of framed message data is:
- a series of buffers, where each buffer consists of:
- a four-byte, big-endian buffer length, followed by
- that many bytes of buffer data.
- A message is always terminated by a zero-lenghted buffer.
Framing is transparent to request and response message formats (described below). Any message may be presented as a single or multiple buffers.
Framing can permit readers to more efficiently get different buffers from different sources and for writers to more efficiently store different buffers to different destinations. In particular, it can reduce the number of times large binary objects are copied. For example, if an RPC parameter consists of a megabyte of file data, that data can be copied directly to a socket from a file descriptor, and, on the other end, it could be written directly to a file descriptor, never entering user space.
A simple, recommended, framing policy is for writers to create a new segment whenever a single binary object is written that is larger than a normal output buffer. Small objects are then appended in buffers, while larger objects are written as their own buffers. When a reader then tries to read a large object the runtime can hand it an entire buffer directly, without having to copy it.
Handshake
RPC requests and responses are prefixed by handshakes. The purpose of the handshake is to ensure that the client and the server have each other’s protocol definition, so that the client can correctly deserialize responses, and the server can correctly deserialize requests. Both clients and servers should maintain a cache of recently seen protocols, so that, in most cases, a handshake will be completed without extra round-trip network exchanges or the transmission of full protocol text.
The handshake process uses the following record schemas:
{
"type": "record",
"name": "HandshakeRequest", "namespace":"org.apache.avro.ipc",
"fields": [
{"name": "clientHash",
"type": {"type": "fixed", "name": "MD5", "size": 16}},
{"name": "clientProtocol", "type": ["null", "string"]},
{"name": "serverHash", "type": "MD5"},
{"name": "meta", "type": ["null", {"type": "map", "values": "bytes"}]}
]
}
{
"type": "record",
"name": "HandshakeResponse", "namespace": "org.apache.avro.ipc",
"fields": [
{"name": "match",
"type": {"type": "enum", "name": "HandshakeMatch",
"symbols": ["BOTH", "CLIENT", "NONE"]}},
{"name": "serverProtocol",
"type": ["null", "string"]},
{"name": "serverHash",
"type": ["null", {"type": "fixed", "name": "MD5", "size": 16}]},
{"name": "meta",
"type": ["null", {"type": "map", "values": "bytes"}]}
]
}
- A client first prefixes each request with a
HandshakeRequest containing just the hash of its protocol and of the server’s protocol (clientHash!=null, clientProtocol=null, serverHash!=null), where the hashes are 128-bit MD5 hashes of the JSON protocol text. If a client has never connected to a given server, it sends its hash as a guess of the server’s hash, otherwise it sends the hash that it previously obtained from this server.
The server responds with a HandshakeResponse containing one of:
-
match=BOTH, serverProtocol=null, serverHash=null if the client sent the valid hash of the server’s protocol and the server knows what protocol corresponds to the client’s hash. In this case, the request is complete and the response data immediately follows the HandshakeResponse.
-
match=CLIENT, serverProtocol!=null, serverHash!=null if the server has previously seen the client’s protocol, but the client sent an incorrect hash of the server’s protocol. The request is complete and the response data immediately follows the HandshakeResponse. The client must use the returned protocol to process the response and should also cache that protocol and its hash for future interactions with this server.
-
match=NONE if the server has not previously seen the client’s protocol. The serverHash and serverProtocol may also be non-null if the server’s protocol hash was incorrect.
In this case the client must then re-submit its request with its protocol text (clientHash!=null, clientProtocol!=null, serverHash!=null) and the server should respond with a successful match (match=BOTH, serverProtocol=null, serverHash=null) as above.
The meta field is reserved for future handshake enhancements.
A call consists of a request message paired with its resulting response or error message. Requests and responses contain extensible metadata, and both kinds of messages are framed as described above.
The format of a call request is:
- request metadata, a map with values of type bytes
- the message name, an Avro string, followed by
- the message parameters. Parameters are serialized according to the message’s request declaration.
The format of a call response is:
- response metadata, a map with values of type bytes
- a one-byte error flag boolean, followed by either:
- if the error flag is false, the message response, serialized per the message’s response schema.
- if the error flag is true, the error, serialized per the message’s error union schema.
Schema Resolution
A reader of Avro data, whether from an RPC or a file, can always parse that data because its schema is provided. But that schema may not be exactly the schema that was expected. For example, if the data was written with a different version of the software than it is read, then records may have had fields added or removed. This section specifies how such schema differences should be resolved.
We call the schema used to write the data as the writer’s schema, and the schema that the application expects the reader’s schema. Differences between these should be resolved as follows:
- It is an error if the two schemas do not match.
To match, one of the following must hold:
-
both schemas are arrays whose item types match
-
both schemas are maps whose value types match
-
both schemas are enums whose names match
-
both schemas are fixed whose sizes and names match
-
both schemas are records with the same name
-
either schema is a union
-
both schemas have same primitive type
-
the writer’s schema may be promoted to the reader’s as follows:
- int is promotable to long, float, or double
- long is promotable to float or double
- float is promotable to double
- string is promotable to bytes
- bytes is promotable to string
-
if both are records:
- the ordering of fields may be different: fields are matched by name.
- schemas for fields with the same name in both records are resolved recursively.
- if the writer’s record contains a field with a name not present in the reader’s record, the writer’s value for that field is ignored.
- if the reader’s record schema has a field that contains a default value, and writer’s schema does not have a field with the same name, then the reader should use the default value from its field.
- if the reader’s record schema has a field with no default value, and writer’s schema does not have a field with the same name, an error is signalled.
-
if both are enums:
if the writer’s symbol is not present in the reader’s enum, then an error is signalled.
This resolution algorithm is applied recursively to the reader’s and writer’s array item schemas.
This resolution algorithm is applied recursively to the reader’s and writer’s value schemas.
The first schema in the reader’s union that matches the selected writer’s union schema is recursively resolved against it. if none match, an error is signalled.
- if reader’s is a union, but writer’s is not:
The first schema in the reader’s union that matches the writer’s schema is recursively resolved against it. If none match, an error is signalled.
- if writer’s is a union, but reader’s is not:
If the reader’s schema matches the selected writer’s schema, it is recursively resolved against it. If they do not match, an error is signalled.
A schema’s “doc” fields are ignored for the purposes of schema resolution. Hence, the “doc” portion of a schema may be dropped at serialization.
32.2 - GenAvro Language
Introduction
This document defines GenAvro, an experimental higher-level language for authoring Avro schemata. Before reading this document, you should have familiarity with the concepts of schemata and protocols, as well as the various primitive and complex types available in Avro.
N.B. This feature is considered experimental in the current version of Avro and the language has not been finalized. Although major changes are unlikely, some syntax may change in future versions of Avro.
Overview
Purpose
The aim of the GenAvro language is to enable developers to author schemata in a way that feels more similar to common programming languages like Java, C++, or Python. Additionally, the GenAvro language may feel more familiar for those users who have previously used the interface description languages (IDLs) in other frameworks like Thrift, Protocol Buffers, or CORBA.
Usage
Each GenAvro file defines a single Avro Protocol, and thus generates as its output a JSON-format Avro Protocol file with extension .avpr.
To convert a .genavro file into a .avpr file, it must be processed by the avroj genavro tool. For example:
$ java -jar avroj-1.3.0.jar genavro src/test/genavro/input/namespaces.genavro /tmp/namespaces.avpr
$ head /tmp/namespaces.avpr
{
"protocol" : "TestNamespace",
"namespace" : "avro.test.protocol",
The avroj genavro tool can also process input to and from stdin and stdout. See avroj genavro –help for full usage information.
Defining a Protocol in GenAvro
A GenAvro file consists of exactly one protocol definition. The minimal protocol is defined by the following code:
This is equivalent to (and generates) the following JSON protocol definition:
{
"protocol" : "MyProtocol",
"types" : [ ],
"messages" : {
}
}
The namespace of the protocol may be changed using the @namespace annotation:
@namespace("mynamespace")
protocol MyProtocol {
}
3 This notation is used throughout GenAvro as a way of specifying properties for the annotated element, as will be described later in this document.
Protocols in GenAvro can contain the following items:
- Definitions of named schemata, including records, errors, enums, and fixeds.
- Definitions of RPC messages
Defining an Enumeration
Enums are defined in GenAvro using a syntax similar to C or Java:
enum Suit {
SPADES, DIAMONDS, CLUBS, HEARTS
}
Note that, unlike the JSON format, anonymous enums cannot be defined.
Defining a Fixed Length Field
Fixed fields are defined using the following syntax:
fixed MD5(16);
This example defines a fixed-length type called MD5 which contains 16 bytes.
Defining Records and Errors
Records are defined in GenAvro using a syntax similar to a struct definition in C:
record Employee {
string name;
boolean active;
long salary;
}
The above example defines a record with the name “Employee” with three fields.
To define an error, simply use the keyword error instead of record. For example:
error Kaboom {
string explanation;
int result_code;
}
Each field in a record or error consists of a type and a name, along with optional property annotations.
A type reference in GenAvro must be one of:
- A primitive type
- A named schema defined prior to this usage in the same Protocol
- A complex type (array, map, or union)
Primitive Types
The primitive types supported by GenAvro are the same as those supported by Avro’s JSON format. This list includes int, long, string, boolean, float, double, null, and bytes.
References to Named Schemata
If a named schema has already been defined in the same GenAvro file, it may be referenced by name as if it were a primitive type:
record Card {
Suit suit; // refers to the enum Card defined above
int number;
}
Complex Types
Arrays
Array types are written in a manner that will seem familiar to C++ or Java programmers. An array of any type t is denoted array<t>. For example, an array of strings is denoted array<string>, and a multidimensional array of Foo records would be array<array<Foo>>.
Maps
Map types are written similarly to array types. An array that contains values of type t is written map<t>. As in the JSON schema format, all maps contain string-type keys.
Unions
Union types are denoted as union { typeA, typeB, typeC, … }. For example, this record contains a string field that is optional (unioned with null):
record RecordWithUnion {
union { null, string } optionalString;
}
Note that the same restrictions apply to GenAvro unions as apply to unions defined in the JSON format; namely, a record may not contain multiple elements of the same type.
Defining RPC Messages
The syntax to define an RPC message within a GenAvro protocol is similar to the syntax for a method declaration within a C header file or a Java interface. To define an RPC message add which takes two arguments named foo and bar, returning an int, simply include the following definition within the protocol:
int add(int foo, int bar);
To define a message with no response, you may use the alias void, equivalent to the Avro null type:
void logMessage(string message);
If you have previously defined an error type within the same protocol, you may declare that a message can throw this error using the syntax:
void goKaboom() throws Kaboom;
Other Language Features
All Java-style comments are supported within a GenAvro file. Any text following // on a line is ignored, as is any text between /* and */, possibly spanning multiple lines.
Escaping Identifiers
Occasionally, one will need to use a reserved language keyword as an identifier. In order to do so, backticks (`) may be used to escape the identifier. For example, to define a message with the literal name error, you may write:
This syntax is allowed anywhere an identifier is expected.
Annotations for Ordering and Namespaces
Java-style annotations may be used to add additional properties to types throughout GenAvro. For example, to specify the sort order of a field within a record, one may use the @order annotation as follows:
record MyRecord {
@order("ascending")
string myAscendingSortField;
@order("descending")
string myDescendingField;
@order("ignore")
string myIgnoredField;
}
Similarly, a @namespace annotation may be used to modify the namespace when defining a named schema. For example:
@namespace("org.apache.avro.firstNamespace")
protocol MyProto {
@namespace("org.apache.avro.someOtherNamespace")
record Foo {}
record Bar {}
}
will define a protocol in the firstNamespace namespace. The record Foo will be defined in someOtherNamespace and Bar will be defined in firstNamespace as it inherits its default from its container.
Complete Example
The following is a complete example of a GenAvro file that shows most of the above features:
/**
* An example protocol in GenAvro
*/
@namespace("org.apache.avro.test")
protocol Simple {
@aliases(["org.foo.KindOf"])
enum Kind {
FOO,
BAR, // the bar enum value
BAZ
}
fixed MD5(16);
record TestRecord {
@order("ignore")
string name;
@order("descending")
Kind kind;
MD5 hash;
union { MD5, null} nullableHash;
array<long> arrayOfLongs;
}
error TestError {
string message;
}
string hello(string greeting);
TestRecord echo(TestRecord `record`);
int add(int arg1, int arg2);
bytes echoBytes(bytes data);
void `error`() throws TestError;
void ping() oneway;
}
Additional examples may be found in the Avro source tree under the src/test/genavro/input directory.
32.3 - Wiki
The Wiki page can be found here.
32.4 - FAQ
What is Avro?
Avro is a data serialization system.
Avro provides:
- Rich data structures.
- A compact, fast, binary data format.
- A container file, to store persistent data.
- Remote procedure call (RPC).
- Simple integration with dynamic languages. Code generation is not required to read or write data files nor to use or implement RPC protocols. Code generation as an optional optimization, only worth implementing for statically typed languages.
How can I get started quickly with Avro?
Check out the Quick Start Guide.
How do I statically compile a schema or protocol into generated code?
In Java
- Add the avro jar, the jackson-mapper-asl.jar and jackson-core-asl.jar to your CLASSPATH.
- Run java org.apache.avro.specific.SpecificCompiler .
This appears to be out of date, the SpecificCompiler requires two arguments, presumably an input and and output file, but it isn’t clear that this does.
Or use the Schema or Protocol Ant tasks. Avro’s build.xml provides examples of how these are used.
Lastly, you can also use the “avro-tools” jar which ships with an Avro release. Just use the “compile (schema|protocol)” command.
How are Strings represented in Java?
They use org.apache.avro.util.Utf8, not java.lang.String.
More generally, how do Avro types map to Java types?
The mappings are documented in the package javadoc for generic, specific and reflect API.
From Doug Cutting:
HDFS splits files into blocks, and mapreduce runs a map task for each block. When the task starts, it needs to be able to seek into the file to the start of the block process through the block’s end. If the file were, e.g., a gzip file, this would not be possible, since gzip files must be decompressed from the start. One cannot seek into the middle of a gzip file and start decompressing. So Hadoop’s SequenceFile places a marker periodically (~64k) in the file at record and compression boundaries, where processing can be sensibly started. Then, when a map task starts processing an HDFS block, it finds the first marker after the block’s start and continues through the first marker in the next block of the file. This requires a bit of non-local access (~0.1%). Avro’s data file uses the same method as SequenceFile.
Why isn’t every value in Avro nullable?
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 of every value stored and
corresponding computational costs to create this bit on write and
interpret it on read. These costs are wasted when values may not in
fact be null, as is the case in many datasets. In Avro such costs are
only paid when values may actually be null.
Also, allowing values to be null is a well-known source of errors. In
Avro, a value declared as non-null will always be non-null and programs
need not test for null values when processing it nor will they ever fail
for lack of such tests.
Tony Hoare calls his invention of null references his “Billion Dollar
Mistake”.
http://qconlondon.com/london-2009/presentation/Null+References:+The+Billion+Dollar+Mistake
Also note that in some programming languages not all values are permitted to be null. For example, in Java, values of type boolean, byte, short, char, int, float, long, and double may not be null.
How can I serialize directly to/from a byte array?
As pointed out in the specification, Avro data should always be stored with its schema. The Avro provided classes DataFileWriter, DataFileReader, and DataFileStream all ensure this by serializing the Schema in a container header. In some special cases, such as when implementing a new storage system or writing unit tests, you may need to write and read directly with the bare Avro serialized values. The following examples use code generated for Java from an Avro Schema, specifically the User example from the Quickstart guide.
Serializing to a byte array
This example takes a User object and returns a newly allocated byte array with the Avro serialization of that user.
ByteArrayOutputStream out = new ByteArrayOutputStream();
BinaryEncoder encoder = EncoderFactory.get().binaryEncoder(out, null);
DatumWriter<User> writer = new SpecificDatumWriter<User>(User.getClassSchema());
writer.write(user, encoder);
encoder.flush();
out.close();
byte[] serializedBytes = out.toByteArray();
Deserializing from a byte array
This example takes a byte array containing the Avro serialization of a user and returns a User object.SpecificDatumReader reader = new SpecificDatumReader(User.getClassSchema());
Decoder decoder = DecoderFactory.get().binaryDecoder(bytes, null);
User user = reader.read(null, decoder);
Note: If you are serializing or deserializing in a loop or as a method, you should be reusing objects, readers and/or writers. Check the JavaDoc to see how to reuse the objects.
33 - Apache Avro™ 1.3.1 Documentation
Introduction
Avro is a data serialization system.
Avro provides:
- Rich data structures.
- A compact, fast, binary data format.
- A container file, to store persistent data.
- Remote procedure call (RPC).
- Simple integration with dynamic languages. Code generation is not required to read or write data files nor to use or implement RPC protocols. Code generation as an optional optimization, only worth implementing for statically typed languages.
Schemas
Avro relies on schemas. When Avro data is read, the schema used when writing it is always present. This permits each datum to be written with no per-value overheads, making serialization both fast and small. This also facilitates use with dynamic, scripting languages, since data, together with its schema, is fully self-describing.
When Avro data is stored in a file, its schema is stored with it, so that files may be processed later by any program. If the program reading the data expects a different schema this can be easily resolved, since both schemas are present.
When Avro is used in RPC, the client and server exchange schemas in the connection handshake. (This can be optimized so that, for most calls, no schemas are actually transmitted.) Since both client and server both have the other’s full schema, correspondence between same named fields, missing fields, extra fields, etc. can all be easily resolved.
Avro schemas are defined with JSON . This facilitates implementation in languages that already have JSON libraries.
Comparison with other systems
Avro provides functionality similar to systems such as Thrift, Protocol Buffers, etc. Avro differs from these systems in the following fundamental aspects.
- Dynamic typing: Avro does not require that code be generated. Data is always accompanied by a schema that permits full processing of that data without code generation, static datatypes, etc. This facilitates construction of generic data-processing systems and languages.
- Untagged data: Since the schema is present when data is read, considerably less type information need be encoded with data, resulting in smaller serialization size.
- No manually-assigned field IDs: When a schema changes, both the old and new schema are always present when processing data, so differences may be resolved symbolically, using field names.
33.1 - Specification
Introduction
This document defines Avro. It is intended to be the authoritative specification. Implementations of Avro must adhere to this document.
Schema Declaration
A Schema is represented in JSON by one of:
- A JSON string, naming a defined type.
- A JSON object, of the form:
{"type": "typeName" ...attributes...}
where typeName is either a primitive or derived type name, as defined below. Attributes not defined in this document are permitted as metadata, but must not affect the format of serialized data.
- A JSON array, representing a union of embedded types.
Primitive Types
The set of primitive type names is:
- null: no value
- boolean: a binary value
- int: 32-bit signed integer
- long: 64-bit signed integer
- float: single precision (32-bit) IEEE 754 floating-point number
- double: double precision (64-bit) IEEE 754 floating-point number
- bytes: sequence of 8-bit unsigned bytes
- string: unicode character sequence
Primitive types have no specified attributes.
Primitive type names are also defined type names. Thus, for example, the schema “string” is equivalent to:
Complex Types
Avro supports six kinds of complex types: records, enums, arrays, maps, unions and fixed.
Records
Records use the type name “record” and support three attributes:
- name: a JSON string providing the name of the record (required).
- namespace, a JSON string that qualifies the name;
- doc: a JSON string providing documentation to the user of this schema (optional).
- fields: a JSON array, listing fields (required). Each field is a JSON object with the following attributes:
- name: a JSON string providing the name of the field (required), and
- doc: a JSON string describing this field for users (optional).
- type: A JSON object defining a schema, or a JSON string naming a record definition (required).
- default: A default value for this field, used when reading instances that lack this field (optional). 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. Default values for bytes and fixed fields are JSON strings, where Unicode code points 0-255 are mapped to unsigned 8-bit byte values 0-255.
field default values
| avro type |
json type |
example |
| null |
null |
null |
| boolean |
boolean |
true |
| int,long |
integer |
1 |
| float,double |
number |
1.1 |
| bytes |
string |
"\u00FF" |
| string |
string |
"foo" |
| record |
object |
{"a": 1} |
| enum |
string |
"FOO" |
| array |
array |
[1] |
| map |
object |
{"a": 1} |
| fixed |
string |
"\u00ff" |
- order: specifies how this field impacts sort ordering of this record (optional). Valid values are “ascending” (the default), “descending”, or “ignore”. For more details on how this is used, see the sort order section below.
For example, a linked-list of 64-bit values may be defined with:
{
"type": "record",
"name": "LongList",
"fields" : [
{"name": "value", "type": "long"}, // each element has a long
{"name": "next", "type": ["LongList", "null"]} // optional next element
]
}
Enums
Enums use the type name “enum” and support the following attributes:
- name: a JSON string providing the name of the enum (required).
- namespace, a JSON string that qualifies the name ;
- doc: a JSON string providing documentation to the user of this schema (optional).
- symbols: a JSON array, listing symbols, as JSON strings (required). All symbols in an enum must be unique; duplicates are prohibited.
For example, playing card suits might be defined with:
{
"type": "enum",
"name": "Suit",
"symbols" : ["SPADES", "HEARTS", "DIAMONDS", "CLUBS"]
}
Arrays
Arrays use the type name “array” and support a single attribute:
- items: the schema of the array’s items.
For example, an array of strings is declared with:
{
"type": "array",
"items" : "string"
}
Maps
Maps use the type name “map” and support one attribute:
- values: the schema of the map’s values.
Map keys are assumed to be strings.
For example, a map from string to long is declared with:
{
"type": "map",
"values" : "long"
}
Unions
Unions, as mentioned above, are represented using JSON arrays. For example, ["string", "null"] declares a schema which may be either a string or null.
Unions may not contain more than one schema with the same type, except for the named types record, fixed and enum. For example, unions containing two array types or two map types are not permitted, but two types with different names are permitted. (Names permit efficient resolution when reading and writing unions.)
Unions may not immediately contain other unions.
Fixed
Fixed uses the type name “fixed” and supports two attributes:
- name: a string naming this fixed (required).
- namespace, a string that qualifies the name;
- size: an integer, specifying the number of bytes per value (required).
For example, 16-byte quantity may be declared with:
{"type": "fixed", "size": 16, "name": "md5"}
Names
Record, enums and fixed are named types. Each has a fullname that is composed of two parts; a name and a namespace. Equality of names is defined on the fullname.
The name portion of a fullname, and record field names must:
- start with [A-Za-z_]
- subsequently contain only [A-Za-z0-9_]
A namespace is a dot-separated sequence of such names.
In record, enum and fixed definitions, the fullname is determined in one of the following ways:
A name and namespace are both specified. For example, one might use “name”: “X”, “namespace”: “org.foo” to indicate the fullname org.foo.X.
-
A fullname is specified. If the name specified contains a dot, then it is assumed to be a fullname, and any namespace also specified is ignored. For example, use “name”: “org.foo.X” to indicate the fullname org.foo.X.
-
A name only is specified, i.e., a name that contains no dots. In this case the namespace is taken from the most tightly enclosing schema or protocol. For example, if “name”: “X” is specified, and this occurs within a field of the record definition of org.foo.Y, then the fullname is org.foo.X.
References to previously defined names are as in the latter two cases above: if they contain a dot they are a fullname, if they do not contain a dot, the namespace is the namespace of the enclosing definition.
Primitive type names have no namespace and their names may not be defined in any namespace. A schema may only contain multiple definitions of a fullname if the definitions are equivalent.
Data Serialization
Avro data is always serialized with its schema. Files that store Avro data should always also include the schema for that data in the same file. Avro-based remote procedure call (RPC) systems must also guarantee that remote recipients of data have a copy of the schema used to write that data.
Because the schema used to write data is always available when the data is read, Avro data itself is not tagged with type information. The schema is required to parse data.
In general, both serialization and deserialization proceed as a depth-first, left-to-right traversal of the schema, serializing primitive types as they are encountered.
Encodings
Avro specifies two serialization encodings: binary and JSON. Most applications will use the binary encoding, as it is smaller and faster. But, for debugging and web-based applications, the JSON encoding may sometimes be appropriate.
Binary Encoding
Primitive Types
Primitive types are encoded in binary as follows:
- null is written as zero bytes.
- a boolean is written as a single byte whose value is either 0 (false) or 1 (true).
- int and long values are written using variable-length zig-zag coding. Some examples:
| value |
hex |
| 0 |
00 |
| -1 |
01 |
| 1 |
02 |
| -2 |
03 |
| 2 |
04 |
| … |
|
| -64 |
7f |
| 64 |
80 01 |
| … |
|
- a float is written as 4 bytes. The float is converted into a 32-bit integer using a method equivalent to Java’s floatToIntBits and then encoded in little-endian format.
- a double is written as 8 bytes. The double is converted into a 64-bit integer using a method equivalent to Java’s doubleToLongBits and then encoded in little-endian format.
- bytes are encoded as a long followed by that many bytes of data.
- a string is encoded as a long followed by that many bytes of UTF-8 encoded character data.
For example, the three-character string “foo” would be encoded as the long value 3 (encoded as hex 06) followed by the UTF-8 encoding of ‘f’, ‘o’, and ‘o’ (the hex bytes 66 6f 6f):
06 66 6f 6f
Complex Types
Complex types are encoded in binary as follows:
Records
A record is encoded by encoding the values of its fields in the order that they are declared. In other words, a record is encoded as just the concatenation of the encodings of its fields. Field values are encoded per their schema.
For example, the record schema
{
"type": "record",
"name": "test",
"fields" : [
{"name": "a", "type": "long"},
{"name": "b", "type": "string"}
]
}
An instance of this record whose a field has value 27 (encoded as hex 36) and whose b field has value “foo” (encoded as hex bytes 06 66 6f 6f), would be encoded simply as the concatenation of these, namely the hex byte sequence:
36 06 66 6f 6f
Enums
An enum is encoded by a int, representing the zero-based position of the symbol in the schema.
For example, consider the enum:
{"type": "enum", "name": "Foo", "symbols": ["A", "B", "C", "D"] }
This would be encoded by an int between zero and three, with zero indicating “A”, and 3 indicating “D”.
Arrays
Arrays are encoded as a series of blocks. Each block consists of a long count value, followed by that many array items. A block with count zero indicates the end of the array. Each item is encoded per the array’s item schema.
If a block’s count is negative, its absolute value is used, and the count is followed immediately by a long block size indicating the number of bytes in the block. This block size permits fast skipping through data, e.g., when projecting a record to a subset of its fields.
For example, the array schema
{"type": "array", "items": "long"}
an array containing the items 3 and 27 could be encoded as the long value 2 (encoded as hex 04) followed by long values 3 and 27 (encoded as hex 06 36) terminated by zero:
04 06 36 00
The blocked representation permits one to read and write arrays larger than can be buffered in memory, since one can start writing items without knowing the full length of the array.
Maps
Maps are encoded as a series of blocks. Each block consists of a long count value, followed by that many key/value pairs. A block with count zero indicates the end of the map. Each item is encoded per the map’s value schema.
If a block’s count is negative, its absolute value is used, and the count is followed immediately by a long block size indicating the number of bytes in the block. This block size permits fast skipping through data, e.g., when projecting a record to a subset of its fields.
The blocked representation permits one to read and write maps larger than can be buffered in memory, since one can start writing items without knowing the full length of the map.
Unions
A union is encoded by first writing a long value indicating the zero-based position within the union of the schema of its value. The value is then encoded per the indicated schema within the union.
For example, the union schema ["string","null"] would encode:
-
null as the integer 1 (the index of “null” in the union, encoded as hex 02):
02
-
the string “a” as zero (the index of “string” in the union), followed by the serialized string:
02 02 61
Fixed
Fixed instances are encoded using the number of bytes declared in the schema.
JSON Encoding
Except for unions, the JSON encoding is the same as is used to encode field default values.
The value of a union is encoded in JSON as follows:
- if its type is null, then it is encoded as a JSON null;
- otherwise it is encoded as a JSON object with one name/value pair whose name is the type’s name and whose value is the recursively encoded value. For Avro’s named types (record, fixed or enum) the user-specified name is used, for other types the type name is used.
For example, the union schema ["null","string","Foo"], where Foo is a record name, would encode:
- null as null;
- the string “a” as
{"string": "a"} and
- a Foo instance as
{"Foo": {...}}, where {...} indicates the JSON encoding of a Foo instance.
Note that a schema is still required to correctly process JSON-encoded data. For example, the JSON encoding does not distinguish between int and long, float and double, records and maps, enums and strings, etc.
Sort Order
Avro defines a standard sort order for data. This permits data written by one system to be efficiently sorted by another system. This can be an important optimization, as sort order comparisons are sometimes the most frequent per-object operation. Note also that Avro binary-encoded data can be efficiently ordered without deserializing it to objects.
Data items may only be compared if they have identical schemas. Pairwise comparisons are implemented recursively with a depth-first, left-to-right traversal of the schema. The first mismatch encountered determines the order of the items.
Two items with the same schema are compared according to the following rules.
- null data is always equal.
- boolean data is ordered with false before true.
- int, long, float and double data is ordered by ascending numeric value.
- bytes and fixed data are compared lexicographically by unsigned 8-bit values.
- string data is compared lexicographically by Unicode code point. Note that since UTF-8 is used as the binary encoding for strings, sorting of bytes and string binary data is identical.
- array data is compared lexicographically by element.
- enum data is ordered by the symbol’s position in the enum schema. For example, an enum whose symbols are
["z", "a"] would sort “z” values before “a” values.
- union data is first ordered by the branch within the union, and, within that, by the type of the branch. For example, an
["int", "string"] union would order all int values before all string values, with the ints and strings themselves ordered as defined above.
- record data is ordered lexicographically by field. If a field specifies that its order is:
- “ascending”, then the order of its values is unaltered.
- “descending”, then the order of its values is reversed.
- “ignore”, then its values are ignored when sorting.
- map data may not be compared. It is an error to attempt to compare data containing maps unless those maps are in an
"order":"ignore" record field.
Object Container Files
Avro includes a simple object container file format. A file has a schema, and all objects stored in the file must be written according to that schema, using binary encoding. Objects are stored in blocks that may be compressed. Syncronization markers are used between blocks to permit efficient splitting of files for MapReduce processing.
Files may include arbitrary user-specified metadata.
A file consists of:
- A file header, followed by
- one or more file data blocks.
A file header consists of:
- Four bytes, ASCII ‘O’, ‘b’, ‘j’, followed by 1.
- file metadata, including the schema.
- The 16-byte, randomly-generated sync marker for this file.
File metadata consists of:
A long indicating the number of metadata key/value pairs.
For each pair, a string key and bytes value.
All metadata properties that start with “avro.” are reserved. The following file metadata properties are currently used:
- avro.schema contains the schema of objects stored in the file, as JSON data (required).
- avro.codec the name of the compression codec used to compress blocks, as a string. Implementations are required to support the following codecs: “null” and “deflate”. If codec is absent, it is assumed to be “null”. The codecs are described with more detail below.
A file header is thus described by the following schema:
{"type": "record", "name": "org.apache.avro.file.Header",
"fields" : [
{"name": "magic", "type": {"type": "fixed", "name": "Magic", "size": 4}},
{"name": "meta", "type": {"type": "map", "values": "bytes"}},
{"name": "sync", "type": {"type": "fixed", "name": "Sync", "size": 16}}
]
}
A file data block consists of:
- A long indicating the count of objects in this block.
- A long indicating the size in bytes of the serialized objects in the current block, after any codec is applied
- The serialized objects. If a codec is specified, this is compressed by that codec.
- The file’s 16-byte sync marker.
Thus, each block’s binary data can be efficiently extracted or skipped without deserializing the contents. The combination of block size, object counts, and sync markers enable detection of corrupt blocks and help ensure data integrity.
Required Codecs
null
The “null” codec simply passes through data uncompressed.
deflate
The “deflate” codec writes the data block using the deflate algorithm as specified in RFC 1951, and typically implemented using the zlib library. Note that this format (unlike the “zlib format” in RFC 1950) does not have a checksum.
Protocol Declaration
Avro protocols describe RPC interfaces. Like schemas, they are defined with JSON text.
A protocol is a JSON object with the following attributes:
- protocol, a string, the name of the protocol (required);
- namespace, an optional string that qualifies the name;
- doc, an optional string describing this protocol;
- types, an optional list of definitions of named types (records, enums, fixed and errors). An error definition is just like a record definition except it uses “error” instead of “record”. Note that forward references to named types are not permitted.
- messages, an optional JSON object whose keys are message names and whose values are objects whose attributes are described below. No two messages may have the same name.
The name and namespace qualification rules defined for schema objects apply to protocols as well.
Messages
A message has attributes:
- a doc, an optional description of the message,
- a request, a list of named, typed parameter schemas (this has the same form as the fields of a record declaration);
- a response schema; and
- an optional union of error schemas.
A request parameter list is processed equivalently to an anonymous record. Since record field lists may vary between reader and writer, request parameters may also differ between the caller and responder, and such differences are resolved in the same manner as record field differences.
Sample Protocol
For example, one may define a simple HelloWorld protocol with:
{
"namespace": "com.acme",
"protocol": "HelloWorld",
"doc": "Protocol Greetings",
"types": [
{"name": "Greeting", "type": "record", "fields": [
{"name": "message", "type": "string"}]},
{"name": "Curse", "type": "error", "fields": [
{"name": "message", "type": "string"}]}
],
"messages": {
"hello": {
"doc": "Say hello.",
"request": [{"name": "greeting", "type": "Greeting" }],
"response": "Greeting",
"errors": ["Curse"]
}
}
}
Message Transport
Messages may be transmitted via different transport mechanisms.
To the transport, a message is an opaque byte sequence.
A transport is a system that supports:
- transmission of request messages
- receipt of corresponding response messages
Servers will send a response message back to the client corresponding to each request message. The mechanism of that correspondance is transport-specific. For example, in HTTP it might be implicit, since HTTP directly supports requests and responses. But a transport that multiplexes many client threads over a single socket would need to tag messages with unique identifiers.
HTTP as Transport
When HTTP is used as a transport, each Avro message exchange is an HTTP request/response pair. All messages of an Avro protocol should share a single URL at an HTTP server. Other protocols may also use that URL. Both normal and error Avro response messages should use the 200 (OK) response code. The chunked encoding may be used for requests and responses, but, regardless the Avro request and response are the entire content of an HTTP request and response. The HTTP Content-Type of requests and responses should be specified as “avro/binary”. Requests should be made using the POST method.
Message Framing
Avro messages are framed as a list of buffers.
Framing is a layer between messages and the transport. It exists to optimize certain operations.
The format of framed message data is:
- a series of buffers, where each buffer consists of:
- a four-byte, big-endian buffer length, followed by
- that many bytes of buffer data.
- A message is always terminated by a zero-lenghted buffer.
Framing is transparent to request and response message formats (described below). Any message may be presented as a single or multiple buffers.
Framing can permit readers to more efficiently get different buffers from different sources and for writers to more efficiently store different buffers to different destinations. In particular, it can reduce the number of times large binary objects are copied. For example, if an RPC parameter consists of a megabyte of file data, that data can be copied directly to a socket from a file descriptor, and, on the other end, it could be written directly to a file descriptor, never entering user space.
A simple, recommended, framing policy is for writers to create a new segment whenever a single binary object is written that is larger than a normal output buffer. Small objects are then appended in buffers, while larger objects are written as their own buffers. When a reader then tries to read a large object the runtime can hand it an entire buffer directly, without having to copy it.
Handshake
RPC requests and responses are prefixed by handshakes. The purpose of the handshake is to ensure that the client and the server have each other’s protocol definition, so that the client can correctly deserialize responses, and the server can correctly deserialize requests. Both clients and servers should maintain a cache of recently seen protocols, so that, in most cases, a handshake will be completed without extra round-trip network exchanges or the transmission of full protocol text.
The handshake process uses the following record schemas:
{
"type": "record",
"name": "HandshakeRequest", "namespace":"org.apache.avro.ipc",
"fields": [
{"name": "clientHash",
"type": {"type": "fixed", "name": "MD5", "size": 16}},
{"name": "clientProtocol", "type": ["null", "string"]},
{"name": "serverHash", "type": "MD5"},
{"name": "meta", "type": ["null", {"type": "map", "values": "bytes"}]}
]
}
{
"type": "record",
"name": "HandshakeResponse", "namespace": "org.apache.avro.ipc",
"fields": [
{"name": "match",
"type": {"type": "enum", "name": "HandshakeMatch",
"symbols": ["BOTH", "CLIENT", "NONE"]}},
{"name": "serverProtocol",
"type": ["null", "string"]},
{"name": "serverHash",
"type": ["null", {"type": "fixed", "name": "MD5", "size": 16}]},
{"name": "meta",
"type": ["null", {"type": "map", "values": "bytes"}]}
]
}
- A client first prefixes each request with a
HandshakeRequest containing just the hash of its protocol and of the server’s protocol (clientHash!=null, clientProtocol=null, serverHash!=null), where the hashes are 128-bit MD5 hashes of the JSON protocol text. If a client has never connected to a given server, it sends its hash as a guess of the server’s hash, otherwise it sends the hash that it previously obtained from this server.
The server responds with a HandshakeResponse containing one of:
-
match=BOTH, serverProtocol=null, serverHash=null if the client sent the valid hash of the server’s protocol and the server knows what protocol corresponds to the client’s hash. In this case, the request is complete and the response data immediately follows the HandshakeResponse.
-
match=CLIENT, serverProtocol!=null, serverHash!=null if the server has previously seen the client’s protocol, but the client sent an incorrect hash of the server’s protocol. The request is complete and the response data immediately follows the HandshakeResponse. The client must use the returned protocol to process the response and should also cache that protocol and its hash for future interactions with this server.
-
match=NONE if the server has not previously seen the client’s protocol. The serverHash and serverProtocol may also be non-null if the server’s protocol hash was incorrect.
In this case the client must then re-submit its request with its protocol text (clientHash!=null, clientProtocol!=null, serverHash!=null) and the server should respond with a successful match (match=BOTH, serverProtocol=null, serverHash=null) as above.
The meta field is reserved for future handshake enhancements.
A call consists of a request message paired with its resulting response or error message. Requests and responses contain extensible metadata, and both kinds of messages are framed as described above.
The format of a call request is:
- request metadata, a map with values of type bytes
- the message name, an Avro string, followed by
- the message parameters. Parameters are serialized according to the message’s request declaration.
The format of a call response is:
- response metadata, a map with values of type bytes
- a one-byte error flag boolean, followed by either:
- if the error flag is false, the message response, serialized per the message’s response schema.
- if the error flag is true, the error, serialized per the message’s error union schema.
Schema Resolution
A reader of Avro data, whether from an RPC or a file, can always parse that data because its schema is provided. But that schema may not be exactly the schema that was expected. For example, if the data was written with a different version of the software than it is read, then records may have had fields added or removed. This section specifies how such schema differences should be resolved.
We call the schema used to write the data as the writer’s schema, and the schema that the application expects the reader’s schema. Differences between these should be resolved as follows:
- It is an error if the two schemas do not match.
To match, one of the following must hold:
-
both schemas are arrays whose item types match
-
both schemas are maps whose value types match
-
both schemas are enums whose names match
-
both schemas are fixed whose sizes and names match
-
both schemas are records with the same name
-
either schema is a union
-
both schemas have same primitive type
-
the writer’s schema may be promoted to the reader’s as follows:
- int is promotable to long, float, or double
- long is promotable to float or double
- float is promotable to double
- string is promotable to bytes
- bytes is promotable to string
-
if both are records:
- the ordering of fields may be different: fields are matched by name.
- schemas for fields with the same name in both records are resolved recursively.
- if the writer’s record contains a field with a name not present in the reader’s record, the writer’s value for that field is ignored.
- if the reader’s record schema has a field that contains a default value, and writer’s schema does not have a field with the same name, then the reader should use the default value from its field.
- if the reader’s record schema has a field with no default value, and writer’s schema does not have a field with the same name, an error is signalled.
-
if both are enums:
if the writer’s symbol is not present in the reader’s enum, then an error is signalled.
This resolution algorithm is applied recursively to the reader’s and writer’s array item schemas.
This resolution algorithm is applied recursively to the reader’s and writer’s value schemas.
The first schema in the reader’s union that matches the selected writer’s union schema is recursively resolved against it. if none match, an error is signalled.
- if reader’s is a union, but writer’s is not:
The first schema in the reader’s union that matches the writer’s schema is recursively resolved against it. If none match, an error is signalled.
- if writer’s is a union, but reader’s is not:
If the reader’s schema matches the selected writer’s schema, it is recursively resolved against it. If they do not match, an error is signalled.
A schema’s “doc” fields are ignored for the purposes of schema resolution. Hence, the “doc” portion of a schema may be dropped at serialization.
33.2 - GenAvro Language
Introduction
This document defines GenAvro, an experimental higher-level language for authoring Avro schemata. Before reading this document, you should have familiarity with the concepts of schemata and protocols, as well as the various primitive and complex types available in Avro.
N.B. This feature is considered experimental in the current version of Avro and the language has not been finalized. Although major changes are unlikely, some syntax may change in future versions of Avro.
Overview
Purpose
The aim of the GenAvro language is to enable developers to author schemata in a way that feels more similar to common programming languages like Java, C++, or Python. Additionally, the GenAvro language may feel more familiar for those users who have previously used the interface description languages (IDLs) in other frameworks like Thrift, Protocol Buffers, or CORBA.
Usage
Each GenAvro file defines a single Avro Protocol, and thus generates as its output a JSON-format Avro Protocol file with extension .avpr.
To convert a .genavro file into a .avpr file, it must be processed by the avroj genavro tool. For example:
$ java -jar avroj-1.3.0.jar genavro src/test/genavro/input/namespaces.genavro /tmp/namespaces.avpr
$ head /tmp/namespaces.avpr
{
"protocol" : "TestNamespace",
"namespace" : "avro.test.protocol",
The avroj genavro tool can also process input to and from stdin and stdout. See avroj genavro –help for full usage information.
Defining a Protocol in GenAvro
A GenAvro file consists of exactly one protocol definition. The minimal protocol is defined by the following code:
This is equivalent to (and generates) the following JSON protocol definition:
{
"protocol" : "MyProtocol",
"types" : [ ],
"messages" : {
}
}
The namespace of the protocol may be changed using the @namespace annotation:
@namespace("mynamespace")
protocol MyProtocol {
}
3 This notation is used throughout GenAvro as a way of specifying properties for the annotated element, as will be described later in this document.
Protocols in GenAvro can contain the following items:
- Definitions of named schemata, including records, errors, enums, and fixeds.
- Definitions of RPC messages
Defining an Enumeration
Enums are defined in GenAvro using a syntax similar to C or Java:
enum Suit {
SPADES, DIAMONDS, CLUBS, HEARTS
}
Note that, unlike the JSON format, anonymous enums cannot be defined.
Defining a Fixed Length Field
Fixed fields are defined using the following syntax:
fixed MD5(16);
This example defines a fixed-length type called MD5 which contains 16 bytes.
Defining Records and Errors
Records are defined in GenAvro using a syntax similar to a struct definition in C:
record Employee {
string name;
boolean active;
long salary;
}
The above example defines a record with the name “Employee” with three fields.
To define an error, simply use the keyword error instead of record. For example:
error Kaboom {
string explanation;
int result_code;
}
Each field in a record or error consists of a type and a name, along with optional property annotations.
A type reference in GenAvro must be one of:
- A primitive type
- A named schema defined prior to this usage in the same Protocol
- A complex type (array, map, or union)
Primitive Types
The primitive types supported by GenAvro are the same as those supported by Avro’s JSON format. This list includes int, long, string, boolean, float, double, null, and bytes.
References to Named Schemata
If a named schema has already been defined in the same GenAvro file, it may be referenced by name as if it were a primitive type:
record Card {
Suit suit; // refers to the enum Card defined above
int number;
}
Complex Types
Arrays
Array types are written in a manner that will seem familiar to C++ or Java programmers. An array of any type t is denoted array<t>. For example, an array of strings is denoted array<string>, and a multidimensional array of Foo records would be array<array<Foo>>.
Maps
Map types are written similarly to array types. An array that contains values of type t is written map<t>. As in the JSON schema format, all maps contain string-type keys.
Unions
Union types are denoted as union { typeA, typeB, typeC, … }. For example, this record contains a string field that is optional (unioned with null):
record RecordWithUnion {
union { null, string } optionalString;
}
Note that the same restrictions apply to GenAvro unions as apply to unions defined in the JSON format; namely, a record may not contain multiple elements of the same type.
Defining RPC Messages
The syntax to define an RPC message within a GenAvro protocol is similar to the syntax for a method declaration within a C header file or a Java interface. To define an RPC message add which takes two arguments named foo and bar, returning an int, simply include the following definition within the protocol:
int add(int foo, int bar);
To define a message with no response, you may use the alias void, equivalent to the Avro null type:
void logMessage(string message);
If you have previously defined an error type within the same protocol, you may declare that a message can throw this error using the syntax:
void goKaboom() throws Kaboom;
Other Language Features
All Java-style comments are supported within a GenAvro file. Any text following // on a line is ignored, as is any text between /* and */, possibly spanning multiple lines.
Escaping Identifiers
Occasionally, one will need to use a reserved language keyword as an identifier. In order to do so, backticks (`) may be used to escape the identifier. For example, to define a message with the literal name error, you may write:
This syntax is allowed anywhere an identifier is expected.
Annotations for Ordering and Namespaces
Java-style annotations may be used to add additional properties to types throughout GenAvro. For example, to specify the sort order of a field within a record, one may use the @order annotation as follows:
record MyRecord {
@order("ascending")
string myAscendingSortField;
@order("descending")
string myDescendingField;
@order("ignore")
string myIgnoredField;
}
Similarly, a @namespace annotation may be used to modify the namespace when defining a named schema. For example:
@namespace("org.apache.avro.firstNamespace")
protocol MyProto {
@namespace("org.apache.avro.someOtherNamespace")
record Foo {}
record Bar {}
}
will define a protocol in the firstNamespace namespace. The record Foo will be defined in someOtherNamespace and Bar will be defined in firstNamespace as it inherits its default from its container.
Complete Example
The following is a complete example of a GenAvro file that shows most of the above features:
/**
* An example protocol in GenAvro
*/
@namespace("org.apache.avro.test")
protocol Simple {
@aliases(["org.foo.KindOf"])
enum Kind {
FOO,
BAR, // the bar enum value
BAZ
}
fixed MD5(16);
record TestRecord {
@order("ignore")
string name;
@order("descending")
Kind kind;
MD5 hash;
union { MD5, null} nullableHash;
array<long> arrayOfLongs;
}
error TestError {
string message;
}
string hello(string greeting);
TestRecord echo(TestRecord `record`);
int add(int arg1, int arg2);
bytes echoBytes(bytes data);
void `error`() throws TestError;
void ping() oneway;
}
Additional examples may be found in the Avro source tree under the src/test/genavro/input directory.
33.3 - Wiki
The Wiki page can be found here.
33.4 - FAQ
What is Avro?
Avro is a data serialization system.
Avro provides:
- Rich data structures.
- A compact, fast, binary data format.
- A container file, to store persistent data.
- Remote procedure call (RPC).
- Simple integration with dynamic languages. Code generation is not required to read or write data files nor to use or implement RPC protocols. Code generation as an optional optimization, only worth implementing for statically typed languages.
How can I get started quickly with Avro?
Check out the Quick Start Guide.
How do I statically compile a schema or protocol into generated code?
In Java
- Add the avro jar, the jackson-mapper-asl.jar and jackson-core-asl.jar to your CLASSPATH.
- Run java org.apache.avro.specific.SpecificCompiler .
This appears to be out of date, the SpecificCompiler requires two arguments, presumably an input and and output file, but it isn’t clear that this does.
Or use the Schema or Protocol Ant tasks. Avro’s build.xml provides examples of how these are used.
Lastly, you can also use the “avro-tools” jar which ships with an Avro release. Just use the “compile (schema|protocol)” command.
How are Strings represented in Java?
They use org.apache.avro.util.Utf8, not java.lang.String.
More generally, how do Avro types map to Java types?
The mappings are documented in the package javadoc for generic, specific and reflect API.
From Doug Cutting:
HDFS splits files into blocks, and mapreduce runs a map task for each block. When the task starts, it needs to be able to seek into the file to the start of the block process through the block’s end. If the file were, e.g., a gzip file, this would not be possible, since gzip files must be decompressed from the start. One cannot seek into the middle of a gzip file and start decompressing. So Hadoop’s SequenceFile places a marker periodically (~64k) in the file at record and compression boundaries, where processing can be sensibly started. Then, when a map task starts processing an HDFS block, it finds the first marker after the block’s start and continues through the first marker in the next block of the file. This requires a bit of non-local access (~0.1%). Avro’s data file uses the same method as SequenceFile.
Why isn’t every value in Avro nullable?
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 of every value stored and
corresponding computational costs to create this bit on write and
interpret it on read. These costs are wasted when values may not in
fact be null, as is the case in many datasets. In Avro such costs are
only paid when values may actually be null.
Also, allowing values to be null is a well-known source of errors. In
Avro, a value declared as non-null will always be non-null and programs
need not test for null values when processing it nor will they ever fail
for lack of such tests.
Tony Hoare calls his invention of null references his “Billion Dollar
Mistake”.
http://qconlondon.com/london-2009/presentation/Null+References:+The+Billion+Dollar+Mistake
Also note that in some programming languages not all values are permitted to be null. For example, in Java, values of type boolean, byte, short, char, int, float, long, and double may not be null.
How can I serialize directly to/from a byte array?
As pointed out in the specification, Avro data should always be stored with its schema. The Avro provided classes DataFileWriter, DataFileReader, and DataFileStream all ensure this by serializing the Schema in a container header. In some special cases, such as when implementing a new storage system or writing unit tests, you may need to write and read directly with the bare Avro serialized values. The following examples use code generated for Java from an Avro Schema, specifically the User example from the Quickstart guide.
Serializing to a byte array
This example takes a User object and returns a newly allocated byte array with the Avro serialization of that user.
ByteArrayOutputStream out = new ByteArrayOutputStream();
BinaryEncoder encoder = EncoderFactory.get().binaryEncoder(out, null);
DatumWriter<User> writer = new SpecificDatumWriter<User>(User.getClassSchema());
writer.write(user, encoder);
encoder.flush();
out.close();
byte[] serializedBytes = out.toByteArray();
Deserializing from a byte array
This example takes a byte array containing the Avro serialization of a user and returns a User object.SpecificDatumReader reader = new SpecificDatumReader(User.getClassSchema());
Decoder decoder = DecoderFactory.get().binaryDecoder(bytes, null);
User user = reader.read(null, decoder);
Note: If you are serializing or deserializing in a loop or as a method, you should be reusing objects, readers and/or writers. Check the JavaDoc to see how to reuse the objects.
34 - Apache Avro™ 1.3.0 Documentation
Introduction
Avro is a data serialization system.
Avro provides:
- Rich data structures.
- A compact, fast, binary data format.
- A container file, to store persistent data.
- Remote procedure call (RPC).
- Simple integration with dynamic languages. Code generation is not required to read or write data files nor to use or implement RPC protocols. Code generation as an optional optimization, only worth implementing for statically typed languages.
Schemas
Avro relies on schemas. When Avro data is read, the schema used when writing it is always present. This permits each datum to be written with no per-value overheads, making serialization both fast and small. This also facilitates use with dynamic, scripting languages, since data, together with its schema, is fully self-describing.
When Avro data is stored in a file, its schema is stored with it, so that files may be processed later by any program. If the program reading the data expects a different schema this can be easily resolved, since both schemas are present.
When Avro is used in RPC, the client and server exchange schemas in the connection handshake. (This can be optimized so that, for most calls, no schemas are actually transmitted.) Since both client and server both have the other’s full schema, correspondence between same named fields, missing fields, extra fields, etc. can all be easily resolved.
Avro schemas are defined with JSON . This facilitates implementation in languages that already have JSON libraries.
Comparison with other systems
Avro provides functionality similar to systems such as Thrift, Protocol Buffers, etc. Avro differs from these systems in the following fundamental aspects.
- Dynamic typing: Avro does not require that code be generated. Data is always accompanied by a schema that permits full processing of that data without code generation, static datatypes, etc. This facilitates construction of generic data-processing systems and languages.
- Untagged data: Since the schema is present when data is read, considerably less type information need be encoded with data, resulting in smaller serialization size.
- No manually-assigned field IDs: When a schema changes, both the old and new schema are always present when processing data, so differences may be resolved symbolically, using field names.
34.1 - Specification
Introduction
This document defines Avro. It is intended to be the authoritative specification. Implementations of Avro must adhere to this document.
Schema Declaration
A Schema is represented in JSON by one of:
- A JSON string, naming a defined type.
- A JSON object, of the form:
{"type": "typeName" ...attributes...}
where typeName is either a primitive or derived type name, as defined below. Attributes not defined in this document are permitted as metadata, but must not affect the format of serialized data.
- A JSON array, representing a union of embedded types.
Primitive Types
The set of primitive type names is:
- string: unicode character sequence
- bytes: sequence of 8-bit unsigned bytes
- int: 32-bit signed integer
- long: 64-bit signed integer
- float: single precision (32-bit) IEEE 754 floating-point number
- double: double precision (64-bit) IEEE 754 floating-point number
- boolean: a binary value
- null: no value
Primitive types have no specified attributes.
Primitive type names are also defined type names. Thus, for example, the schema “string” is equivalent to:
Complex Types
Avro supports six kinds of complex types: records, enums, arrays, maps, unions and fixed.
Records
Records use the type name “record” and support three attributes:
- name: a JSON string providing the name of the record (required).
- namespace, a JSON string that qualifies the name;
- doc: a JSON string providing documentation to the user of this schema (optional).
- fields: a JSON array, listing fields (required). Each field is a JSON object with the following attributes:
- name: a JSON string providing the name of the field (required), and
- doc: a JSON string describing this field for users (optional).
- type: A JSON object defining a schema, or a JSON string naming a record definition (required).
- default: A default value for this field, used when reading instances that lack this field (optional). 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. Default values for bytes and fixed fields are JSON strings, where Unicode code points 0-255 are mapped to unsigned 8-bit byte values 0-255.
field default values
| avro type |
json type |
example |
| string |
string |
"foo" |
| bytes |
string |
"\u00FF" |
| int,long |
integer |
1 |
| float,double |
number |
1.1 |
| boolean |
boolean |
true |
| null |
null |
null |
| record |
object |
{"a": 1} |
| enum |
string |
"FOO" |
| array |
array |
[1] |
| map |
object |
{"a": 1} |
| fixed |
string |
"\u00ff" |
- order: specifies how this field impacts sort ordering of this record (optional). Valid values are “ascending” (the default), “descending”, or “ignore”. For more details on how this is used, see the sort order section below.
For example, a linked-list of 64-bit values may be defined with:
{
"type": "record",
"name": "LongList",
"fields" : [
{"name": "value", "type": "long"}, // each element has a long
{"name": "next", "type": ["LongList", "null"]} // optional next element
]
}
Enums
Enums use the type name “enum” and support the following attributes:
- name: a JSON string providing the name of the enum (required).
- namespace, a JSON string that qualifies the name ;
- doc: a JSON string providing documentation to the user of this schema (optional).
- symbols: a JSON array, listing symbols, as JSON strings (required). All symbols in an enum must be unique; duplicates are prohibited.
For example, playing card suits might be defined with:
{
"type": "enum",
"name": "Suit",
"symbols" : ["SPADES", "HEARTS", "DIAMONDS", "CLUBS"]
}
Arrays
Arrays use the type name “array” and support a single attribute:
- items: the schema of the array’s items.
For example, an array of strings is declared with:
{
"type": "array",
"items" : "string"
}
Maps
Maps use the type name “map” and support one attribute:
- values: the schema of the map’s values.
Map keys are assumed to be strings.
For example, a map from string to long is declared with:
{
"type": "map",
"values" : "long"
}
Unions
Unions, as mentioned above, are represented using JSON arrays. For example, ["string", "null"] declares a schema which may be either a string or null.
Unions may not contain more than one schema with the same type, except for the named types record, fixed and enum. For example, unions containing two array types or two map types are not permitted, but two types with different names are permitted. (Names permit efficient resolution when reading and writing unions.)
Unions may not immediately contain other unions.
Fixed
Fixed uses the type name “fixed” and supports two attributes:
- name: a string naming this fixed (required).
- namespace, a string that qualifies the name;
- size: an integer, specifying the number of bytes per value (required).
For example, 16-byte quantity may be declared with:
{"type": "fixed", "size": 16, "name": "md5"}
Names
Record, enums and fixed are named types. Each has a fullname that is composed of two parts; a name and a namespace. Equality of names is defined on the fullname.
The name portion of a fullname, and record field names must:
- start with [A-Za-z_]
- subsequently contain only [A-Za-z0-9_]
A namespace is a dot-separated sequence of such names.
In record, enum and fixed definitions, the fullname is determined in one of the following ways:
A name and namespace are both specified. For example, one might use “name”: “X”, “namespace”: “org.foo” to indicate the fullname org.foo.X.
-
A fullname is specified. If the name specified contains a dot, then it is assumed to be a fullname, and any namespace also specified is ignored. For example, use “name”: “org.foo.X” to indicate the fullname org.foo.X.
-
A name only is specified, i.e., a name that contains no dots. In this case the namespace is taken from the most tightly enclosing schema or protocol. For example, if “name”: “X” is specified, and this occurs within a field of the record definition of org.foo.Y, then the fullname is org.foo.X.
References to previously defined names are as in the latter two cases above: if they contain a dot they are a fullname, if they do not contain a dot, the namespace is the namespace of the enclosing definition.
Primitive type names have no namespace and their names may not be defined in any namespace. A schema may only contain multiple definitions of a fullname if the definitions are equivalent.
Data Serialization
Avro data is always serialized with its schema. Files that store Avro data should always also include the schema for that data in the same file. Avro-based remote procedure call (RPC) systems must also guarantee that remote recipients of data have a copy of the schema used to write that data.
Because the schema used to write data is always available when the data is read, Avro data itself is not tagged with type information. The schema is required to parse data.
In general, both serialization and deserialization proceed as a depth-first, left-to-right traversal of the schema, serializing primitive types as they are encountered.
Encodings
Avro specifies two serialization encodings: binary and JSON. Most applications will use the binary encoding, as it is smaller and faster. But, for debugging and web-based applications, the JSON encoding may sometimes be appropriate.
Binary Encoding
Primitive Types
Primitive types are encoded in binary as follows:
- a string is encoded as a long followed by that many bytes of UTF-8 encoded character data.
- For example, the three-character string “foo” would be encoded as the long value 3 (encoded as hex 06) followed by the UTF-8 encoding of ‘f’, ‘o’, and ‘o’ (the hex bytes 66 6f 6f):
06 66 6f 6f bytes are encoded as a long followed by that many bytes of data.
- int and long values are written using variable-length zig-zag coding. Some examples:
| value |
hex |
| 0 |
00 |
| -1 |
01 |
| 1 |
02 |
| -2 |
03 |
| 2 |
04 |
| … |
|
| -64 |
7f |
| 64 |
80 01 |
| … |
|
- a float is written as 4 bytes. The float is converted into a 32-bit integer using a method equivalent to Java’s floatToIntBits and then encoded in little-endian format.
- a double is written as 8 bytes. The double is converted into a 64-bit integer using a method equivalent to Java’s doubleToLongBits and then encoded in little-endian format.
- a boolean is written as a single byte whose value is either 0 (false) or 1 (true).
null is written as zero bytes.
Complex Types
Complex types are encoded in binary as follows:
Records
A record is encoded by encoding the values of its fields in the order that they are declared. In other words, a record is encoded as just the concatenation of the encodings of its fields. Field values are encoded per their schema.
For example, the record schema
{
"type": "record",
"name": "test",
"fields" : [
{"name": "a", "type": "long"},
{"name": "b", "type": "string"}
]
}
An instance of this record whose a field has value 27 (encoded as hex 36) and whose b field has value “foo” (encoded as hex bytes 06 66 6f 6f), would be encoded simply as the concatenation of these, namely the hex byte sequence:
36 06 66 6f 6f
Enums
An enum is encoded by a int, representing the zero-based position of the symbol in the schema.
For example, consider the enum:
{"type": "enum", "name": "Foo", "symbols": ["A", "B", "C", "D"] }
This would be encoded by an int between zero and three, with zero indicating “A”, and 3 indicating “D”.
Arrays
Arrays are encoded as a series of blocks. Each block consists of a long count value, followed by that many array items. A block with count zero indicates the end of the array. Each item is encoded per the array’s item schema.
If a block’s count is negative, then the count is followed immediately by a long block size, indicating the number of bytes in the block. The actual count in this case is the absolute value of the count written.
For example, the array schema
{"type": "array", "items": "long"}
an array containing the items 3 and 27 could be encoded as the long value 2 (encoded as hex 04) followed by long values 3 and 27 (encoded as hex 06 36) terminated by zero:
04 06 36 00
The blocked representation permits one to read and write arrays larger than can be buffered in memory, since one can start writing items without knowing the full length of the array.
Maps
Maps are encoded as a series of blocks. Each block consists of a long count value, followed by that many key/value pairs. A block with count zero indicates the end of the map. Each item is encoded per the map’s value schema.
If a block’s count is negative, then the count is followed immediately by a long block size, indicating the number of bytes in the block. The actual count in this case is the absolute value of the count written.
The blocked representation permits one to read and write maps larger than can be buffered in memory, since one can start writing items without knowing the full length of the map. The optional block sizes permit fast skipping through data, e.g., when projecting a record to a subset of its fields.
NOTE: Blocking has not yet been fully implemented and may change. Arbitrarily large objects must be easily writable and readable but until we have proven this with an implementation and tests this part of the specification should be considered draft.
Unions
A union is encoded by first writing a long value indicating the zero-based position within the union of the schema of its value. The value is then encoded per the indicated schema within the union.
For example, the union schema ["string","null"] would encode:
-
null as the integer 1 (the index of “null” in the union, encoded as hex 02):
02
-
the string “a” as zero (the index of “string” in the union), followed by the serialized string:
02 02 61
Fixed
Fixed instances are encoded using the number of bytes declared in the schema.
JSON Encoding
Except for unions, the JSON encoding is the same as is used to encode field default values.
The value of a union is encoded in JSON as follows:
- if its type is null, then it is encoded as a JSON null;
- otherwise it is encoded as a JSON object with one name/value pair whose name is the type’s name and whose value is the recursively encoded value. For Avro’s named types (record, fixed or enum) the user-specified name is used, for other types the type name is used.
For example, the union schema ["null","string","Foo"], where Foo is a record name, would encode:
- null as null;
- the string “a” as
{"string": "a"} and
- a Foo instance as
{"Foo": {...}}, where {...} indicates the JSON encoding of a Foo instance.
Note that a schema is still required to correctly process JSON-encoded data. For example, the JSON encoding does not distinguish between int and long, float and double, records and maps, enums and strings, etc.
Sort Order
Avro defines a standard sort order for data. This permits data written by one system to be efficiently sorted by another system. This can be an important optimization, as sort order comparisons are sometimes the most frequent per-object operation. Note also that Avro binary-encoded data can be efficiently ordered without deserializing it to objects.
Data items may only be compared if they have identical schemas. Pairwise comparisons are implemented recursively with a depth-first, left-to-right traversal of the schema. The first mismatch encountered determines the order of the items.
Two items with the same schema are compared according to the following rules.
- int, long, float and double data is ordered by ascending numeric value.
- boolean data is ordered with false before true.
- null data is always equal.
- string data is compared lexicographically by Unicode code point. Note that since UTF-8 is used as the binary encoding for strings, sorting of bytes and string binary data is identical.
- bytes and fixed data are compared lexicographically by unsigned 8-bit values.
- array data is compared lexicographically by element.
- enum data is ordered by the symbol’s position in the enum schema. For example, an enum whose symbols are
["z", "a"] would sort “z” values before “a” values.
- union data is first ordered by the branch within the union, and, within that, by the type of the branch. For example, an
["int", "string"] union would order all int values before all string values, with the ints and strings themselves ordered as defined above.
- record data is ordered lexicographically by field. If a field specifies that its order is:
- “ascending”, then the order of its values is unaltered.
- “descending”, then the order of its values is reversed.
- “ignore”, then its values are ignored when sorting.
- map data may not be compared. It is an error to attempt to compare data containing maps unless those maps are in an
"order":"ignore" record field.
Object Container Files
Avro includes a simple object container file format. A file has a schema, and all objects stored in the file must be written according to that schema. Objects are stored in blocks that may be compressed. Syncronization markers are used between blocks to permit efficient splitting of files for MapReduce processing.
Files may include arbitrary user-specified metadata.
A file consists of:
- A file header, followed by
- one or more file data blocks.
A file header consists of:
- Four bytes, ASCII ‘O’, ‘b’, ‘j’, followed by 1.
- file metadata, including the schema.
- The 16-byte, randomly-generated sync marker for this file.
File metadata consists of:
A long indicating the number of metadata key/value pairs.
For each pair, a string key and bytes value.
All metadata properties that start with “avro.” are reserved. The following file metadata properties are currently used:
- avro.schema contains the schema of objects stored in the file, as JSON data (required).
- avro.codec the name of the compression codec used to compress blocks, as a string. Implementations are required to support the following codecs: “null” and “deflate”. If codec is absent, it is assumed to be “null”. The codecs are described with more detail below.
A file header is thus described by the following schema:
{"type": "record", "name": "org.apache.avro.file.Header",
"fields" : [
{"name": "magic", "type": {"type": "fixed", "name": "Magic", "size": 4}},
{"name": "meta", "type": {"type": "map", "values": "bytes"}},
{"name": "sync", "type": {"type": "fixed", "name": "Sync", "size": 16}}
]
}
A file data block consists of:
- A long indicating the count of objects in this block.
- A long indicating the size in bytes of the serialized objects in the current block, after any codec is applied
- The serialized objects. If a codec is specified, this is compressed by that codec.
- The file’s 16-byte sync marker.
Thus, each block’s binary data can be efficiently extracted or skipped without deserializing the contents. The combination of block size, object counts, and sync markers enable detection of corrupt blocks and help ensure data integrity.
Required Codecs
null
The “null” codec simply passes through data uncompressed.
deflate
The “deflate” codec writes the length of the compressed data (as an Avro-encoded long), followed by data compressed using the deflate algorithm as specified in RFC 1951, and typically implemented using the zlib library. Note that this format (unlike the “zlib format” in RFC 1950) does not have a checksum.
Protocol Declaration
Avro protocols describe RPC interfaces. Like schemas, they are defined with JSON text.
A protocol is a JSON object with the following attributes:
- protocol, a string, the name of the protocol (required);
- namespace, an optional string that qualifies the name;
- doc, an optional string describing this protocol;
- types, an optional list of definitions of named types (records, enums, fixed and errors). An error definition is just like a record definition except it uses “error” instead of “record”. Note that forward references to named types are not permitted.
- messages, an optional JSON object whose keys are message names and whose values are objects whose attributes are described below. No two messages may have the same name.
Messages
A message has attributes:
- a doc, an optional description of the message,
- a request, a list of named, typed parameter schemas (this has the same form as the fields of a record declaration);
- a response schema; and
- an optional union of error schemas.
A request parameter list is processed equivalently to an anonymous record. Since record field lists may vary between reader and writer, request parameters may also differ between the caller and responder, and such differences are resolved in the same manner as record field differences.
Sample Protocol
For example, one may define a simple HelloWorld protocol with:
{
"namespace": "com.acme",
"protocol": "HelloWorld",
"doc": "Protocol Greetings",
"types": [
{"name": "Greeting", "type": "record", "fields": [
{"name": "message", "type": "string"}]},
{"name": "Curse", "type": "error", "fields": [
{"name": "message", "type": "string"}]}
],
"messages": {
"hello": {
"doc": "Say hello.",
"request": [{"name": "greeting", "type": "Greeting" }],
"response": "Greeting",
"errors": ["Curse"]
}
}
}
Message Transport
Messages may be transmitted via different transport mechanisms.
To the transport, a message is an opaque byte sequence.
A transport is a system that supports:
- transmission of request messages
- receipt of corresponding response messages
Servers will send a response message back to the client corresponding to each request message. The mechanism of that correspondance is transport-specific. For example, in HTTP it might be implicit, since HTTP directly supports requests and responses. But a transport that multiplexes many client threads over a single socket would need to tag messages with unique identifiers.
HTTP as Transport
When HTTP is used as a transport, each Avro message exchange is an HTTP request/response pair. All messages of an Avro protocol should share a single URL at an HTTP server. Other protocols may also use that URL. Both normal and error Avro response messages should use the 200 (OK) response code. The chunked encoding may be used for requests and responses, but, regardless the Avro request and response are the entire content of an HTTP request and response. The HTTP Content-Type of requests and responses should be specified as “avro/binary”. Requests should be made using the POST method.
Message Framing
Avro messages are framed as a list of buffers.
Framing is a layer between messages and the transport. It exists to optimize certain operations.
The format of framed message data is:
- a series of buffers, where each buffer consists of:
- a four-byte, big-endian buffer length, followed by
- that many bytes of buffer data.
- A message is always terminated by a zero-lenghted buffer.
Framing is transparent to request and response message formats (described below). Any message may be presented as a single or multiple buffers.
Framing can permit readers to more efficiently get different buffers from different sources and for writers to more efficiently store different buffers to different destinations. In particular, it can reduce the number of times large binary objects are copied. For example, if an RPC parameter consists of a megabyte of file data, that data can be copied directly to a socket from a file descriptor, and, on the other end, it could be written directly to a file descriptor, never entering user space.
A simple, recommended, framing policy is for writers to create a new segment whenever a single binary object is written that is larger than a normal output buffer. Small objects are then appended in buffers, while larger objects are written as their own buffers. When a reader then tries to read a large object the runtime can hand it an entire buffer directly, without having to copy it.
Handshake
RPC requests and responses are prefixed by handshakes. The purpose of the handshake is to ensure that the client and the server have each other’s protocol definition, so that the client can correctly deserialize responses, and the server can correctly deserialize requests. Both clients and servers should maintain a cache of recently seen protocols, so that, in most cases, a handshake will be completed without extra round-trip network exchanges or the transmission of full protocol text.
The handshake process uses the following record schemas:
{
"type": "record",
"name": "HandshakeRequest", "namespace":"org.apache.avro.ipc",
"fields": [
{"name": "clientHash",
"type": {"type": "fixed", "name": "MD5", "size": 16}},
{"name": "clientProtocol", "type": ["null", "string"]},
{"name": "serverHash", "type": "MD5"},
{"name": "meta", "type": ["null", {"type": "map", "values": "bytes"}]}
]
}
{
"type": "record",
"name": "HandshakeResponse", "namespace": "org.apache.avro.ipc",
"fields": [
{"name": "match",
"type": {"type": "enum", "name": "HandshakeMatch",
"symbols": ["BOTH", "CLIENT", "NONE"]}},
{"name": "serverProtocol",
"type": ["null", "string"]},
{"name": "serverHash",
"type": ["null", {"type": "fixed", "name": "MD5", "size": 16}]},
{"name": "meta",
"type": ["null", {"type": "map", "values": "bytes"}]}
]
}
- A client first prefixes each request with a
HandshakeRequest containing just the hash of its protocol and of the server’s protocol (clientHash!=null, clientProtocol=null, serverHash!=null), where the hashes are 128-bit MD5 hashes of the JSON protocol text. If a client has never connected to a given server, it sends its hash as a guess of the server’s hash, otherwise it sends the hash that it previously obtained from this server.
The server responds with a HandshakeResponse containing one of:
-
match=BOTH, serverProtocol=null, serverHash=null if the client sent the valid hash of the server’s protocol and the server knows what protocol corresponds to the client’s hash. In this case, the request is complete and the response data immediately follows the HandshakeResponse.
-
match=CLIENT, serverProtocol!=null, serverHash!=null if the server has previously seen the client’s protocol, but the client sent an incorrect hash of the server’s protocol. The request is complete and the response data immediately follows the HandshakeResponse. The client must use the returned protocol to process the response and should also cache that protocol and its hash for future interactions with this server.
-
match=NONE if the server has not previously seen the client’s protocol. The serverHash and serverProtocol may also be non-null if the server’s protocol hash was incorrect.
In this case the client must then re-submit its request with its protocol text (clientHash!=null, clientProtocol!=null, serverHash!=null) and the server should respond with a successful match (match=BOTH, serverProtocol=null, serverHash=null) as above.
The meta field is reserved for future handshake enhancements.
A call consists of a request message paired with its resulting response or error message. Requests and responses contain extensible metadata, and both kinds of messages are framed as described above.
The format of a call request is:
- request metadata, a map with values of type bytes
- the message name, an Avro string, followed by
- the message parameters. Parameters are serialized according to the message’s request declaration.
The format of a call response is:
- response metadata, a map with values of type bytes
- a one-byte error flag boolean, followed by either:
- if the error flag is false, the message response, serialized per the message’s response schema.
- if the error flag is true, the error, serialized per the message’s error union schema.
Schema Resolution
A reader of Avro data, whether from an RPC or a file, can always parse that data because its schema is provided. But that schema may not be exactly the schema that was expected. For example, if the data was written with a different version of the software than it is read, then records may have had fields added or removed. This section specifies how such schema differences should be resolved.
We call the schema used to write the data as the writer’s schema, and the schema that the application expects the reader’s schema. Differences between these should be resolved as follows:
- It is an error if the two schemas do not match.
To match, one of the following must hold:
-
both schemas are arrays whose item types match
-
both schemas are maps whose value types match
-
both schemas are enums whose names match
-
both schemas are fixed whose sizes and names match
-
both schemas are records with the same name
-
either schema is a union
-
both schemas have same primitive type
-
the writer’s schema may be promoted to the reader’s as follows:
- int is promotable to long, float, or double
- long is promotable to float or double
- float is promotable to double
- string is promotable to bytes
- bytes is promotable to string
-
if both are records:
- the ordering of fields may be different: fields are matched by name.
- schemas for fields with the same name in both records are resolved recursively.
- if the writer’s record contains a field with a name not present in the reader’s record, the writer’s value for that field is ignored.
- if the reader’s record schema has a field that contains a default value, and writer’s schema does not have a field with the same name, then the reader should use the default value from its field.
- if the reader’s record schema has a field with no default value, and writer’s schema does not have a field with the same name, an error is signalled.
-
if both are enums:
if the writer’s symbol is not present in the reader’s enum, then an error is signalled.
This resolution algorithm is applied recursively to the reader’s and writer’s array item schemas.
This resolution algorithm is applied recursively to the reader’s and writer’s value schemas.
The first schema in the reader’s union that matches the selected writer’s union schema is recursively resolved against it. if none match, an error is signalled.
- if reader’s is a union, but writer’s is not:
The first schema in the reader’s union that matches the writer’s schema is recursively resolved against it. If none match, an error is signalled.
- if writer’s is a union, but reader’s is not:
If the reader’s schema matches the selected writer’s schema, it is recursively resolved against it. If they do not match, an error is signalled.
A schema’s “doc” fields are ignored for the purposes of schema resolution. Hence, the “doc” portion of a schema may be dropped at serialization.
34.2 - GenAvro Language
Introduction
This document defines GenAvro, an experimental higher-level language for authoring Avro schemata. Before reading this document, you should have familiarity with the concepts of schemata and protocols, as well as the various primitive and complex types available in Avro.
N.B. This feature is considered experimental in the current version of Avro and the language has not been finalized. Although major changes are unlikely, some syntax may change in future versions of Avro.
Overview
Purpose
The aim of the GenAvro language is to enable developers to author schemata in a way that feels more similar to common programming languages like Java, C++, or Python. Additionally, the GenAvro language may feel more familiar for those users who have previously used the interface description languages (IDLs) in other frameworks like Thrift, Protocol Buffers, or CORBA.
Usage
Each GenAvro file defines a single Avro Protocol, and thus generates as its output a JSON-format Avro Protocol file with extension .avpr.
To convert a .genavro file into a .avpr file, it must be processed by the avroj genavro tool. For example:
$ java -jar avroj-1.3.0.jar genavro src/test/genavro/input/namespaces.genavro /tmp/namespaces.avpr
$ head /tmp/namespaces.avpr
{
"protocol" : "TestNamespace",
"namespace" : "avro.test.protocol",
The avroj genavro tool can also process input to and from stdin and stdout. See avroj genavro –help for full usage information.
Defining a Protocol in GenAvro
A GenAvro file consists of exactly one protocol definition. The minimal protocol is defined by the following code:
This is equivalent to (and generates) the following JSON protocol definition:
{
"protocol" : "MyProtocol",
"types" : [ ],
"messages" : {
}
}
The namespace of the protocol may be changed using the @namespace annotation:
@namespace("mynamespace")
protocol MyProtocol {
}
3 This notation is used throughout GenAvro as a way of specifying properties for the annotated element, as will be described later in this document.
Protocols in GenAvro can contain the following items:
- Definitions of named schemata, including records, errors, enums, and fixeds.
- Definitions of RPC messages
Defining an Enumeration
Enums are defined in GenAvro using a syntax similar to C or Java:
enum Suit {
SPADES, DIAMONDS, CLUBS, HEARTS
}
Note that, unlike the JSON format, anonymous enums cannot be defined.
Defining a Fixed Length Field
Fixed fields are defined using the following syntax:
fixed MD5(16);
This example defines a fixed-length type called MD5 which contains 16 bytes.
Defining Records and Errors
Records are defined in GenAvro using a syntax similar to a struct definition in C:
record Employee {
string name;
boolean active;
long salary;
}
The above example defines a record with the name “Employee” with three fields.
To define an error, simply use the keyword error instead of record. For example:
error Kaboom {
string explanation;
int result_code;
}
Each field in a record or error consists of a type and a name, along with optional property annotations.
A type reference in GenAvro must be one of:
- A primitive type
- A named schema defined prior to this usage in the same Protocol
- A complex type (array, map, or union)
Primitive Types
The primitive types supported by GenAvro are the same as those supported by Avro’s JSON format. This list includes int, long, string, boolean, float, double, null, and bytes.
References to Named Schemata
If a named schema has already been defined in the same GenAvro file, it may be referenced by name as if it were a primitive type:
record Card {
Suit suit; // refers to the enum Card defined above
int number;
}
Complex Types
Arrays
Array types are written in a manner that will seem familiar to C++ or Java programmers. An array of any type t is denoted array<t>. For example, an array of strings is denoted array<string>, and a multidimensional array of Foo records would be array<array<Foo>>.
Maps
Map types are written similarly to array types. An array that contains values of type t is written map<t>. As in the JSON schema format, all maps contain string-type keys.
Unions
Union types are denoted as union { typeA, typeB, typeC, … }. For example, this record contains a string field that is optional (unioned with null):
record RecordWithUnion {
union { null, string } optionalString;
}
Note that the same restrictions apply to GenAvro unions as apply to unions defined in the JSON format; namely, a record may not contain multiple elements of the same type.
Defining RPC Messages
The syntax to define an RPC message within a GenAvro protocol is similar to the syntax for a method declaration within a C header file or a Java interface. To define an RPC message add which takes two arguments named foo and bar, returning an int, simply include the following definition within the protocol:
int add(int foo, int bar);
To define a message with no response, you may use the alias void, equivalent to the Avro null type:
void logMessage(string message);
If you have previously defined an error type within the same protocol, you may declare that a message can throw this error using the syntax:
void goKaboom() throws Kaboom;
Other Language Features
All Java-style comments are supported within a GenAvro file. Any text following // on a line is ignored, as is any text between /* and */, possibly spanning multiple lines.
Escaping Identifiers
Occasionally, one will need to use a reserved language keyword as an identifier. In order to do so, backticks (`) may be used to escape the identifier. For example, to define a message with the literal name error, you may write:
This syntax is allowed anywhere an identifier is expected.
Annotations for Ordering and Namespaces
Java-style annotations may be used to add additional properties to types throughout GenAvro. For example, to specify the sort order of a field within a record, one may use the @order annotation as follows:
record MyRecord {
@order("ascending")
string myAscendingSortField;
@order("descending")
string myDescendingField;
@order("ignore")
string myIgnoredField;
}
Similarly, a @namespace annotation may be used to modify the namespace when defining a named schema. For example:
@namespace("org.apache.avro.firstNamespace")
protocol MyProto {
@namespace("org.apache.avro.someOtherNamespace")
record Foo {}
record Bar {}
}
will define a protocol in the firstNamespace namespace. The record Foo will be defined in someOtherNamespace and Bar will be defined in firstNamespace as it inherits its default from its container.
Complete Example
The following is a complete example of a GenAvro file that shows most of the above features:
/**
* An example protocol in GenAvro
*/
@namespace("org.apache.avro.test")
protocol Simple {
@aliases(["org.foo.KindOf"])
enum Kind {
FOO,
BAR, // the bar enum value
BAZ
}
fixed MD5(16);
record TestRecord {
@order("ignore")
string name;
@order("descending")
Kind kind;
MD5 hash;
union { MD5, null} nullableHash;
array<long> arrayOfLongs;
}
error TestError {
string message;
}
string hello(string greeting);
TestRecord echo(TestRecord `record`);
int add(int arg1, int arg2);
bytes echoBytes(bytes data);
void `error`() throws TestError;
void ping() oneway;
}
Additional examples may be found in the Avro source tree under the src/test/genavro/input directory.
34.3 - Wiki
The Wiki page can be found here.
34.4 - FAQ
What is Avro?
Avro is a data serialization system.
Avro provides:
- Rich data structures.
- A compact, fast, binary data format.
- A container file, to store persistent data.
- Remote procedure call (RPC).
- Simple integration with dynamic languages. Code generation is not required to read or write data files nor to use or implement RPC protocols. Code generation as an optional optimization, only worth implementing for statically typed languages.
How can I get started quickly with Avro?
Check out the Quick Start Guide.
How do I statically compile a schema or protocol into generated code?
In Java
- Add the avro jar, the jackson-mapper-asl.jar and jackson-core-asl.jar to your CLASSPATH.
- Run java org.apache.avro.specific.SpecificCompiler .
This appears to be out of date, the SpecificCompiler requires two arguments, presumably an input and and output file, but it isn’t clear that this does.
Or use the Schema or Protocol Ant tasks. Avro’s build.xml provides examples of how these are used.
Lastly, you can also use the “avro-tools” jar which ships with an Avro release. Just use the “compile (schema|protocol)” command.
How are Strings represented in Java?
They use org.apache.avro.util.Utf8, not java.lang.String.
More generally, how do Avro types map to Java types?
The mappings are documented in the package javadoc for generic, specific and reflect API.
From Doug Cutting:
HDFS splits files into blocks, and mapreduce runs a map task for each block. When the task starts, it needs to be able to seek into the file to the start of the block process through the block’s end. If the file were, e.g., a gzip file, this would not be possible, since gzip files must be decompressed from the start. One cannot seek into the middle of a gzip file and start decompressing. So Hadoop’s SequenceFile places a marker periodically (~64k) in the file at record and compression boundaries, where processing can be sensibly started. Then, when a map task starts processing an HDFS block, it finds the first marker after the block’s start and continues through the first marker in the next block of the file. This requires a bit of non-local access (~0.1%). Avro’s data file uses the same method as SequenceFile.
Why isn’t every value in Avro nullable?
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 of every value stored and
corresponding computational costs to create this bit on write and
interpret it on read. These costs are wasted when values may not in
fact be null, as is the case in many datasets. In Avro such costs are
only paid when values may actually be null.
Also, allowing values to be null is a well-known source of errors. In
Avro, a value declared as non-null will always be non-null and programs
need not test for null values when processing it nor will they ever fail
for lack of such tests.
Tony Hoare calls his invention of null references his “Billion Dollar
Mistake”.
http://qconlondon.com/london-2009/presentation/Null+References:+The+Billion+Dollar+Mistake
Also note that in some programming languages not all values are permitted to be null. For example, in Java, values of type boolean, byte, short, char, int, float, long, and double may not be null.
How can I serialize directly to/from a byte array?
As pointed out in the specification, Avro data should always be stored with its schema. The Avro provided classes DataFileWriter, DataFileReader, and DataFileStream all ensure this by serializing the Schema in a container header. In some special cases, such as when implementing a new storage system or writing unit tests, you may need to write and read directly with the bare Avro serialized values. The following examples use code generated for Java from an Avro Schema, specifically the User example from the Quickstart guide.
Serializing to a byte array
This example takes a User object and returns a newly allocated byte array with the Avro serialization of that user.
ByteArrayOutputStream out = new ByteArrayOutputStream();
BinaryEncoder encoder = EncoderFactory.get().binaryEncoder(out, null);
DatumWriter<User> writer = new SpecificDatumWriter<User>(User.getClassSchema());
writer.write(user, encoder);
encoder.flush();
out.close();
byte[] serializedBytes = out.toByteArray();
Deserializing from a byte array
This example takes a byte array containing the Avro serialization of a user and returns a User object.SpecificDatumReader reader = new SpecificDatumReader(User.getClassSchema());
Decoder decoder = DecoderFactory.get().binaryDecoder(bytes, null);
User user = reader.read(null, decoder);
Note: If you are serializing or deserializing in a loop or as a method, you should be reusing objects, readers and/or writers. Check the JavaDoc to see how to reuse the objects.
35 - Apache Avro™ 1.2.0 Documentation
Introduction
Avro is a data serialization system.
Avro provides:
- Rich data structures.
- A compact, fast, binary data format.
- A container file, to store persistent data.
- Remote procedure call (RPC).
- Simple integration with dynamic languages. Code generation is not required to read or write data files nor to use or implement RPC protocols. Code generation as an optional optimization, only worth implementing for statically typed languages.
Schemas
Avro relies on schemas. When Avro data is read, the schema used when writing it is always present. This permits each datum to be written with no per-value overheads, making serialization both fast and small. This also facilitates use with dynamic, scripting languages, since data, together with its schema, is fully self-describing.
When Avro data is stored in a file, its schema is stored with it, so that files may be processed later by any program. If the program reading the data expects a different schema this can be easily resolved, since both schemas are present.
When Avro is used in RPC, the client and server exchange schemas in the connection handshake. (This can be optimized so that, for most calls, no schemas are actually transmitted.) Since both client and server both have the other’s full schema, correspondence between same named fields, missing fields, extra fields, etc. can all be easily resolved.
Avro schemas are defined with JSON . This facilitates implementation in languages that already have JSON libraries.
Comparison with other systems
Avro provides functionality similar to systems such as Thrift, Protocol Buffers, etc. Avro differs from these systems in the following fundamental aspects.
- Dynamic typing: Avro does not require that code be generated. Data is always accompanied by a schema that permits full processing of that data without code generation, static datatypes, etc. This facilitates construction of generic data-processing systems and languages.
- Untagged data: Since the schema is present when data is read, considerably less type information need be encoded with data, resulting in smaller serialization size.
- No manually-assigned field IDs: When a schema changes, both the old and new schema are always present when processing data, so differences may be resolved symbolically, using field names.
35.1 - Specification
Introduction
This document defines Avro. It is intended to be the authoritative specification. Implementations of Avro must adhere to this document.
Schema Declaration
A Schema is represented in JSON by one of:
- A JSON string, naming a defined type.
- A JSON object, of the form:
{"type": "typeName" ...attributes...}
where typeName is either a primitive or derived type name, as defined below. Attributes not defined in this document are permitted as metadata, but must not affect the format of serialized data.
- A JSON array, representing a union of embedded types.
Primitive Types
The set of primitive type names is:
- string: unicode character sequence
- bytes: sequence of 8-bit bytes
- int: 32-bit signed integer
- long: 64-bit signed integer
- float: single precision (32-bit) IEEE 754 floating-point number
- double: double precision (64-bit) IEEE 754 floating-point number
- boolean: a binary value
- null: no value
Primitive types have no specified attributes.
Primitive type names are also defined type names. Thus, for example, the schema “string” is equivalent to:
Complex Types
Avro supports six kinds of complex types: records, enums, arrays, maps, unions and fixed.
Records
Records use the type name “record” and support three attributes:
- name: a JSON string providing the name of the record (required).
- doc: a JSON string providing documentation to the user of this schema (optional).
- fields: a JSON array, listing fields (required). Each field is a JSON object with the following attributes:
- name: a JSON string providing the name of the field (required), and
- type: A JSON object defining a schema, or a JSON string naming a record definition (required).
- default: A default value for this field, used when reading instances that lack this field (optional). 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.
field default values
| avro type |
json type |
example |
| string |
string |
"foo" |
| bytes |
string |
"\u00FF" |
| int,long |
integer |
1 |
| float,double |
number |
1.1 |
| boolean |
boolean |
true |
| null |
null |
null |
| record |
object |
{"a": 1} |
| enum |
string |
"FOO" |
| array |
array |
[1] |
| map |
object |
{"a": 1} |
| fixed |
string |
"\u00ff" |
- order: specifies how this field impacts sort ordering of this record (optional). Valid values are “ascending” (the default), “descending”, or “ignore”. For more details on how this is used, see the sort order section below.
For example, a linked-list of 64-bit values may be defined with:
{
"type": "record",
"name": "LongList",
"fields" : [
{"name": "value", "type": "long"}, // each element has a long
{"name": "next", "type": ["LongList", "null"]} // optional next element
]
}
Enums
Enums use the type name “enum” and support the following attributes:
- name: a JSON string providing the name of the enum (required).
- symbols: a JSON array, listing symbols, as JSON strings (required).
For example, playing card suits might be defined with:
{
"type": "enum",
"name": "Suit",
"symbols" : ["SPADES", "HEARTS", "DIAMONDS", "CLUBS"]
}
Arrays
Arrays use the type name “array” and support a single attribute:
- items: the schema of the array’s items.
For example, an array of strings is declared with:
{
"type": "array",
"items" : "string"
}
Maps
Maps use the type name “map” and support one attribute:
- values: the schema of the map’s values.
Map keys are assumed to be strings.
For example, a map from string to long is declared with:
{
"type": "map",
"values" : "long"
}
Unions
Unions, as mentioned above, are represented using JSON arrays. For example, ["string", "null"] declares a schema which may be either a string or null.
Unions may not contain more than one schema with the same type, except for the named types record, fixed and enum. For example, unions containing two array types or two map types are not permitted, but two types with different names are permitted. (Names permit efficient resolution when reading and writing unions.)
Unions may not immediately contain other unions.
Fixed
Fixed uses the type name “fixed” and supports three attributes:
- name: the name of the fixed (required).
- size: an integer, specifying the number of bytes per value (required).
For example, 16-byte quantity may be declared with:
{"type": "fixed", "size": 16, "name": "md5"}
Identifiers
Record, field and enum names must:
- start with [A-Za-z_]
- subsequently contain only [A-Za-z0-9_]
Data Serialization
Avro data is always serialized with its schema. Files that store Avro data should always also include the schema for that data in the same file. Avro-based remote procedure call (RPC) systems must also guarantee that remote recipients of data have a copy of the schema used to write that data.
Because the schema used to write data is always available when the data is read, Avro data itself is not tagged with type information. The schema is required to parse data.
In general, both serialization and deserialization proceed as a depth-first, left-to-right traversal of the schema, serializing primitive types as they are encountered.
Encodings
Avro specifies two serialization encodings: binary and JSON. Most applications will use the binary encoding, as it is smaller and faster. But, for debugging and web-based applications, the JSON encoding may sometimes be appropriate.
Binary Encoding
Primitive Types
Primitive types are encoded in binary as follows:
- a string is encoded as a long followed by that many bytes of UTF-8 encoded character data.
- For example, the three-character string “foo” would be encoded as the long value 3 (encoded as hex 06) followed by the UTF-8 encoding of ‘f’, ‘o’, and ‘o’ (the hex bytes 66 6f 6f):
06 66 6f 6f bytes are encoded as a long followed by that many bytes of data.
- int and long values are written using variable-length zig-zag coding. Some examples:
| value |
hex |
| 0 |
00 |
| -1 |
01 |
| 1 |
02 |
| -2 |
03 |
| 2 |
04 |
| … |
|
| -64 |
7f |
| 64 |
80 01 |
| … |
|
- a float is written as 4 bytes. The float is converted into a 32-bit integer using a method equivalent to Java’s floatToIntBits and then encoded in little-endian format.
- a double is written as 8 bytes. The double is converted into a 64-bit integer using a method equivalent to Java’s doubleToLongBits and then encoded in little-endian format.
- a boolean is written as a single byte whose value is either 0 (false) or 1 (true).
null is written as zero bytes.
Complex Types
Complex types are encoded in binary as follows:
Records
A record is encoded by encoding the values of its fields in the order that they are declared. In other words, a record is encode as just the concatenation of its field’s encodings. Field values are encoded per their schema.
For example, the record schema.
{
"type": "record",
"name": "test",
"fields" : [
{"name": "a", "type": "long"},
{"name": "b", "type": "string"}
]
}
An instance of this record whose a field has value 27 (encoded as hex 36) and whose b field has value “foo” (encoded as hex bytes 06 66 6f 6f), would be encoded simply as the concatenation of these, namely the hex byte sequence:
36 06 66 6f 6f
Enums
An enum is encoded by a int, representing the zero-based position of the symbol in the schema.
For example, consider the enum:
{"type": "enum", "name": "Foo", "symbols": ["A", "B", "C", "D"] }
This would be encoded by an int between zero and three, with zero indicating “A”, and 3 indicating “D”.
Arrays
Arrays are encoded as a series of blocks. Each block consists of a long count value, followed by that many array items. A block with count zero indicates the end of the array. Each item is encoded per the array’s item schema.
If a block’s count is negative, then the count is followed immediately by a long block size, indicating the number of bytes in the block. The actual count in this case is the absolute value of the count written.
For example, the array schema
{"type": "array", "items": "long"}
an array containing the items 3 and 27 could be encoded as the long value 2 (encoded as hex 04) followed by long values 3 and 27 (encoded as hex 06 36) terminated by zero:
04 06 36 00
The blocked representation permits one to read and write arrays larger than can be buffered in memory, since one can start writing items without knowing the full length of the array. The optional block sizes permit fast skipping through data, e.g., when projecting a record to a subset of its fields.
Maps
Maps are encoded as a series of blocks. Each block consists of a long count value, followed by that many key/value pairs. A block with count zero indicates the end of the map. Each item is encoded per the map’s value schema.
If a block’s count is negative, then the count is followed immediately by a long block size, indicating the number of bytes in the block. The actual count in this case is the absolute value of the count written.
The blocked representation permits one to read and write maps larger than can be buffered in memory, since one can start writing items without knowing the full length of the map. The optional block sizes permit fast skipping through data, e.g., when projecting a record to a subset of its fields.
NOTE: Blocking has not yet been fully implemented and may change. Arbitrarily large objects must be easily writable and readable but until we have proven this with an implementation and tests this part of the specification should be considered draft.
Unions
A union is encoded by first writing a long value indicating the zero-based position within the union of the schema of its value. The value is then encoded per the indicated schema within the union.
For example, the union schema ["string","null"] would encode:
-
null as the integer 1 (the index of “null” in the union, encoded as hex 02):
02
-
the string “a” as zero (the index of “string” in the union), followed by the serialized string:
00 02 61
Fixed
Fixed instances are encoded using the number of bytes declared in the schema.
JSON Encoding
Except for unions, the JSON encoding is the same as is used to encode field default values.
The value of a union is encoded in JSON as follows:
- if its type is null, then it is encoded as a JSON null;
- otherwise it is encoded as a JSON object with one name/value pair whose name is the type’s name and whose value is the recursively encoded value. For Avro’s named types (record, fixed or enum) the user-specified name is used, for other types the type name is used.
For example, the union schema ["null","string","Foo"], where Foo is a record name, would encode:
- null as null;
- the string “a” as
{"string": "a"} and
- a Foo instance as
{"Foo": {...}}, where {...} indicates the JSON encoding of a Foo instance.
Note that a schema is still required to correctly process JSON-encoded data. For example, the JSON encoding does not distinguish between int and long, float and double, records and maps, enums and strings, etc.
Sort Order
Avro defines a standard sort order for data. This permits data written by one system to be efficiently sorted by another system. This can be an important optimization, as sort order comparisons are sometimes the most frequent per-object operation. Note also that Avro binary-encoded data can be efficiently ordered without deserializing it to objects.
Data items may only be compared if they have identical schemas. Pairwise comparisons are implemented recursively with a depth-first, left-to-right traversal of the schema. The first mismatch encountered determines the order of the items.
Two items with the same schema are compared according to the following rules.
- int, long, float and double data is ordered by ascending numeric value.
- boolean data is ordered with false before true.
- null data is always equal.
- string data is compared lexicographically. Note that since UTF-8 is used as the binary encoding of strings, sorting by bytes and characters is equivalent.
- bytes and fixed data are compared lexicographically by byte.
- array data is compared lexicographically by element.
- enum data is ordered by the symbol’s position in the enum schema. For example, an enum whose symbols are
["z", "a"] would sort “z” values before “a” values.
- union data is first ordered by the branch within the union, and, within that, by the type of the branch. For example, an
["int", "string"] union would order all int values before all string values, with the ints and strings themselves ordered as defined above.
- record data is ordered lexicographically by field. If a field specifies that its order is:
- “ascending”, then the order of its values is unaltered.
- “descending”, then the order of its values is reversed.
- “ignore”, then its values are ignored when sorting.
- map data may not be compared. It is an error to attempt to compare data containing maps unless those maps are in an
"order":"ignore" record field.
Object Container Files
Avro includes a simple object container file format. A file has a schema, and all objects stored in the file must be written according to that schema. Objects are stored in blocks that may be compressed. Syncronization markers are used between blocks to permit efficient splitting of files for MapReduce processing.
Files may include arbitrary user-specified metadata.
A file consists of:
- A header, followed by
- one or more blocks.
There are two kinds of blocks, normal and metadata. All files must contain at least one metadata block. A file terminates with its last metadata block. Any data after the last metadata block is ignored.
A header consists of:
- Four bytes, ASCII ‘O’, ‘b’, ‘j’, followed by zero.
- A 16-byte sync marker.
A metadata block consists of:
The file’s 16-byte sync marker.
A long with value -1, identifying this as a metadata block.
A long indicating the size in bytes of this block.
A long indicating the number of metadata key/value pairs.
For each pair, a string key and bytes value.
The size in bytes of this block as a 4-byte big-endian integer.
When a file is closed normally, this terminates the file and permits one to efficiently seek to the start of the metadata. If the sync marker there does not match that at the start of the file, then one must scan for the last metadata in the file.
The following metadata properties are reserved:
- schema contains the schema of objects stored in the file, as a string.
- count contains the number of objects in the file as a decimal ASCII string.
- codec codec the name of the compression codec used to compress blocks, as a string. The only value for codec currently supported is “null” (meaning no compression is performed). If codec is absent, it is assumed to be “null”.
sync the 16-byte sync marker used in this file, as a byte sequence.
A normal block consists of:
- The file’s 16-byte sync marker.
- A long indicating the size in bytes of this block in the file.
- The serialized objects. If a codec is specified, this is compressed by that codec.
- Note that this format supports appends, since multiple metadata blocks are permitted.
To be robust to application failure, implementations can write metadata periodically to limit the amount of the file that must be scanned to find the last metadata block.
Protocol Declaration
Avro protocols describe RPC interfaces. Like schemas, they are defined with JSON text.
A protocol is a JSON object with the following attributes:
- name, string, to distinguish it from other protocols;
- namespace, a string which qualifies the name;
- types, a list of record, enum and error definitions. An error definition is just like a record definition except it uses “error” instead of “record”. Note that forward references to records, enums and errors are not currently supported.
- messages, a JSON object whose keys are message names and whose values are objects whose attributes are described below. No two messages may have the same name.
Messages
A message has attributes:
- a request, a list of named, typed parameter schemas (this has the same form as the fields of a record declaration);
- a response schema; and
- an optional union of error schemas.
A request parameter list is processed equivalently to an anonymous record. Since record field lists may vary between reader and writer, request parameters may also differ between the caller and responder, and such differences are resolved in the same manner as record field differences.
Sample Protocol
For example, one may define a simple HelloWorld protocol with:
{
"namespace": "com.acme",
"protocol": "HelloWorld",
"types": [
{"name": "Greeting", "type": "record", "fields": [
{"name": "message", "type": "string"}]},
{"name": "Curse", "type": "error", "fields": [
{"name": "message", "type": "string"}]}
],
"messages": {
"hello": {
"request": [{"name": "greeting", "type": "Greeting" }],
"response": "Greeting",
"errors": ["Curse"]
}
}
}
Message Transport
Messages may be transmitted via different transport mechanisms. For example, one might use the HTTP, raw sockets, or SSL, etc. This document specifies formats for request and response message data, but it does not yet specify any details of how message data is encapsulated in different transports.
To the transport, a message is an opaque byte sequence.
A transport is a system that supports:
- transmission of request messages
- receipt of corresponding response messages
Servers will send a response message back to the client corresponding to each request message. The mechanism of that correspondance is transport-specific. For example, in HTTP it might be implicit, since HTTP directly supports requests and responses. But a transport that multiplexes many client threads over a single socket would need to tag messages with unique identifiers.
Message Framing
Avro messages are framed as a list of buffers.
Framing is a layer between messages and the transport. It exists to optimize certain operations.
The format of framed message data is:
- a series of buffers, where each buffer consists of:
- a four-byte, big-endian buffer length, followed by
- that many bytes of buffer data.
- A message is always terminated by a zero-lenghted buffer.
Framing is transparent to request and response message formats (described below). Any message may be presented as a single or multiple buffers.
Framing can permit readers to more efficiently get different buffers from different sources and for writers to more efficiently store different buffers to different destinations. In particular, it can reduce the number of times large binary objects are copied. For example, if an RPC parameter consists of a megabyte of file data, that data can be copied directly to a socket from a file descriptor, and, on the other end, it could be written directly to a file descriptor, never entering user space.
A simple, recommended, framing policy is for writers to create a new segment whenever a single binary object is written that is larger than a normal output buffer. Small objects are then appended in buffers, while larger objects are written as their own buffers. When a reader then tries to read a large object the runtime can hand it an entire buffer directly, without having to copy it.
Handshake
RPC requests and responses are prefixed by handshakes. The purpose of the handshake is to ensure that the client and the server have each other’s protocol definition, so that the client can correctly deserialize responses, and the server can correctly deserialize requests. Both clients and servers should maintain a cache of recently seen protocols, so that, in most cases, a handshake will be completed without extra round-trip network exchanges or the transmission of full protocol text.
The handshake process uses the following record schemas:
{
"type": "record",
"name": "HandshakeRequest", "namespace":"org.apache.avro.ipc",
"fields": [
{"name": "clientHash",
"type": {"type": "fixed", "name": "MD5", "size": 16}},
{"name": "clientProtocol", "type": ["null", "string"]},
{"name": "serverHash", "type": "MD5"},
{"name": "meta", "type": ["null", {"type": "map", "values": "bytes"}]}
]
}
{
"type": "record",
"name": "HandshakeResponse", "namespace": "org.apache.avro.ipc",
"fields": [
{"name": "match",
"type": {"type": "enum", "name": "HandshakeMatch",
"symbols": ["BOTH", "CLIENT", "NONE"]}},
{"name": "serverProtocol",
"type": ["null", "string"]},
{"name": "serverHash",
"type": ["null", {"type": "fixed", "name": "MD5", "size": 16}]},
{"name": "meta",
"type": ["null", {"type": "map", "values": "bytes"}]}
]
}
- A client first prefixes each request with a
HandshakeRequest containing just the hash of its protocol and of the server’s protocol (clientHash!=null, clientProtocol=null, serverHash!=null), where the hashes are 128-bit MD5 hashes of the JSON protocol text. If a client has never connected to a given server, it sends its hash as a guess of the server’s hash, otherwise it sends the hash that it previously obtained from this server.
The server responds with a HandshakeResponse containing one of:
-
match=BOTH, serverProtocol=null, serverHash=null if the client sent the valid hash of the server’s protocol and the server knows what protocol corresponds to the client’s hash. In this case, the request is complete and the response data immediately follows the HandshakeResponse.
-
match=CLIENT, serverProtocol!=null, serverHash!=null if the server has previously seen the client’s protocol, but the client sent an incorrect hash of the server’s protocol. The request is complete and the response data immediately follows the HandshakeResponse. The client must use the returned protocol to process the response and should also cache that protocol and its hash for future interactions with this server.
-
match=NONE serverProtocol!=null, serverHash!=null if the server has not previously seen the client’s protocol and the client sent an incorrect hash of the server’s protocol.
In this case The client must then re-submit its request with its protocol text (clientHash!=null, clientProtocol!=null, serverHash!=null) and the server should respond with with a successful match (match=BOTH, serverProtocol=null, serverHash=null) as above.
The meta field is reserved for future handshake enhancements.
A call consists of a request message paired with its resulting response or error message. Requests and responses contain extensible metadata, and both kinds of messages are framed as described above.
The format of a call request is:
- request metadata, a map with values of type bytes
- the message name, an Avro string, followed by
- the message parameters. Parameters are serialized according to the message’s request declaration.
The format of a call response is:
- response metadata, a map with values of type bytes
- a one-byte error flag boolean, followed by either:
- if the error flag is false, the message response, serialized per the message’s response schema.
- if the error flag is true, the error, serialized per the message’s error union schema.
Schema Resolution
A reader of Avro data, whether from an RPC or a file, can always parse that data because its schema is provided. But that schema may not be exactly the schema that was expected. For example, if the data was written with a different version of the software than it is read, then records may have had fields added or removed. This section specifies how such schema differences should be resolved.
We call the schema used to write the data as the writer’s schema, and the schema that the application expects the reader’s schema. Differences between these should be resolved as follows:
- It is an error if the two schemas do not match.
To match, one of the following must hold:
-
both schemas are arrays whose item types match
-
both schemas are maps whose value types match
-
both schemas are enums whose names match
-
both schemas are fixed whose sizes and names match
-
both schemas are records with the same name
-
either schema is a union
-
both schemas have same primitive type
-
the writer’s schema may be promoted to the reader’s as follows:
- int is promotable to long, float, or double
- long is promotable to float or double
- float is promotable to double
-
if both are records:
- the ordering of fields may be different: fields are matched by name.
- schemas for fields with the same name in both records are resolved recursively.
- if the writer’s record contains a field with a name not present in the reader’s record, the writer’s value for that field is ignored.
- if the reader’s record schema has a field that contains a default value, and writer’s schema does not have a field with the same name, then the reader should use the default value from its field.
- if the reader’s record schema has a field with no default value, and writer’s schema does not have a field with the same name, then the field’s value is unset.
-
if both are enums:
if the writer’s symbol is not present in the reader’s enum, then the enum’s value is unset.
This resolution algorithm is applied recursively to the reader’s and writer’s array item schemas.
This resolution algorithm is applied recursively to the reader’s and writer’s value schemas.
The first schema in the reader’s union that matches the selected writer’s union schema is recursively resolved against it. if none match, an error is signalled.
- if reader’s is a union, but writer’s is not:
The first schema in the reader’s union that matches the writer’s schema is recursively resolved against it. If none match, an error is signalled.
- if writer’s is a union, but reader’s is not:
If the reader’s schema matches the selected writer’s schema, it is recursively resolved against it. If they do not match, an error is signalled.
35.2 - Wiki
The Wiki page can be found here.
35.3 - FAQ
What is Avro?
Avro is a data serialization system.
Avro provides:
- Rich data structures.
- A compact, fast, binary data format.
- A container file, to store persistent data.
- Remote procedure call (RPC).
- Simple integration with dynamic languages. Code generation is not required to read or write data files nor to use or implement RPC protocols. Code generation as an optional optimization, only worth implementing for statically typed languages.
How can I get started quickly with Avro?
Check out the Quick Start Guide.
How do I statically compile a schema or protocol into generated code?
In Java
- Add the avro jar, the jackson-mapper-asl.jar and jackson-core-asl.jar to your CLASSPATH.
- Run java org.apache.avro.specific.SpecificCompiler .
This appears to be out of date, the SpecificCompiler requires two arguments, presumably an input and and output file, but it isn’t clear that this does.
Or use the Schema or Protocol Ant tasks. Avro’s build.xml provides examples of how these are used.
Lastly, you can also use the “avro-tools” jar which ships with an Avro release. Just use the “compile (schema|protocol)” command.
How are Strings represented in Java?
They use org.apache.avro.util.Utf8, not java.lang.String.
More generally, how do Avro types map to Java types?
The mappings are documented in the package javadoc for generic, specific and reflect API.
From Doug Cutting:
HDFS splits files into blocks, and mapreduce runs a map task for each block. When the task starts, it needs to be able to seek into the file to the start of the block process through the block’s end. If the file were, e.g., a gzip file, this would not be possible, since gzip files must be decompressed from the start. One cannot seek into the middle of a gzip file and start decompressing. So Hadoop’s SequenceFile places a marker periodically (~64k) in the file at record and compression boundaries, where processing can be sensibly started. Then, when a map task starts processing an HDFS block, it finds the first marker after the block’s start and continues through the first marker in the next block of the file. This requires a bit of non-local access (~0.1%). Avro’s data file uses the same method as SequenceFile.
Why isn’t every value in Avro nullable?
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 of every value stored and
corresponding computational costs to create this bit on write and
interpret it on read. These costs are wasted when values may not in
fact be null, as is the case in many datasets. In Avro such costs are
only paid when values may actually be null.
Also, allowing values to be null is a well-known source of errors. In
Avro, a value declared as non-null will always be non-null and programs
need not test for null values when processing it nor will they ever fail
for lack of such tests.
Tony Hoare calls his invention of null references his “Billion Dollar
Mistake”.
http://qconlondon.com/london-2009/presentation/Null+References:+The+Billion+Dollar+Mistake
Also note that in some programming languages not all values are permitted to be null. For example, in Java, values of type boolean, byte, short, char, int, float, long, and double may not be null.
How can I serialize directly to/from a byte array?
As pointed out in the specification, Avro data should always be stored with its schema. The Avro provided classes DataFileWriter, DataFileReader, and DataFileStream all ensure this by serializing the Schema in a container header. In some special cases, such as when implementing a new storage system or writing unit tests, you may need to write and read directly with the bare Avro serialized values. The following examples use code generated for Java from an Avro Schema, specifically the User example from the Quickstart guide.
Serializing to a byte array
This example takes a User object and returns a newly allocated byte array with the Avro serialization of that user.
ByteArrayOutputStream out = new ByteArrayOutputStream();
BinaryEncoder encoder = EncoderFactory.get().binaryEncoder(out, null);
DatumWriter<User> writer = new SpecificDatumWriter<User>(User.getClassSchema());
writer.write(user, encoder);
encoder.flush();
out.close();
byte[] serializedBytes = out.toByteArray();
Deserializing from a byte array
This example takes a byte array containing the Avro serialization of a user and returns a User object.SpecificDatumReader reader = new SpecificDatumReader(User.getClassSchema());
Decoder decoder = DecoderFactory.get().binaryDecoder(bytes, null);
User user = reader.read(null, decoder);
Note: If you are serializing or deserializing in a loop or as a method, you should be reusing objects, readers and/or writers. Check the JavaDoc to see how to reuse the objects.
36 - Apache Avro™ 1.1.0 Documentation
Introduction
Avro is a data serialization system.
Avro provides:
- Rich data structures.
- A compact, fast, binary data format.
- A container file, to store persistent data.
- Remote procedure call (RPC).
- Simple integration with dynamic languages. Code generation is not required to read or write data files nor to use or implement RPC protocols. Code generation as an optional optimization, only worth implementing for statically typed languages.
Schemas
Avro relies on schemas. When Avro data is read, the schema used when writing it is always present. This permits each datum to be written with no per-value overheads, making serialization both fast and small. This also facilitates use with dynamic, scripting languages, since data, together with its schema, is fully self-describing.
When Avro data is stored in a file, its schema is stored with it, so that files may be processed later by any program. If the program reading the data expects a different schema this can be easily resolved, since both schemas are present.
When Avro is used in RPC, the client and server exchange schemas in the connection handshake. (This can be optimized so that, for most calls, no schemas are actually transmitted.) Since both client and server both have the other’s full schema, correspondence between same named fields, missing fields, extra fields, etc. can all be easily resolved.
Avro schemas are defined with with JSON . This facilitates implementation in languages that already have JSON libraries.
Comparison with other systems
Avro provides functionality similar to systems such as Thrift, Protocol Buffers, etc. Avro differs from these systems in the following fundamental aspects.
- Dynamic typing: Avro does not require that code be generated. Data is always accompanied by a schema that permits full processing of that data without code generation, static datatypes, etc. This facilitates construction of generic data-processing systems and languages.
- Untagged data: Since the schema is present when data is read, considerably less type information need be encoded with data, resulting in smaller serialization size.
- No manually-assigned field IDs: When a schema changes, both the old and new schema are always present when processing data, so differences may be resolved symbolically, using field names.
36.1 - Specification
Introduction
This document defines Avro. It is intended to be the authoritative specification. Implementations of Avro must adhere to this document.
Schema Declaration
A Schema is represented in JSON by one of:
- A JSON string, naming a defined type.
- A JSON object, of the form:
{"type": "typeName" ...attributes...}
where typeName is either a primitive or derived type name, as defined below. Attributes not defined in this document are permitted as metadata, but must not affect the format of serialized data.
- A JSON array, representing a union of embedded types.
Primitive Types
The set of primitive type names is:
- string: unicode character sequence
- bytes: sequence of 8-bit bytes
- int: 32-bit signed integer
- long: 64-bit signed integer
- float: single precision (32-bit) IEEE 754 floating-point number
- double: double precision (64-bit) IEEE 754 floating-point number
- boolean: a binary value
- null: no value
Primitive types have no specified attributes.
Primitive type names are also defined type names. Thus, for example, the schema “string” is equivalent to:
Complex Types
Avro supports six kinds of complex types: records, enums, arrays, maps, unions and fixed.
Records
Records use the type name “record” and support two attributes:
- name: a JSON string providing the name of the record (required).
- fields: a JSON array, listing fields (required). Each field is a JSON object with the following attributes:
- name: a JSON string providing the name of the field (required), and
- type: A JSON object defining a schema, or a JSON string naming a record definition (required).
- default: A default value for this field, used when reading instances that lack this field (optional). 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.
field default values
| avro type |
json type |
example |
| string |
string |
"foo" |
| bytes |
string |
"\u00FF" |
| int,long |
integer |
1 |
| float,double |
number |
1.1 |
| boolean |
boolean |
true |
| null |
null |
null |
| record |
object |
{"a": 1} |
| enum |
string |
"FOO" |
| array |
array |
[1] |
| map |
object |
{"a": 1} |
| fixed |
string |
"\u00ff" |
- order: specifies how this field impacts sort ordering of this record (optional). Valid values are “ascending” (the default), “descending”, or “ignore”. For more details on how this is used, see the sort order section below.
For example, a linked-list of 64-bit values may be defined with:
{
"type": "record",
"name": "LongList",
"fields" : [
{"name": "value", "type": "long"}, // each element has a long
{"name": "next", "type": ["LongList", "null"]} // optional next element
]
}
Enums
Enums use the type name “enum” and support the following attributes:
- name: a JSON string providing the name of the enum (required).
- symbols: a JSON array, listing symbols, as JSON strings (required).
For example, playing card suits might be defined with:
{
"type": "enum",
"name": "Suit",
"symbols" : ["SPADES", "HEARTS", "DIAMONDS", "CLUBS"]
}
Arrays
Arrays use the type name “array” and support a single attribute:
- items: the schema of the array’s items.
For example, an array of strings is declared with:
{
"type": "array",
"items" : "string"
}
Maps
Maps use the type name “map” and support one attribute:
- values: the schema of the map’s values.
Map keys are assumed to be strings.
For example, a map from string to long is declared with:
{
"type": "map",
"values" : "long"
}
Unions
Unions, as mentioned above, are represented using JSON arrays. For example, ["string", "null"] declares a schema which may be either a string or null.
Unions may not contain more than one schema with the same type, except for the named types record, fixed and enum. For example, unions containing two array types or two map types are not permitted, but two types with different names are permitted. (Names permit efficient resolution when reading and writing unions.)
Unions may not immediately contain other unions.
Fixed
Fixed uses the type name “fixed” and supports two attributes:
- name: the name of the fixed (required).
- size: an integer, specifying the number of bytes per value (required).
For example, 16-byte quantity may be declared with:
{"type": "fixed", "size": 16, "name": "md5"}
Identifiers
Record, field and enum names must:
- start with [A-Za-z_]
- subsequently contain only [A-Za-z0-9_]
Data Serialization
Avro data is always serialized with its schema. Files that store Avro data should always also include the schema for that data in the same file. Avro-based remote procedure call (RPC) systems must also guarantee that remote recipients of data have a copy of the schema used to write that data.
Because the schema used to write data is always available when the data is read, Avro data itself is not tagged with type information. The schema is required to parse data.
In general, both serialization and deserialization proceed as a depth-first, left-to-right traversal of the schema, serializing primitive types as they are encountered.
Encodings
Avro specifies two serialization encodings: binary and JSON. Most applications will use the binary encoding, as it is smaller and faster. But, for debugging and web-based applications, the JSON encoding may sometimes be appropriate.
Binary Encoding
Primitive Types
Primitive types are encoded in binary as follows:
- a string is encoded as a long followed by that many bytes of UTF-8 encoded character data.
- For example, the three-character string “foo” would be encoded as the long value 3 (encoded as hex 06) followed by the UTF-8 encoding of ‘f’, ‘o’, and ‘o’ (the hex bytes 66 6f 6f):
06 66 6f 6f bytes are encoded as a long followed by that many bytes of data.
- int and long values are written using variable-length zig-zag coding. Some examples:
| value |
hex |
| 0 |
00 |
| -1 |
01 |
| 1 |
02 |
| -2 |
03 |
| 2 |
04 |
| … |
|
| -64 |
7f |
| 64 |
80 01 |
| … |
|
- a float is written as 4 bytes. The float is converted into a 32-bit integer using a method equivalent to Java’s floatToIntBits and then encoded in little-endian format.
- a double is written as 8 bytes. The double is converted into a 64-bit integer using a method equivalent to Java’s doubleToLongBits and then encoded in little-endian format.
- a boolean is written as a single byte whose value is either 0 (false) or 1 (true).
null is written as zero bytes.
Complex Types
Complex types are encoded in binary as follows:
Records
A record is encoded by encoding the values of its fields in the order that they are declared. In other words, a record is encode as just the concatenation of its field’s encodings. Field values are encoded per their schema.
For example, the record schema.
{
"type": "record",
"name": "test",
"fields" : [
{"name": "a", "type": "long"},
{"name": "b", "type": "string"}
]
}
An instance of this record whose a field has value 27 (encoded as hex 36) and whose b field has value “foo” (encoded as hex bytes 06 66 6f 6f), would be encoded simply as the concatenation of these, namely the hex byte sequence:
36 0C 66 6f 6f
Enums
An enum is encoded by a int, representing the zero-based position of the symbol in the schema.
For example, consider the enum:
{"type": "enum", "name": "Foo", "symbols": ["A", "B", "C", "D"] }
This would be encoded by an int between zero and three, with zero indicating “A”, and 3 indicating “D”.
Arrays
Arrays are encoded as a series of blocks. Each block consists of a long count value, followed by that many array items. A block with count zero indicates the end of the array. Each item is encoded per the array’s item schema.
If a block’s count is negative, then the count is followed immediately by a long block size, indicating the number of bytes in the block. The actual count in this case is the absolute value of the count written.
For example, the array schema
{"type": "array", "items": "long"}
an array containing the items 3 and 27 could be encoded as the long value 2 (encoded as hex 04) followed by long values 3 and 27 (encoded as hex 06 36) terminated by zero:
04 06 36 00
The blocked representation permits one to read and write arrays larger than can be buffered in memory, since one can start writing items without knowing the full length of the array. The optional block sizes permit fast skipping through data, e.g., when projecting a record to a subset of its fields.
Maps
Maps are encoded as a series of blocks. Each block consists of a long count value, followed by that many key/value pairs. A block with count zero indicates the end of the map. Each item is encoded per the map’s value schema.
If a block’s count is negative, then the count is followed immediately by a long block size, indicating the number of bytes in the block. The actual count in this case is the absolute value of the count written.
The blocked representation permits one to read and write maps larger than can be buffered in memory, since one can start writing items without knowing the full length of the map. The optional block sizes permit fast skipping through data, e.g., when projecting a record to a subset of its fields.
NOTE: Blocking has not yet been fully implemented and may change. Arbitrarily large objects must be easily writable and readable but until we have proven this with an implementation and tests this part of the specification should be considered draft.
Unions
A union is encoded by first writing a long value indicating the zero-based position within the union of the schema of its value. The value is then encoded per the indicated schema within the union.
For example, the union schema ["string","null"] would encode:
-
null as the integer 1 (the index of “null” in the union, encoded as hex 02):
02
-
the string “a” as zero (the index of “string” in the union), followed by the serialized string:
00 02 61
Fixed
Fixed instances are encoded using the number of bytes declared in the schema.
JSON Encoding
Except for unions, the JSON encoding is the same as is used to encode field default values.
The value of a union is encoded in JSON as follows:
- if its type is null, then it is encoded as a JSON null;
- otherwise it is encoded as a JSON object with one name/value pair whose name is the type’s name and whose value is the recursively encoded value. For Avro’s named types (record, fixed or enum) the user-specified name is used, for other types the type name is used.
For example, the union schema ["null","string","Foo"], where Foo is a record name, would encode:
- null as null;
- the string “a” as
{"string": "a"} and
- a Foo instance as
{"Foo": {...}}, where {...} indicates the JSON encoding of a Foo instance.
Note that a schema is still required to correctly process JSON-encoded data. For example, the JSON encoding does not distinguish between int and long, float and double, records and maps, enums and strings, etc.
Sort Order
Avro defines a standard sort order for data. This permits data written by one system to be efficiently sorted by another system. This can be an important optimization, as sort order comparisons are sometimes the most frequent per-object operation. Note also that Avro binary-encoded data can be efficiently ordered without deserializing it to objects.
Data items may only be compared if they have identical schemas. Pairwise comparisons are implemented recursively with a depth-first, left-to-right traversal of the schema. The first mismatch encountered determines the order of the items.
Two items with the same schema are compared according to the following rules.
- int, long, float and double data is ordered by ascending numeric value.
- boolean data is ordered with false before true.
- null data is always equal.
- string data is compared lexicographically. Note that since UTF-8 is used as the binary encoding of strings, sorting by bytes and characters is equivalent.
- bytes and fixed data are compared lexicographically by byte.
- array data is compared lexicographically by element.
- enum data is ordered by the symbol’s position in the enum schema. For example, an enum whose symbols are
["z", "a"] would sort “z” values before “a” values.
- union data is first ordered by the branch within the union, and, within that, by the type of the branch. For example, an
["int", "string"] union would order all int values before all string values, with the ints and strings themselves ordered as defined above.
- record data is ordered lexicographically by field. If a field specifies that its order is:
- “ascending”, then the order of its values is unaltered.
- “descending”, then the order of its values is reversed.
- “ignore”, then its values are ignored when sorting.
- map data may not be compared. It is an error to attempt to compare data containing maps unless those maps are in an
"order":"ignore" record field.
Object Container Files
Avro includes a simple object container file format. A file has a schema, and all objects stored in the file must be written according to that schema. Objects are stored in blocks that may be compressed. Syncronization markers are used between blocks to permit efficient splitting of files for MapReduce processing.
Files may include arbitrary user-specified metadata.
A file consists of:
- A header, followed by
- one or more blocks.
There are two kinds of blocks, normal and metadata. All files must contain at least one metadata block. A file terminates with its last metadata block. Any data after the last metadata block is ignored.
A header consists of:
- Four bytes, ASCII ‘O’, ‘b’, ‘j’, followed by zero.
- A 16-byte sync marker.
A metadata block consists of:
The file’s 16-byte sync marker.
A long with value -1, identifying this as a metadata block.
A long indicating the size in bytes of this block.
A long indicating the number of metadata key/value pairs.
For each pair, a string key and bytes value.
The size in bytes of this block as a 4-byte big-endian integer.
When a file is closed normally, this terminates the file and permits one to efficiently seek to the start of the metadata. If the sync marker there does not match that at the start of the file, then one must scan for the last metadata in the file.
The following metadata properties are reserved:
- schema contains the schema of objects stored in the file, as a string.
- count contains the number of objects in the file as a decimal ASCII string.
- codec the name of the compression codec used to compress blocks.
A normal block consists of:
- The file’s 16-byte sync marker.
- A long indicating the size in bytes of this block in the file.
- The serialized objects. If a codec is specified, this is compressed by that codec.
- Note that this format supports appends, since multiple metadata blocks are permitted.
To be robust to application failure, implementations can write metadata periodically to limit the amount of the file that must be scanned to find the last metadata block.
Protocol Declaration
Avro protocols describe RPC interfaces. Like schemas, they are defined with JSON text.
A protocol is a JSON object with the following attributes:
- name, string, to distinguish it from other protocols;
- namespace, a string which qualifies the name;
- types, a list of record, enum and error definitions. An error definition is just like a record definition except it uses “error” instead of “record”. Note that forward references to records, enums and errors are not currently supported.
- messages, a JSON object whose keys are message names and whose values are objects whose attributes are described below. No two messages may have the same name.
Messages
A message has attributes:
- a request, a list of named, typed parameter schemas (this has the same form as the fields of a record declaration);
- a response schema; and
- an optional union of error schemas.
A request parameter list is processed equivalently to an anonymous record. Since record field lists may vary between reader and writer, request parameters may also differ between the caller and responder, and such differences are resolved in the same manner as record field differences.
Sample Protocol
For example, one may define a simple HelloWorld protocol with:
{
"namespace": "com.acme",
"protocol": "HelloWorld",
"types": [
{"name": "Greeting", "type": "record", "fields": [
{"name": "message", "type": "string"}]},
{"name": "Curse", "type": "error", "fields": [
{"name": "message", "type": "string"}]}
],
"messages": {
"hello": {
"request": {"greeting": "Greeting" },
"response": "Greeting",
"errors": ["Curse"]
}
}
}
Message Transport
Messages may be transmitted via different transport mechanisms. For example, one might use the HTTP, raw sockets, or SSL, etc. This document specifies formats for request and response message data, but it does not yet specify any details of how message data is encapsulated in different transports.
To the transport, a message is an opaque byte sequence.
A transport is a system that supports:
session creation
A session forms the context under which multiple messages may be transcieved. A client must establish a session with a server before any requests may be processed.
- transmission of request messages
Once a session has been established, clients may send servers request messages using that session.
- receipt of corresponding response messages
Servers will send a response message back to the client corresponding to each request message. The mechanism of that correspondance is transport-specific. For example, in HTTP it might be implicit, since HTTP directly supports requests and responses. But a transport that multiplexes many client threads over a single socket would need to tag messages with unique identifiers.
Message Framing
Avro messages are framed as a list of buffers.
Framing is a layer between messages and the transport. It exists to optimize certain operations.
The format of framed message data is:
- a series of buffers, where each buffer consists of:
- a four-byte, big-endian buffer length, followed by
- that many bytes of buffer data.
- A message is always terminated by a zero-lenghted buffer.
Framing is transparent to request and response message formats (described below). Any message may be presented as a single or multiple buffers.
Framing can permit readers to more efficiently get different buffers from different sources and for writers to more efficiently store different buffers to different destinations. In particular, it can reduce the number of times large binary objects are copied. For example, if an RPC parameter consists of a megabyte of file data, that data can be copied directly to a socket from a file descriptor, and, on the other end, it could be written directly to a file descriptor, never entering user space.
A simple, recommended, framing policy is for writers to create a new segment whenever a single binary object is written that is larger than a normal output buffer. Small objects are then appended in buffers, while larger objects are written as their own buffers. When a reader then tries to read a large object the runtime can hand it an entire buffer directly, without having to copy it.
Handshake
RPC sessions are initiated by handshake. The purpose of the handshake is to ensure that the client and the server have each other’s protocol definition, so that the client can correctly deserialize responses, and the server can correctly deserialize requests. Both clients and servers should maintain a cache of recently seen protocols, so that, in most cases, a handshake will be completed without extra round-trip network exchanges or the transmission of full protocol text.
The handshake process uses the following record schemas:
{
"type": "record",
"name": "HandshakeRequest", "namespace":"org.apache.avro.ipc",
"fields": [
{"name": "clientHash",
"type": {"type": "fixed", "name": "MD5", "size": 16}},
{"name": "clientProtocol", "type": ["null", "string"]},
{"name": "serverHash", "type": "MD5"},
{"name": "meta", "type": ["null", {"type": "map", "values": "bytes"}]}
]
}
{
"type": "record",
"name": "HandshakeResponse", "namespace": "org.apache.avro.ipc",
"fields": [
{"name": "match",
"type": {"type": "enum", "name": "HandshakeMatch",
"symbols": ["BOTH", "CLIENT", "NONE"]}},
{"name": "serverProtocol",
"type": ["null", "string"]},
{"name": "serverHash",
"type": ["null", {"type": "fixed", "name": "MD5", "size": 16}]},
{"name": "meta",
"type": ["null", {"type": "map", "values": "bytes"}]}
]
}
- In a new session, a client first sends a HandshakeRequest containing just the hash of its protocol and of the server’s protocol (clientHash!=null, clientProtocol=null, serverHash!=null), where the hashes are 128-bit MD5 hashes of the JSON protocol text. If a client has never connected to a given server, it sends its hash as a guess of the server’s hash, otherwise it sends the hash that it previously obtained from this server.
The server responds with a HandshakeResponse containing one of:
-
match=BOTH, serverProtocol=null, serverHash=null, if the client sent the valid hash of the server’s protocol and the server knows what protocol corresponds to the client’s hash. In this case, the request is complete and the session is established.
-
match=CLIENT, serverProtocol!=null, serverHash!=null if the server has previously seen the client’s protocol, but the client sent an incorrect hash of the server’s protocol. The client must then re-send the request with the correct server hash.
-
match=NONE serverProtocol!=null, serverHash!=null if the server has not previously seen the client’s protocol and the client sent and incorrect hash of the server’s protocol.
In this case The client must then re-submit its request with its protocol text (clientHash!=null, clientProtocol!=null, serverHash!=null) and the server should respond with with a successful match (match=BOTH, serverProtocol=null, serverHash=null) as above.
Until a connection is established, call request data sent by the client must be preceded by a HandshakeRequest and call response data returned by the server must be preceded by a HandshakeResponse. A connection is not established until a HandshakeResponse with match=BOTH or match=CLIENT is returned. In these cases, the call response data immmediately follows the HandShakeResponse. When match=NONE no response call data is sent and the request call data is ignored.
The meta field is reserved for future handshake enhancements.
A call consists of a request message paired with its resulting response or error message. Requests and responses contain extensible metadata, and both kinds of messages are framed as described above.
The format of a call request is:
- request metadata, a map with values of type bytes
- the message name, an Avro string, followed by
- the message parameters. Parameters are serialized according to the message’s request declaration.
The format of a call response is:
- response metadata, a map with values of type bytes
- a one-byte error flag boolean, followed by either:
- if the error flag is false, the message response, serialized per the message’s response schema.
- if the error flag is true, the error, serialized per the message’s error union schema.
Schema Resolution
A reader of Avro data, whether from an RPC or a file, can always parse that data because its schema is provided. But that schema may not be exactly the schema that was expected. For example, if the data was written with a different version of the software than it is read, then records may have had fields added or removed. This section specifies how such schema differences should be resolved.
We call the schema used to write the data as the writer’s schema, and the schema that the application expects the reader’s schema. Differences between these should be resolved as follows:
- It is an error if the two schemas do not match.
To match, one of the following must hold:
-
both schemas are arrays whose item types match
-
both schemas are maps whose value types match
-
both schemas are enums whose names match
-
both schemas are fixed whose sizes and names match
-
both schemas are records with the same name
-
either schema is a union
-
both schemas have same primitive type
-
the writer’s schema may be promoted to the reader’s as follows:
- int is promotable to long, float, or double
- long is promotable to float or double
- float is promotable to double
-
if both are records:
- the ordering of fields may be different: fields are matched by name.
- schemas for fields with the same name in both records are resolved recursively.
- if the writer’s record contains a field with a name not present in the reader’s record, the writer’s value for that field is ignored.
- if the reader’s record schema has a field that contains a default value, and writer’s schema does not have a field with the same name, then the reader should use the default value from its field.
- if the reader’s record schema has a field with no default value, and writer’s schema does not have a field with the same name, then the field’s value is unset.
-
if both are enums:
if the writer’s symbol is not present in the reader’s enum, then the enum’s value is unset.
This resolution algorithm is applied recursively to the reader’s and writer’s array item schemas.
This resolution algorithm is applied recursively to the reader’s and writer’s value schemas.
The first schema in the reader’s union that matches the selected writer’s union schema is recursively resolved against it. if none match, an error is signalled.
- if reader’s is a union, but writer’s is not
The first schema in the reader’s union that matches the writer’s schema is recursively resolved against it. If none match, an error is signalled.
- if writer’s is a union, but reader’s is not
If the reader’s schema matches the selected writer’s schema, it is recursively resolved against it. If they do not match, an error is signalled.
36.2 - Wiki
The Wiki page can be found here.
36.3 - FAQ
What is Avro?
Avro is a data serialization system.
Avro provides:
- Rich data structures.
- A compact, fast, binary data format.
- A container file, to store persistent data.
- Remote procedure call (RPC).
- Simple integration with dynamic languages. Code generation is not required to read or write data files nor to use or implement RPC protocols. Code generation as an optional optimization, only worth implementing for statically typed languages.
How can I get started quickly with Avro?
Check out the Quick Start Guide.
How do I statically compile a schema or protocol into generated code?
In Java
- Add the avro jar, the jackson-mapper-asl.jar and jackson-core-asl.jar to your CLASSPATH.
- Run java org.apache.avro.specific.SpecificCompiler .
This appears to be out of date, the SpecificCompiler requires two arguments, presumably an input and and output file, but it isn’t clear that this does.
Or use the Schema or Protocol Ant tasks. Avro’s build.xml provides examples of how these are used.
Lastly, you can also use the “avro-tools” jar which ships with an Avro release. Just use the “compile (schema|protocol)” command.
How are Strings represented in Java?
They use org.apache.avro.util.Utf8, not java.lang.String.
More generally, how do Avro types map to Java types?
The mappings are documented in the package javadoc for generic, specific and reflect API.
From Doug Cutting:
HDFS splits files into blocks, and mapreduce runs a map task for each block. When the task starts, it needs to be able to seek into the file to the start of the block process through the block’s end. If the file were, e.g., a gzip file, this would not be possible, since gzip files must be decompressed from the start. One cannot seek into the middle of a gzip file and start decompressing. So Hadoop’s SequenceFile places a marker periodically (~64k) in the file at record and compression boundaries, where processing can be sensibly started. Then, when a map task starts processing an HDFS block, it finds the first marker after the block’s start and continues through the first marker in the next block of the file. This requires a bit of non-local access (~0.1%). Avro’s data file uses the same method as SequenceFile.
Why isn’t every value in Avro nullable?
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 of every value stored and
corresponding computational costs to create this bit on write and
interpret it on read. These costs are wasted when values may not in
fact be null, as is the case in many datasets. In Avro such costs are
only paid when values may actually be null.
Also, allowing values to be null is a well-known source of errors. In
Avro, a value declared as non-null will always be non-null and programs
need not test for null values when processing it nor will they ever fail
for lack of such tests.
Tony Hoare calls his invention of null references his “Billion Dollar
Mistake”.
http://qconlondon.com/london-2009/presentation/Null+References:+The+Billion+Dollar+Mistake
Also note that in some programming languages not all values are permitted to be null. For example, in Java, values of type boolean, byte, short, char, int, float, long, and double may not be null.
How can I serialize directly to/from a byte array?
As pointed out in the specification, Avro data should always be stored with its schema. The Avro provided classes DataFileWriter, DataFileReader, and DataFileStream all ensure this by serializing the Schema in a container header. In some special cases, such as when implementing a new storage system or writing unit tests, you may need to write and read directly with the bare Avro serialized values. The following examples use code generated for Java from an Avro Schema, specifically the User example from the Quickstart guide.
Serializing to a byte array
This example takes a User object and returns a newly allocated byte array with the Avro serialization of that user.
ByteArrayOutputStream out = new ByteArrayOutputStream();
BinaryEncoder encoder = EncoderFactory.get().binaryEncoder(out, null);
DatumWriter<User> writer = new SpecificDatumWriter<User>(User.getClassSchema());
writer.write(user, encoder);
encoder.flush();
out.close();
byte[] serializedBytes = out.toByteArray();
Deserializing from a byte array
This example takes a byte array containing the Avro serialization of a user and returns a User object.SpecificDatumReader reader = new SpecificDatumReader(User.getClassSchema());
Decoder decoder = DecoderFactory.get().binaryDecoder(bytes, null);
User user = reader.read(null, decoder);
Note: If you are serializing or deserializing in a loop or as a method, you should be reusing objects, readers and/or writers. Check the JavaDoc to see how to reuse the objects.
37 - Apache Avro™ 1.0.0 Documentation
Introduction
Avro is a data serialization system.
Avro provides:
- Rich data structures.
- A compact, fast, binary data format.
- A container file, to store persistent data.
- Remote procedure call (RPC).
- Simple integration with dynamic languages. Code generation is not required to read or write data files nor to use or implement RPC protocols. Code generation as an optional optimization, only worth implementing for statically typed languages.
Schemas
Avro relies on schemas. When Avro data is read, the schema used when writing it is always present. This permits each datum to be written with no per-value overheads, making serialization both fast and small. This also facilitates use with dynamic, scripting languages, since data, together with its schema, is fully self-describing.
When Avro data is stored in a file, its schema is stored with it, so that files may be processed later by any program. If the program reading the data expects a different schema this can be easily resolved, since both schemas are present.
When Avro is used in RPC, the client and server exchange schemas in the connection handshake. (This can be optimized so that, for most calls, no schemas are actually transmitted.) Since both client and server both have the other’s full schema, correspondence between same named fields, missing fields, extra fields, etc. can all be easily resolved.
Avro schemas are defined with with JSON . This facilitates implementation in languages that already have JSON libraries.
Comparison with other systems
Avro provides functionality similar to systems such as Thrift, Protocol Buffers, etc. Avro differs from these systems in the following fundamental aspects.
- Dynamic typing: Avro does not require that code be generated. Data is always accompanied by a schema that permits full processing of that data without code generation, static datatypes, etc. This facilitates construction of generic data-processing systems and languages.
- Untagged data: Since the schema is present when data is read, considerably less type information need be encoded with data, resulting in smaller serialization size.
- No manually-assigned field IDs: When a schema changes, both the old and new schema are always present when processing data, so differences may be resolved symbolically, using field names.
37.1 - Specification
Introduction
This document defines Avro. It is intended to be the authoritative specification. Implementations of Avro must adhere to this document.
Schema Declaration
A Schema is represented in JSON by one of:
- A JSON string, naming a defined type.
- A JSON object, of the form:
{"type": "typeName" ...attributes...}
where typeName is either a primitive or derived type name, as defined below. Attributes not defined in this document are permitted as metadata, but must not affect the format of serialized data.
- A JSON array, representing a union of embedded types.
Primitive Types
The set of primitive type names is:
- string: unicode character sequence
- bytes: sequence of 8-bit bytes
- int: 32-bit signed integer
- long: 64-bit signed integer
- float: 32-bit IEEE floating-point number
- double: 64-bit IEEE floating-point number
- boolean: a binary value
- null: no value
Primitive types have no specified attributes.
Primitive type names are also defined type names. Thus, for example, the schema “string” is equivalent to:
Complex Types
Avro supports six kinds of complex types: records, enums, arrays, maps, unions and fixed.
Records
Records use the type name “record” and support two attributes:
- name: a JSON string providing the name of the record (required).
- fields: a JSON array, listing fields (required). Each field is a JSON object with the following attributes:
- name: a JSON string providing the name of the field (required), and
- type: A JSON object defining a schema, or a JSON string naming a record definition (required).
- default: A default value for this field, used when reading instances that lack this field (optional). Permitted values depend on the field’s schema type, according to the table below. Default values for union fields correspond the first schema in the union.
field default values
| avro type |
json type |
example |
| string |
string |
"foo" |
| bytes |
string |
"\u00FF" |
| int,long |
integer |
1 |
| float,double |
number |
1.1 |
| boolean |
boolean |
true |
| null |
null |
null |
| record |
object |
{"a": 1} |
| enum |
string |
"FOO" |
| array |
array |
[1] |
| map |
object |
{"a": 1} |
For example, a linked-list of 64-bit values may be defined with:
{
"type": "record",
"name": "LongList",
"fields" : [
{"name": "value", "type": "long"}, // each element has a long
{"name": "next", "type": ["LongList", "null"]} // optional next element
]
}
Enums
Enums use the type name “enum” and support the following attributes:
- name: a JSON string providing the name of the enum (required).
- symbols: a JSON array, listing symbols, as JSON strings (required).
For example, playing card suits might be defined with:
{
"type": "enum",
"name": "Suit",
"symbols" : ["SPADES", "HEARTS", "DIAMONDS", "CLUBS"]
}
Arrays
Arrays use the type name “array” and support a single attribute:
- items: the schema of the array’s items.
For example, an array of strings is declared with:
{
"type": "array",
"items" : "string"
}
Maps
Maps use the type name “map” and support one attribute:
- values: the schema of the map’s values.
Map keys are assumed to be strings.
For example, a map from string to long is declared with:
{
"type": "map",
"values" : "long"
}
Unions
Unions, as mentioned above, are represented using JSON arrays. For example, ["string", "null"] declares a schema which may be either a string or null.
Unions may not contain more than one schema with the same type, except for the named types record, fixed and enum. For example, unions containing two array types or two map types are not permitted, but two types with different names are permitted. (Names permit efficient resolution when reading and writing unions.)
Unions may not immediately contain other unions.
Fixed
Fixed uses the type name “fixed” and supports two attributes:
- name: the name of the fixed (required).
- size: an integer, specifying the number of bytes per value (required).
For example, 16-byte quantity may be declared with:
{"type": "fixed", "size": 16, "name": "md5"}
Identifiers
Record, field and enum names must:
- start with [A-Za-z_]
- subsequently contain only [A-Za-z0-9_]
Data Serialization
Avro data is always serialized with its schema. Files that store Avro data should always also include the schema for that data in the same file. Avro-based remote procedure call (RPC) systems must also guarantee that remote recipients of data have a copy of the schema used to write that data.
Because the schema used to write data is always available when the data is read, Avro data itself is not tagged with type information. The schema is required to parse data.
In general, both serialization and deserialization proceed as a depth-first, left-to-right traversal of the schema, serializing primitive types as they are encountered.
Primitive Type Serialization
Primitive types are serialized as follows:
- a string is serialized as a long followed by that many bytes of UTF-8 encoded character data.
- For example, the three-character string “foo” would be serialized as 3 (encoded as hex 0C) followed by the UTF-8 encoding of ‘f’, ‘o’, and ‘o’ (the hex bytes 66 6f 6f):
0C 66 6f 6f
- bytes are serialized as a long followed by that many bytes of data.
- int and long values are written using variable-length zig-zag coding. Some examples:
| value |
hex |
| 0 |
00 |
| -1 |
01 |
| 1 |
02 |
| -2 |
03 |
| 2 |
04 |
| … |
|
| -64 |
7f |
| 64 |
80 01 |
| … |
|
- a float is written as 4 bytes.
- a double is written as 8 bytes.
- a boolean is written as a single byte whose value is either 0 (false) or 1 (true).
null is written as zero bytes.
Complex Type Serialization
Complex types are serialized as follows:
Records
A record is serialized by serializing the values of its fields in the order that they are declared. In other words, a record is serialized as just the concatenation of its field’s serializations. Field values are serialized per their schema.
For example, the record schema.
{
"type": "record",
"name": "test",
"fields" : [
{"name": "a", "type": "long"},
{"name": "b", "type": "string"}
]
}
An instance of this record whose a field has value 27 (encoded as hex 36) and whose b field has value “foo” (encoded as hex bytes OC 66 6f 6f), would be serialized simply as the concatenation of these, namely the hex byte sequence:
36 0C 66 6f 6f
Enums
An enum is serialized by a int, representing the zero-based position of the symbol in the schema.
For example, consider the enum:
{"type": "enum", "name": "Foo", "symbols": ["A", "B", "C", "D"] }
This would be serialized by an int between zero and three, with zero indicating “A”, and 3 indicating “D”.
Arrays
Arrays are serialized as a series of blocks. Each block consists of a long count value, followed by that many array items. A block with count zero indicates the end of the array. Each item is serialized per the array’s item schema.
If a block’s count is negative, then the count is followed immediately by a long block size, indicating the number of bytes in the block. The actual count in this case is the absolute value of the count written.
For example, the array schema
{"type": "array", "items": "long"}
serializing an array containing the items 3 and 27 could be serialized as 2 (encoded as hex 04) followed by 3 and 27 (encoded as hex 06 36) terminated by zero:
04 06 36 00
The blocked representation permits one to read and write arrays larger than can be buffered in memory, since one can start writing items without knowing the full length of the array. The optional block sizes permit fast skipping through data, e.g., when projecting a record to a subset of its fields.
NOTE: Blocking has not yet been fully implemented and may change. Arbitrarily large objects must be easily writable and readable but until we have proven this with an implementation and tests this part of the specification should be considered draft.
Maps
Maps are serialized as a series of blocks. Each block consists of a long count value, followed by that many key/value pairs. A block with count zero indicates the end of the map. Each item is serialized per the map’s value schema.
If a block’s count is negative, then the count is followed immediately by a long block size, indicating the number of bytes in the block. The actual count in this case is the absolute value of the count written.
The blocked representation permits one to read and write maps larger than can be buffered in memory, since one can start writing items without knowing the full length of the map. The optional block sizes permit fast skipping through data, e.g., when projecting a record to a subset of its fields.
NOTE: Blocking has not yet been fully implemented and may change. Arbitrarily large objects must be easily writable and readable but until we have proven this with an implementation and tests this part of the specification should be considered draft.
Unions
A union is serialized by first writing a long value indicating the zero-based position within the union of the schema of its value. The value is then serialized per the indicated schema within the union.
For example, the union schema ["string","null"] would serialize:
-
null as 1 (the index of “null” in the union, encoded as hex 02):
02
-
the string “a” as zero (the index of “string” in the union), followed by the serialized string:
00 02 61
Fixed
Fixed instances are serialized using the number of bytes declared in the schema.
Object Container Files
Avro includes a simple object container file format. A file has a schema, and all objects stored in the file must be written according to that schema. Objects are stored in blocks that may be compressed. Syncronization markers are used between blocks to permit efficient splitting of files for MapReduce processing.
Files may include arbitrary user-specified metadata.
A file consists of:
- A header, followed by
- one or more blocks.
There are two kinds of blocks, normal and metadata. All files must contain at least one metadata block. A file terminates with its last metadata block. Any data after the last metadata block is ignored.
A header consists of:
- Four bytes, ASCII ‘O’, ‘b’, ‘j’, followed by zero.
- A 16-byte sync marker.
A metadata block consists of:
The file’s 16-byte sync marker.
A long with value -1, identifying this as a metadata block.
A long indicating the size in bytes of this block.
A long indicating the number of metadata key/value pairs.
For each pair, a string key and bytes value.
The size in bytes of this block as a 4-byte big-endian integer.
When a file is closed normally, this terminates the file and permits one to efficiently seek to the start of the metadata. If the sync marker there does not match that at the start of the file, then one must scan for the last metadata in the file.
The following metadata properties are reserved:
- schema contains the schema of objects stored in the file, as a string.
- count contains the number of objects in the file as a decimal ASCII string.
- codec the name of the compression codec used to compress blocks.
A normal block consists of:
- The file’s 16-byte sync marker.
- A long indicating the size in bytes of this block in the file.
- The serialized objects. If a codec is specified, this is compressed by that codec.
- Note that this format supports appends, since multiple metadata blocks are permitted.
To be robust to application failure, implementations can write metadata periodically to limit the amount of the file that must be scanned to find the last metadata block.
Protocol Declaration
Avro protocols describe RPC interfaces. Like schemas, they are defined with JSON text.
A protocol is a JSON object with the following attributes:
- name, string, to distinguish it from other protocols;
- namespace, a string which qualifies the name;
- types, a list of record, enum and error definitions. An error definition is just like a record definition except it uses “error” instead of “record”. Note that forward references to records, enums and errors are not currently supported.
- messages, a JSON object whose keys are message names and whose values are objects whose attributes are described below. No two messages may have the same name.
Messages
A message has attributes:
- a request, a list of named, typed parameter schemas (this has the same form as the fields of a record declaration);
- a response schema; and
- an optional union of error schemas.
Sample Protocol
For example, one may define a simple HelloWorld protocol with:
{
"namespace": "com.acme",
"protocol": "HelloWorld",
"types": [
{"name": "Greeting", "type": "record", "fields": [
{"name": "message", "type": "string"}]}
{"name": "Curse", "type": "error", "fields": [
{"name": "message", "type": "string"}]}
],
"messages": {
"hello": {
"request": {"greeting": "Greeting" },
"response": "Greeting",
"errors": ["Curse"]
}
}
}
Message Transport
Messages may be transmitted via different transport mechanisms. For example, one might use the HTTP, raw sockets, or SSL, etc. This document specifies formats for request and response message data, but it does not yet specify any details of how message data is encapsulated in different transports.
To the transport, a message is an opaque byte sequence.
A transport is a system that supports:
session creation
A session forms the context under which multiple messages may be transcieved. A client must establish a session with a server before any requests may be processed.
- transmission of request messages
Once a session has been established, clients may send servers request messages using that session.
- receipt of corresponding response messages
Servers will send a response message back to the client corresponding to each request message. The mechanism of that correspondance is transport-specific. For example, in HTTP it might be implicit, since HTTP directly supports requests and responses. But a transport that multiplexes many client threads over a single socket would need to tag messages with unique identifiers.
Message Framing
Avro messages are framed as a list of buffers.
Framing is a layer between messages and the transport. It exists to optimize certain operations.
The format of framed message data is:
- a series of buffers, where each buffer consists of:
- a four-byte, big-endian buffer length, followed by
- that many bytes of buffer data.
- A message is always terminated by a zero-lenghted buffer.
Framing is transparent to request and response message formats (described below). Any message may be presented as a single or multiple buffers.
Framing can permit readers to more efficiently get different buffers from different sources and for writers to more efficiently store different buffers to different destinations. In particular, it can reduce the number of times large binary objects are copied. For example, if an RPC parameter consists of a megabyte of file data, that data can be copied directly to a socket from a file descriptor, and, on the other end, it could be written directly to a file descriptor, never entering user space.
A simple, recommended, framing policy is for writers to create a new segment whenever a single binary object is written that is larger than a normal output buffer. Small objects are then appended in buffers, while larger objects are written as their own buffers. When a reader then tries to read a large object the runtime can hand it an entire buffer directly, without having to copy it.
Handshake
RPC sessions are initiated by handshake. The purpose of the handshake is to ensure that the client and the server have each other’s protocol definition, so that the client can correctly deserialize responses, and the server can correctly deserialize requests. Both clients and servers should maintain a cache of recently seen protocols, so that, in most cases, a handshake will be completed without extra round-trip network exchanges or the transmission of full protocol text.
The handshake process uses the following record schemas:
{
"type": "record",
"name": "HandshakeRequest", "namespace":"org.apache.avro.ipc",
"fields": [
{"name": "clientHash",
"type": {"type": "fixed", "name": "MD5", "size": 16}},
{"name": "clientProtocol", "type": ["null", "string"]},
{"name": "serverHash", "type": "MD5"},
{"name": "meta", "type": ["null", {"type": "map", "values": "bytes"}]}
]
}
{
"type": "record",
"name": "HandshakeResponse", "namespace": "org.apache.avro.ipc",
"fields": [
{"name": "match",
"type": {"type": "enum", "name": "HandshakeMatch",
"symbols": ["BOTH", "CLIENT", "NONE"]}},
{"name": "serverProtocol",
"type": ["null", "string"]},
{"name": "serverHash",
"type": ["null", {"type": "fixed", "name": "MD5", "size": 16}]},
{"name": "meta",
"type": ["null", {"type": "map", "values": "bytes"}]}
]
}
- In a new session, a client first sends a HandshakeRequest containing just the hash of its protocol and of the server’s protocol (clientHash!=null, clientProtocol=null, serverHash!=null), where the hashes are 128-bit MD5 hashes of the JSON protocol text. If a client has never connected to a given server, it sends its hash as a guess of the server’s hash, otherwise it sends the hash that it previously obtained from this server.
The server responds with a HandshakeResponse containing one of:
-
match=BOTH, serverProtocol=null, serverHash=null, if the client sent the valid hash of the server’s protocol and the server knows what protocol corresponds to the client’s hash. In this case, the request is complete and the session is established.
-
match=CLIENT, serverProtocol!=null, serverHash!=null if the server has previously seen the client’s protocol, but the client sent an incorrect hash of the server’s protocol. The client must then re-send the request with the correct server hash.
-
match=NONE serverProtocol!=null, serverHash!=null if the server has not previously seen the client’s protocol and the client sent and incorrect hash of the server’s protocol.
In this case The client must then re-submit its request with its protocol text (clientHash!=null, clientProtocol!=null, serverHash!=null) and the server should respond with with a successful match (match=BOTH, serverProtocol=null, serverHash=null) as above.
Until a connection is established, call request data sent by the client must be preceded by a HandshakeRequest and call response data returned by the server must be preceded by a HandshakeResponse. A connection is not established until a HandshakeResponse with match=BOTH or match=CLIENT is returned. In these cases, the call response data immmediately follows the HandShakeResponse. When match=NONE no response call data is sent and the request call data is ignored.
The meta field is reserved for future handshake enhancements.
A call consists of a request message paired with its resulting response or error message. Requests and responses contain extensible metadata, and both kinds of messages are framed as described above.
The format of a call request is:
- request metadata, a map with values of type bytes
- the message name, an Avro string, followed by
- the message parameters. Parameters are serialized according to the message’s request declaration.
The format of a call response is:
- response metadata, a map with values of type bytes
- a one-byte error flag boolean, followed by either:
- if the error flag is false, the message response, serialized per the message’s response schema.
- if the error flag is true, the error, serialized per the message’s error union schema.
Schema Resolution
A reader of Avro data, whether from an RPC or a file, can always parse that data because its schema is provided. But that schema may not be exactly the schema that was expected. For example, if the data was written with a different version of the software than it is read, then records may have had fields added or removed. This section specifies how such schema differences may be resolved.
We call the schema used to write the data as the writer’s schema, and the schema that the application expects the reader’s schema. To resolve differences between these two schemas, the following resolution algorithm is recommended.
- It is an error if the two schemas do not match.
To match, one of the following must hold:
-
both schemas are arrays whose item types match
-
both schemas are maps whose value types match
-
both schemas are enums whose names match
-
both schemas are fixed whose sizes and names match
-
both schemas are records with the same name
-
either schema is a union
-
both schemas have same primitive type
-
the writer’s schema may be promoted to the reader’s as follows:
- int is promotable to long, float, or double
- long is promotable to float or double
- float is promotable to double
-
if both are records:
- if the writer’s record contains a field with a name not present in the reader’s record, that writer’s value is ignored.
- schemas for fields with the same name in both records are resolved recursively.
- Note that method parameter lists are equivalent to records. Note also that, since the ordering of record fields may vary between reader and writer, method parameter list order may also vary.
-
if both are enums:
if the writer’s symbol is not present in the reader’s enum, then the enum’s value is unset.
This resolution algorithm is applied recursively to the reader’s and writer’s array item schemas.
This resolution algorithm is applied recursively to the reader’s and writer’s value schemas.
The first schema in the reader’s union that matches the selected writer’s union schema is recursively resolved against it. if none match, an error is signalled.
- if reader’s is a union, but writer’s is not
The first schema in the reader’s union that matches the writer’s schema is recursively resolved against it. If none match, an error is signalled.
- if writer’s is a union, but reader’s is not
If the reader’s schema matches the selected writer’s schema, it is recursively resolved against it. If they do not match, an error is signalled.
37.2 - Wiki
The Wiki page can be found here.
37.3 - FAQ
What is Avro?
Avro is a data serialization system.
Avro provides:
- Rich data structures.
- A compact, fast, binary data format.
- A container file, to store persistent data.
- Remote procedure call (RPC).
- Simple integration with dynamic languages. Code generation is not required to read or write data files nor to use or implement RPC protocols. Code generation as an optional optimization, only worth implementing for statically typed languages.
How can I get started quickly with Avro?
Check out the Quick Start Guide.
How do I statically compile a schema or protocol into generated code?
In Java
- Add the avro jar, the jackson-mapper-asl.jar and jackson-core-asl.jar to your CLASSPATH.
- Run java org.apache.avro.specific.SpecificCompiler .
This appears to be out of date, the SpecificCompiler requires two arguments, presumably an input and and output file, but it isn’t clear that this does.
Or use the Schema or Protocol Ant tasks. Avro’s build.xml provides examples of how these are used.
Lastly, you can also use the “avro-tools” jar which ships with an Avro release. Just use the “compile (schema|protocol)” command.
How are Strings represented in Java?
They use org.apache.avro.util.Utf8, not java.lang.String.
More generally, how do Avro types map to Java types?
The mappings are documented in the package javadoc for generic, specific and reflect API.
From Doug Cutting:
HDFS splits files into blocks, and mapreduce runs a map task for each block. When the task starts, it needs to be able to seek into the file to the start of the block process through the block’s end. If the file were, e.g., a gzip file, this would not be possible, since gzip files must be decompressed from the start. One cannot seek into the middle of a gzip file and start decompressing. So Hadoop’s SequenceFile places a marker periodically (~64k) in the file at record and compression boundaries, where processing can be sensibly started. Then, when a map task starts processing an HDFS block, it finds the first marker after the block’s start and continues through the first marker in the next block of the file. This requires a bit of non-local access (~0.1%). Avro’s data file uses the same method as SequenceFile.
Why isn’t every value in Avro nullable?
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 of every value stored and
corresponding computational costs to create this bit on write and
interpret it on read. These costs are wasted when values may not in
fact be null, as is the case in many datasets. In Avro such costs are
only paid when values may actually be null.
Also, allowing values to be null is a well-known source of errors. In
Avro, a value declared as non-null will always be non-null and programs
need not test for null values when processing it nor will they ever fail
for lack of such tests.
Tony Hoare calls his invention of null references his “Billion Dollar
Mistake”.
http://qconlondon.com/london-2009/presentation/Null+References:+The+Billion+Dollar+Mistake
Also note that in some programming languages not all values are permitted to be null. For example, in Java, values of type boolean, byte, short, char, int, float, long, and double may not be null.
How can I serialize directly to/from a byte array?
As pointed out in the specification, Avro data should always be stored with its schema. The Avro provided classes DataFileWriter, DataFileReader, and DataFileStream all ensure this by serializing the Schema in a container header. In some special cases, such as when implementing a new storage system or writing unit tests, you may need to write and read directly with the bare Avro serialized values. The following examples use code generated for Java from an Avro Schema, specifically the User example from the Quickstart guide.
Serializing to a byte array
This example takes a User object and returns a newly allocated byte array with the Avro serialization of that user.
ByteArrayOutputStream out = new ByteArrayOutputStream();
BinaryEncoder encoder = EncoderFactory.get().binaryEncoder(out, null);
DatumWriter<User> writer = new SpecificDatumWriter<User>(User.getClassSchema());
writer.write(user, encoder);
encoder.flush();
out.close();
byte[] serializedBytes = out.toByteArray();
Deserializing from a byte array
This example takes a byte array containing the Avro serialization of a user and returns a User object.SpecificDatumReader reader = new SpecificDatumReader(User.getClassSchema());
Decoder decoder = DecoderFactory.get().binaryDecoder(bytes, null);
User user = reader.read(null, decoder);
Note: If you are serializing or deserializing in a loop or as a method, you should be reusing objects, readers and/or writers. Check the JavaDoc to see how to reuse the objects.