Skip to content

Conversation

@alexvanboxel
Copy link
Contributor

@alexvanboxel alexvanboxel commented Dec 18, 2019

This is the first PR of a multipart commit: this ticket implements the
basic infrastructure of options on row and field. Options in Beam
Schema add extra context to fields and schema. In contracts to
metadata, options would be added to fields, logical types and rows.

Options are key/typed value combination. The type system is using
the beam schema itself and the value can be any type that is supported
by the beam schema, including row.

Post-Commit Tests Status (on master branch)

Lang SDK Apex Dataflow Flink Gearpump Samza Spark
Go Build Status --- --- Build Status --- --- Build Status
Java Build Status Build Status Build Status Build Status
Build Status
Build Status
Build Status Build Status Build Status
Build Status
Build Status
Python Build Status
Build Status
Build Status
Build Status
--- Build Status
Build Status
Build Status
Build Status
--- --- Build Status
XLang --- --- --- Build Status --- --- ---

Pre-Commit Tests Status (on master branch)

--- Java Python Go Website
Non-portable Build Status Build Status
Build Status
Build Status Build Status
Portable --- Build Status --- ---

See .test-infra/jenkins/README for trigger phrase, status and link of all Jenkins jobs.

@alexvanboxel alexvanboxel changed the title Schema Options [WIP] Schema Options Dec 18, 2019
@alexvanboxel alexvanboxel force-pushed the feature/options branch 4 times, most recently from 3c69b4a to 432b319 Compare December 23, 2019 21:51
@alexvanboxel
Copy link
Contributor Author

@reuvenlax this is draft, but before investing more into testing the builders and documentation I like your opinion.

@alexvanboxel alexvanboxel force-pushed the feature/options branch 3 times, most recently from 829c44c to 0816326 Compare December 27, 2019 18:04
@alexvanboxel alexvanboxel changed the title [WIP] Schema Options [BEAM-9035] Typed options for Row Schema and Field Dec 27, 2019
@alexvanboxel
Copy link
Contributor Author

Run Java PreCommit

@alexvanboxel alexvanboxel marked this pull request as ready for review December 28, 2019 08:54
@alexvanboxel
Copy link
Contributor Author

@TheNeuralBit @reuvenlax the first PR of the schema options feature is ready for review. Reuven already done a pre-review when this PR was in draft. See tickets for more details, but coming in later pull requests are: options on Logical types (removing metadata and making LT portable), conversion from proto and avro, ...

note: it seems the build infrastructure for java is hanging.

@alexvanboxel alexvanboxel force-pushed the feature/options branch 2 times, most recently from fd49963 to 58bb737 Compare January 1, 2020 08:24
@alexvanboxel
Copy link
Contributor Author

Run Java PreCommit

@alexvanboxel
Copy link
Contributor Author

Run Java PreCommit

@alexvanboxel
Copy link
Contributor Author

