[
https://issues.apache.org/jira/browse/BEAM-9035?focusedWorklogId=402439&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-402439
]
ASF GitHub Bot logged work on BEAM-9035:
----------------------------------------
Author: ASF GitHub Bot
Created on: 12/Mar/20 19:10
Start Date: 12/Mar/20 19:10
Worklog Time Spent: 10m
Work Description: alexvanboxel commented on issue #10413: [BEAM-9035]
Typed options for Row Schema and Field
URL: https://github.com/apache/beam/pull/10413#issuecomment-598367815
@reuvenlax
> copying schema's
most of the use cases we have result in a transformation based on options
they generally result on a change of the schema. But it's important to not drop
all the meta data because in another (following) transform maybe needs that
meta data. Examples
```protobuf
message PaymentServiceProvider {
PaymentPlatform payment_platform = 1 [
(vptech.data.contract.v1.description) = "The booking platform
payment",
(vptech.data.contract.v1.enum_prefix) = "PAYMENT_PLATFORM_",
(vptech.data.contract.v1.optional) = true ]
}
```
in beam the `payment_platform` is a field with a Enum logical type. It has 3
options.
1. transforms that handles `description`, the field now has a real Beam
description and 2 options left
2. transforms that handles `enum_prefix`, strips the prefix from the enum
and changes the type from enum to string, and with one remaining option
3. the validation transforms looks to the `optional` option. if data quality
is not good with move the element to the deadletter otherwise, field is now
string with no options.
as you see, each transform changes the schema slightly.
> FieldType can be nullable
The implementation for each format (proto, avro, zeta, calcite) has control
over setting it. It should set type nullable too `false`.
> Options are no different than schemas
I disagree that they are the same. A schema describe the types of rows and
fields, options are meta data on top of the those rows and fields. An option
being there has a meaning, I never saw an option that was null. It's not
possible in proto, neither in avro.
Proto makes it possible to describe a field extension that lives in a
package that you can attach on any arbitrary
field/message/enum/file/service/method (resulting on this being an option).
Avro you can add arbitrary json on the record and field. It's not that you
attach a strongly typed record to each field...
If it's a blocker, I'll change it too move this along (but then I need to
change the BIP-1:
https://cwiki.apache.org/confluence/display/BEAM/%5BBIP-1%5D+Beam+Schema+Options)
> should translate to a single Beam option named "my_method_option" not to
two separate options.
No, it would translate to
`package.where.the.extension.is.defined.my_method_option`
> dots are also used to represent package names of course. Is this the main
reason you need to support dots?
yes. How could I ever map the above options if options would really be a
row? I also have an avro implementation ready. It also has don't. Here is a
real world example:
```avro
{ "name": "ts", "type": {
"type": "string",
"connect.version": 1,
"connect.default": "1970-01-01T00:00:00Z",
"connect.name": "io.debezium.time.ZonedTimestamp"
}
},
```
this results in a Beam type of `STRING` with 3 options: `connect.version`,
`connect.version` and `connect.name`.
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]
Issue Time Tracking
-------------------
Worklog Id: (was: 402439)
Time Spent: 7.5h (was: 7h 20m)
> BIP-1: Typed options for Row Schema and Fields
> ----------------------------------------------
>
> Key: BEAM-9035
> URL: https://issues.apache.org/jira/browse/BEAM-9035
> Project: Beam
> Issue Type: Sub-task
> Components: sdk-java-core
> Reporter: Alex Van Boxel
> Assignee: Alex Van Boxel
> Priority: Major
> Fix For: 2.19.0
>
> Time Spent: 7.5h
> Remaining Estimate: 0h
>
> This is the first issue of a multipart commit: this ticket implements the
> basic infrastructure of options on row and field.
> Full explanation:
> Introduce the concept of Options in Beam Schema’s to add extra context to
> fields and schema. In contracts to metadata, options would be added to
> fields, logical types and rows. In the options schema convertors can add
> options/annotations/decorators that were in the original schema, this context
> can be used in the rest of the pipeline for specific transformations or
> augment the end schema in the target output.
> Examples of options are:
> * informational: like the source of the data, ...
> * drive decisions further in the pipeline: flatten a row into another,
> rename a field, ...
> * influence something in the output: like cluster index, primary key, ...
> * logical type information
> And option is a key/typed value combination. The advantages of having the
> value types is:
> * Having strongly typed options would give a *portable way of Logical Types*
> to have structured information that could be shared over different languages.
> * This could keep the type intact when mapping from a formats that have
> strongly typed options (example: Protobuf).
> This is part of a multi ticket implementation. The following tickets are
> related:
> # Typed options for Row Schema and Fields
> # Convert Proto Options to Beam Schema options
> # Convert Avro extra information for Beam string options
> # Replace meta data with Logical Type options
> # Extract meta data in Calcite SQL to Beam options
> # Extract meta data in Zeta SQL to Beam options
> # Add java example of using option in a transform
> This feature is discussed with Reuven Lax, Brian Hulette
--
This message was sent by Atlassian Jira
(v8.3.4#803005)