> > Schema-aware transforms are not restricted to I/Os. An arbitrary transform > can be a Schema-Transform. Also, designation Read/Write does not map to an > arbitrary transform. Probably we should try to make this more generic ? >
Agreed, I suggest keeping everything on the left side of the name unique to the transform, so that the right side is consistently SchemaTransform | SchemaTransformProvider | SchemaTransformConfiguration. What do others think? Also, probably what's more important is the identifier of the > SchemaTransformProvider being unique. FWIW, we came up with a similar generic URN naming scheme for > cross-language transforms: > https://beam.apache.org/documentation/programming-guide/#1314-defining-a-urn The URN convention in that link looks good, it may be a good idea to replace transform with schematransform in the URN in this case to make a distinction. ie. beam:schematransform:org.apache.beam:kafka_read_with_metadata:v1. I will mention this in the other thread when I go over the comments in the Supporting SchemaTransforms doc [1]. [1] Supporting existing connectors with SchemaTrans... <https://docs.google.com/document/d/1qW9O3VxdGxUM887TdwhD1iH9AdNbpu0_wXbCGvFP0OM/edit?usp=drive_web> On Tue, Nov 15, 2022 at 3:41 PM John Casey via dev <dev@beam.apache.org> wrote: > One distinction here is the difference between the URN for a provider / > transform, and the class name in Java. > > We should have a standard for both, but they are distinct > > On Tue, Nov 15, 2022 at 3:39 PM Chamikara Jayalath via dev < > dev@beam.apache.org> wrote: > >> >> >> On Tue, Nov 15, 2022 at 11:50 AM Damon Douglas via dev < >> dev@beam.apache.org> wrote: >> >>> Hello Everyone, >>> >>> Do we like the following Java class naming convention for >>> SchemaTransformProviders [1]? The proposal is: >>> >>> <IOName>(Read|Write)SchemaTransformProvider >>> >>> >>> *For those new to Beam, even if this is your first day, consider >>> yourselves a welcome contributor to this conversation. Below are >>> definitions/references and a suggested learning guide to understand this >>> email.* >>> >>> Explanation >>> >>> The <IOName> identifies the Beam I/O [2] and Read or Write identifies a >>> read or write Ptransform, respectively. >>> >> >> Schema-aware transforms are not restricted to I/Os. An arbitrary >> transform can be a Schema-Transform. Also, designation Read/Write does not >> map to an arbitrary transform. Probably we should try to make this more >> generic ? >> >> Also, probably what's more important is the identifier of the >> SchemaTransformProvider being unique. Note the class name (the latter is >> guaranteed to be unique if we follow the Java package naming guidelines). >> >> FWIW, we came up with a similar generic URN naming scheme for >> cross-language transforms: >> https://beam.apache.org/documentation/programming-guide/#1314-defining-a-urn >> >> Thanks, >> Cham >> >> >>> For example, to implement a SchemaTransformProvider [1] for >>> BigQueryIO.Write[7], would look like: >>> >>> BigQueryWriteSchemaTransformProvider >>> >>> >>> And to implement a SchemaTransformProvider for PubSubIO.Read[8] would >>> like like: >>> >>> PubsubReadSchemaTransformProvider >>> >>> >>> Definitions/References >>> >>> [1] *SchemaTransformProvider*: A way for us to instantiate Beam IO >>> transforms using a language agnostic configuration. >>> SchemaTransformProvider builds a SchemaTransform[3] from a Beam Row[4] that >>> functions as the configuration of that SchemaProvider. >>> >>> https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/schemas/transforms/SchemaTransformProvider.html >>> >>> [2] *Beam I/O*: PTransform for reading from or writing to sources and >>> sinks. >>> https://beam.apache.org/documentation/programming-guide/#pipeline-io >>> >>> [3] *SchemaTransform*: An interface containing a buildTransform method >>> that returns a PCollectionRowTuple[5] to PCollectionRowTuple PTransform. >>> >>> https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/schemas/transforms/SchemaTransform.html >>> >>> [4] *Row*: A Beam Row is a generic element of data whose properties are >>> defined by a Schema[5]. >>> >>> https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/values/Row.html >>> >>> [5] *Schema*: A description of expected field names and their data >>> types. >>> >>> https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/schemas/Schema.html >>> >>> [6] *PCollectionRowTuple*: A grouping of Beam Rows[4] into a single >>> PInput or POutput tagged by a String name. >>> >>> https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/values/PCollectionRowTuple.html >>> >>> [7] *BigQueryIO.Write*: A PTransform for writing Beam elements to a >>> BigQuery table. >>> >>> https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.Write.html >>> >>> [8] *PubSubIO.Read*: A PTransform for reading from Pub/Sub and emitting >>> message payloads into a PCollection. >>> >>> https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/io/gcp/pubsub/PubsubIO.Read.html >>> >>> Suggested Learning/Reading to understand this email >>> >>> 1. https://beam.apache.org/documentation/programming-guide/#overview >>> 2. https://beam.apache.org/documentation/programming-guide/#transforms >>> (Up to 4.1) >>> 3. https://beam.apache.org/documentation/programming-guide/#pipeline-io >>> 4. https://beam.apache.org/documentation/programming-guide/#schemas >>> >>