As promised, schema aware topics adding: @iemejia @aromanenko-dev (please read the associated doc: https://docs.google.com/document/d/1yCCRU5pViVQIO8-YAb66VRh3I-kl0F7bMez616tgM8Q/edit# )

@alexvanboxel
Copy link
Contributor Author

@iemejia would appreciate if you also had a look at this.

@iemejia
Copy link
Member

iemejia commented Feb 6, 2020

Sure @alexvanboxel sorry for delay taking a look later today.

@alexvanboxel
Copy link
Contributor Author

Run Java11 PreCommit

@alexvanboxel
Copy link
Contributor Author

Run Java PreCommit

@alexvanboxel
Copy link
Contributor Author

Run Python PreCommit

@alexvanboxel alexvanboxel requested a review from iemejia March 2, 2020 20:02
@alexvanboxel
Copy link
Contributor Author

Verified that the failed Python 2 build isn't due to the PR. This PR is ready for review. I have 2 PR's based on this one ready: one for AVRO and other for Protobuf.

@kennknowles kennknowles requested a review from reuvenlax March 3, 2020 18:04
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This fields redundant since Option already contains a name. Maybe just make repeated?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Looking back to the history I see that it was a repeated option. I don't have a preference. I can set it back to repeated option.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

this is the same as arrayValueFromProto.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why is this refactoring needed?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Because validation is used by both setOption as the Row builder. It's the same validation.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'm wondering if its worth having all these overload. Maybe just have the setOption() method? Having users pass in the FieldType doesn't seem horrible.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thru, I can remove this. Most people will just use the getters anyway. The setters will be used by proto/avro/ etc implementations.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

why special case null?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Options should never be null, if a null is set it's equivalent to removing the option. Or do you prefer and Exception?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Simpler to use Preconditions.checkNotNull

@alexvanboxel alexvanboxel force-pushed the feature/options branch 3 times, most recently from 9685373 to 5ec1553 Compare March 5, 2020 12:06
@alexvanboxel alexvanboxel requested a review from reuvenlax March 5, 2020 15:27
@alexvanboxel
Copy link
Contributor Author

@reuvenlax the fixup addresses the following issues:

  • map<string,Option> -> repeated option
  • removed setters, except setOption and setRowOption (kept getters though)
  • throw IllegalArgument when option is null

Answered:

  • SchemaVerification needed to be pullout out of row, used for Row and now Option validation
  • array/iterableValueFromProto same, but getElementList is not a shared method (different class, same method name)

@alexvanboxel
Copy link
Contributor Author

@reuvenlax can you have a look at the changes?

@alexvanboxel
Copy link
Contributor Author

@reuvenlax can you have a look at the changes?

Ping

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Simpler to use Preconditions.checkNotNull

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

could you instead use Row.Equals.deepEquals(thisOption.getValue(), otherOption.getValue(), thisOption.getType()).

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Any actually you should move that call into Option.equals, and not run that code here.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

As mentioned above, use Row.Equals.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

use Row.Equals.deepHashCode on the value.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

it's kind of odd to have a remove method in a builder - usually a builder only has add methods. What's the use case here?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

make final

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Maybe best to throw exception if optionName isn't found? That's what Row.getValue does.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think best to throw if option not found.

@alexvanboxel
Copy link
Contributor Author

@reuvenlax did most of your fixup's. Except:

  1. Kept remove, as this is the only way to remove an option when you copy options from one schema into another.

  2. getValue/Type should return null, as it's different then a Row (a row has a schema, and you would expect a field to be available). An option you first (in general) inspect for availability before doing something. Here is an example with the current code base:

        .getFields()
        .forEach(
            f -> {
              Integer pkSequence = f.getOptions().getValue("vptech.data.contract.v1.primary_key");
              if (pkSequence != null) {
                primaryKeyMap.put(pkSequence, dataSchema.indexOf(f.getName()));
              }
            });

if it would throw an exception... or you try catch everything (no no no!) or you use the hasOption method like this:

            .getFields()
            .forEach(
                    f -> {
                      if (f.getOptions().hasOption("vptech.data.contract.v1.primary_key")) {
                        primaryKeyMap.put(f.getOptions().getValue("vptech.data.contract.v1.primary_key", Integer.class), dataSchema.indexOf(f.getName()));
                      }
                    });

I and my colleagues prefer the simpler null check.

@alexvanboxel
Copy link
Contributor Author

Run Java PreCommit

@reuvenlax
Copy link
Contributor

@alexvanboxel

  1. I'm not entirely sure I understand the use case. Why do options need to be copied, while other constructs (such as schemas) don't?

  2. I disagree for a couple of reasons:

    • A FieldType can be nullable, and there's no reason not to support null FieldTypes in options. By returning null for non-existent fields, you make it hard to distinguish between a null value and an invalid get call (i.e. passing an option that is not part of the schema). This feels like the sort of behavior that seems simpler, but actually makes things more complex (e.g. the fact that Java Maps return null for non-existent values is often considered a mistaken design, for this very reason. It's hard to distinguish between a missing value and an explicit null stored in a map).

    • Options are no different than schemas - there is a static list of options for each field, so I don't see why this is so different.

I also want to understand better why we can't just use Row here (revisiting Brian's question). You mentioned using dots in field names, but AFAIK protobuf also prohibits dots. It's true that dots are often used in option specifications in proto, but that's just to set individual fields of a message. In Beam, we really should model those as a single row-valued option, not as multiple individual options. I.e. (from the proto docs):

option (my_method_option).foo = 567;
option (my_method_option).bar = "Some string";

should translate to a single Beam option named "my_method_option" not to two separate options.

dots are also used to represent package names of course. Is this the main reason you need to support dots?

