*(To those who identify on the Beam learning path, I supplemented the original email with additional references/definitions that could help understand this reply.)*
Hello Robert, Thank you for reading the document and taking your time to review. The numbered answers below correspond to the numbered questions. 1) A machine is the intended configuration producer and SchemaTransform output consumer. However, I started with the perspective of a human user as an automated process would serve those needs. User stories are best written in collaboration with intended users. Yet, not having this I had to draw on my experience working with various Beam and non-Beam customers. I derived the following user story which is also in the document. The key phrase is underlined. *As a Beam IO Developer, …* *I want a schema aware configuration to produce a file writing PTransform, so that I can unify and normalize a single point of entry to write Row elements to a file or object system.* *I want the provider to decide the intended format and file type based on configured inputs so I don’t need to write code to support this.* To prevent messy if/then statements and enumeration of mapping a format to the resulting PTransform<PCollection<Row>, PDone> transform, I implemented FileWriteSchemaTransformFormatProvider [1] as an extension of Providers.Identifyable [2] whereas implementations are annotated with @AutoService [3]. 2) The Read side could adopt the same. We decided to put the code in its own module and will coordinate efforts with the individual who volunteered on the Read side implementation to maintain consistency. Potentially, the same format String parameter could map to the appropriate PTransform<PBegin, PCollection<Row>> using the same mechanism. *References / Definitions* 1. FileWriteSchemaTransformFormatProvider - an interface extension of Providers.Identifyable [2]. The intended cardinality of a file format such as Json, Avro, XML, etc and a FileWriteSchemaTransformFormatProvider is 1 to 1. See https://github.com/apache/beam/blob/master/sdks/java/io/fileschematransform/src/main/java/org/apache/beam/sdk/io/fileschematransform/FileWriteSchemaTransformFormatProvider.java and https://github.com/apache/beam/blob/master/sdks/java/io/fileschematransform/src/main/java/org/apache/beam/sdk/io/fileschematransform/FileWriteSchemaTransformFormatProviders.java 2. Providers.Identifyable - allows us to use a string value to map to a class in Beam. See https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/schemas/io/Providers.Identifyable.html 3. @AutoService - a Java class annotation that allows us to list all classes that are annotated with a particular class. The practical implications are that AutoService gives us a list. In combination with Providers.Identifyable gives us a convenient lookup Map. See https://www.baeldung.com/google-autoservice for a tutorial on AutoService. 4. PBegin - the "input" to a root PTransform used typically in transforms that read from sources. See https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/values/PBegin.html 5. PDone - the "output" of a PTransform typically in transforms that write to sinks. See https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/values/PDone.html On Fri, Dec 2, 2022 at 1:41 PM Robert Bradshaw via dev <dev@beam.apache.org> wrote: > Thanks for looking into this and the careful writeup. I've read the > design doc and it looks great, but have a couple of questions. > > (1) Why did you decide on having a single top-level FileWrite > transform whose config is ([common_parameters], [xml-params], > [csv-params], ...) rather than separate schema transforms for each. > (2) Is there a plan to do a similar thing for the Read side? > > On Fri, Dec 2, 2022 at 9:48 AM Damon Douglas <douglas.da...@gmail.com> > wrote: > > > > Hello Everyone, > > > > For those new to Beam, even if this is your first day, consider > yourselves a welcome contributor to this conversation. I remember what it > was like first learning Beam on my own and I am passionate about everyone's > learning experience. Below are definitions/references and a suggested > learning path to understand this email. > > > > Short Version (assumes Beam knowledge): Could someone review > https://github.com/apache/beam/pull/24479? Based on the design document > [1], It's the first of a series of pull requests that enable FileIO.Write > [2] support for Schema Transforms [3]. > > > > Long Version (for those first learning Beam): > > > > Explaining this without using Beam specific language. > > > > Suppose my team needs to quickly write to a file or object storage > system without writing the specific code to accomplish this final step. > This pull request begins work in enabling such ability. I can specify the > format such as avro, json, xml, etc in the configuration file and a backend > service will deal with the remaining details of how to achieve this at > scale. > > > > If you are interested in how this works, please see the design document > [1]. > > > > Definitions/References: > > > > 1. bit.ly/fileioschematransformwriteprovider > > 2. FileIO.Write - A Beam transform that writes to file or object storage > systems > > See > https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/io/FileIO.Write.html > > 3. Schema Transform - Enables Schema Aware transforms when reading from > and writing to sources and sinks using Beam IOs defined declaratively using > a single configuration input > > 4. Schema Awareness refers to transforms that know how to process > pipeline elements with inherent knowledge of their properties and types. > This collection of properties and types is called a Schema. A Beam Row > contains properties and a data structure described by a Schema. Think of > the Row as the data element described by its Schema. > > > > Best, > > > > Damon > > >