PTransforms for reading and writing Avro files.
Reading Avro files
To read a
PCollection from one or more Avro files with the same schema known at
pipeline construction time, use
#read, using
AvroIO.Read#from to specify the
filename or filepattern to read from. If the filepatterns to be read are themselves in a
PCollection, apply
#readAll. If the schema is unknown at pipeline construction time, use
#parseGenericRecords or
#parseAllGenericRecords.
Many configuration options below apply to several or all of these transforms.
See
FileSystems for information on supported file systems and filepatterns.
Filepattern expansion and watching
By default,
#read prohibits filepatterns that match no files, and
#readAllallows them in case the filepattern contains a glob wildcard character. Use
Read#withEmptyMatchTreatment to configure this behavior.
By default, the filepatterns are expanded only once.
Read#watchForNewFiles allows
streaming of new files matching the filepattern(s).
Reading records of a known schema
To read specific records, such as Avro-generated classes, use
#read(Class). To read
GenericRecord, use
#readGenericRecords(Schema) which takes a
Schema object, or
#readGenericRecords(String) which takes an Avro schema in a
JSON-encoded string form. An exception will be thrown if a record doesn't match the specified
schema. Likewise, to read a
PCollection of filepatterns, apply
#readAllGenericRecords.
For example:
Pipeline p = ...;
Reading records of an unknown schema
To read records from files whose schema is unknown at pipeline construction time or differs
between files, use
#parseGenericRecords - in this case, you will need to specify a
parsing function for converting each
GenericRecord into a value of your custom type.
Likewise, to read a
PCollection of filepatterns with unknown schema, use
#parseAllGenericRecords.
For example:
Pipeline p = ...;}));
}
Reading from a
PCollection of filepatterns
Pipeline p = ...;
Streaming new files matching a filepattern
Pipeline p = ...;
Reading a very large number of files
If it is known that the filepattern will match a very large number of files (e.g. tens of
thousands or more), use
Read#withHintMatchesManyFiles for better performance and
scalability. Note that it may decrease performance if the filepattern matches only a small number
of files.
Writing Avro files
To write a
PCollection to one or more Avro files, use
AvroIO.Write, using
AvroIO.write().to(String) to specify the output filename prefix. The default
DefaultFilenamePolicy will use this prefix, in conjunction with a
ShardNameTemplate (set
via
Write#withShardNameTemplate(String)) and optional filename suffix (set via
Write#withSuffix(String), to generate output filenames in a sharded way. You can override this
default write filename policy using
Write#to(FileBasedSink.FilenamePolicy) to specify a
custom file naming policy.
By default,
AvroIO.Write produces output files that are compressed using the
org.apache.avro.file.Codec. This default can be changed or overridden
using
AvroIO.Write#withCodec.
Writing specific or generic records
To write specific records, such as Avro-generated classes, use
#write(Class). To write
GenericRecord, use either
#writeGenericRecords(Schema) which takes
a
Schema object, or
#writeGenericRecords(String) which takes a schema in a
JSON-encoded string form. An exception will be thrown if a record doesn't match the specified
schema.
For example:
// A simple Write to a local file (only runs locally):
Writing windowed or unbounded data
By default, all input is put into the global window before writing. If per-window writes are
desired - for example, when using a streaming runner -
AvroIO.Write#withWindowedWrites()will cause windowing and triggering to be preserved. When producing windowed writes with a
streaming runner that supports triggers, the number of output shards must be set explicitly using
AvroIO.Write#withNumShards(int); some runners may set this for you to a runner-chosen
value, so you may need not set it yourself. A
FileBasedSink.FilenamePolicy must be set,
and unique windows and triggers must produce unique filenames.
Writing data to multiple destinations
The following shows a more-complex example of AvroIO.Write usage, generating dynamic file
destinations as well as a dynamic Avro schema per file. In this example, a PCollection of user
events (e.g. actions on a website) is written out to Avro files. Each event contains the user id
as an integer field. We want events for each user to go into a specific directory for that user,
and each user's data should be written with a specific schema for that user; a side input is
used, so the schema can be calculated in a different stage.
// This is the user class that controls dynamic destinations for this avro write. The input topublic GenericRecord formatRecord(UserEvent record)
return formatUserRecord(record, getSchema(record.getUserId()));
}
public Schema getSchema(Integer userId)
return new Schema.Parser().parse(sideInput(userToSchemaMap).get(userId));
}
public Integer getDestination(UserEvent record)
return record.getUserId();
}
public Integer getDefaultDestination()
return 0;
}
public FilenamePolicy getFilenamePolicy(Integer userId)
return DefaultFilenamePolicy.fromParams(new Params().withBaseFilename(baseDir + "/user-"
+ userId + "/events"));
}
public List> getSideInputs()
return ImmutableList.>of(userToSchemaMap);
}
}
PCollection events = ...;
PCollectionView> userToSchemaMap = events.apply(
"ComputePerUserSchemas", new ComputePerUserSchemas());
events.apply("WriteAvros", AvroIO.writeCustomTypeToGenericRecords()
.to(new UserDynamicAvroDestinations(userToSchemaMap)));
}