@alexvanboxel
Copy link
Contributor Author

@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

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:

    { "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.

@reuvenlax
Copy link
Contributor

Some thoughts:

I know your use case for options involves mapping specific input types to schemas, however we need to make sure that the actual mechanism is more generic than that. We have many uses for field metadata (or options) inside of Beam, completely unrelated to any option concept on input data types. I would not like to make a design choice that precludes nullable FieldTypes now, because we may very well find that we need nullable field types.

That being said: I notice that you haven't removed the FieldType metadata in this PR. Is that planned for a future PR? I don't think we want both metadata and options in the protocol - Options seemed to me to be a better-typed version of metadata.

I don't personally think that the hasOption code is too bad, and it's definitely better than constraining the type system for Options in a way that we quite likely will regret. That being said, you've already added the getValueOrDefault method. So you can still write this code:

.getFields()
.forEach(
f -> {
Integer pkSequence = f.getOptions().getValueOrDefault("vptech.data.contract.v1.primary_key", null);
if (pkSequence != null) {
primaryKeyMap.put(pkSequence, dataSchema.indexOf(f.getName()));
}
});

A tiny bit more verbose, but much more explicit about what you are trying to do. (FYI the reason I don't suggest allowing getValue to return an Optional - even though it might be the "better" solution - is because the rest of the code base does not use Optional, and I want things to stay consistent).

About the removeOption builder method. I'm struggling with this a bit, since it feels like a weird pattern to put remove into a builder. I'm also not sure whether copying a a Schema field with just a subset of options is a common use case or not. Generally we should start off by keeping such things out of the core Beam API until we are convinced that there are multiple users who need them. I think you can still accomplish the same thing by simply adding a new Options.Builder.setOptions override that takes in a Map.

newField = newField.withOptions(Options.builder().setOptions(
Maps.filterKeys(
oldField.getOptions().getAllOptions(),
n -> !n.equals(optionToRemove))));

This code is probably a bit more annoying for you to write. If we are convinced that this is a common use case then I'm ok with adding removeOption, but if it's specific to your use case then I think we're better off adding the new setOptions method and using that.

@alexvanboxel
Copy link
Contributor Author

I think I addresses now all the issues in the latest fixup.

Talking about field metadata: yes this will be another PR. I didn't want this all in one PR. This is the groundwork to get a decent API in. I have at least 2 related PR's (proto and avro support) waiting before tackling removal of metadata though.

@reuvenlax
Copy link
Contributor

reuvenlax commented Mar 15, 2020 via email

@alexvanboxel
Copy link
Contributor Author

Taking another look. I'm fine with metadata being in a separate PR - it makes it easier to review. I just wanted to ensure that metadata was still in the plan.

Let me know if you see other concerns, otherwise I need a lgtm. I don't expect anyone else to review this.

@reuvenlax
Copy link
Contributor

@alexvanboxel I'm still not 100% sure that I agree that options are not schemas, and I don't love making SchemaVerification public. However I think it will be easier to iterate on those issues in isolation once the basic support is in, so I'm approving for now.

LGTM

This is the first PR of a multipart commit: this ticket implements the
basic infrastructure of options on row and field. Options in Beam
Schema add extra context to fields and schema. In contracts to
metadata, options would be added to fields, logical types and rows.

Options are key/typed value combination. The type system is using
the beam schema itself and the value can be any type that is supported
by the beam schema, including row.
@alexvanboxel
Copy link
Contributor Author

@alexvanboxel I'm still not 100% sure that I agree that options are not schemas, and I don't love making SchemaVerification public. However I think it will be easier to iterate on those issues in isolation once the basic support is in, so I'm approving for now.

Thanks, I'm always open for the schema discussion.

@alexvanboxel alexvanboxel merged commit 03c8544 into apache:master Mar 19, 2020
@alexvanboxel alexvanboxel deleted the feature/options branch March 19, 2020 08:02
/** Find the index of a given field. */
public int indexOf(String fieldName) {
Integer index = fieldIndices.get(fieldName);
if (index == null) {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This change caused BEAM-10631, we should always use if-blocks (as it was), or lazy Preconditions.checkArgument(boolean condition, String format, Object args...).

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Projects

None yet

Development

Successfully merging this pull request may close these issues.

5 participants