Skip to content

Conversation

@alexvanboxel
Copy link
Contributor

The default implementation for Protobuf messages. Supports both
generated as dynamic messages.


NOTE: This implementation doesn't support Unknown Fields. I like to have a discussion with the reviewers how todo this.

@alexvanboxel alexvanboxel force-pushed the feature/BEAM-7274-proto-schema branch 2 times, most recently from 8f3473c to 799e5b3 Compare May 26, 2019 21:10
@alexvanboxel
Copy link
Contributor Author

@reuvenlax here is the first draft of Protobuf support. I hope the unit test testCoder and testCoderOnDynamic proves that having oneof's work as nullable fields (if you look at the Java API., they are actually treated as such. I think almost all permutations are unit-tested.

Things to discuss: Unknown fields.

@reuvenlax
Copy link
Contributor

I'm still reviewing this. A few high-level thoughts off the bat:

  • I'm not thrilled about creating a ProtoRow object, as I've so far attempted to prevent Row from becoming an "open" class that can by extended by users. I'd be happier if we could do this the way we did Avro classes, by generating a set of getters/setters (it would probably be more efficient as well)
  • There are various places where logical types would be preferable - e.g. all the places where multiple proto types map to a single Beam schema type. As written today the conversion is lossy - if you selected just that field, there would be no way to get the original proto type back; LogicalTypes provide a simple way of fixing this, though we could defer that to a future PR.

@alexvanboxel
Copy link
Contributor Author

@reuvenlax I'll wait for your complete review, but keep in mind that I wanted a uniform way for handling concrete typed classes and DynamicType (DynamicType do require access via the FieldDescriptor). I had trouble getting it in the same way as the AvroUtils, but maybe this is due to my unfamiliarity with it. (I'm open to try to convert it, as most of the effort is spend in the tests anyway. The for internal use only note in the Java doc didn't seem encouraging to use though.

On LogicalTypes: Are you talking about Fixed32/Fixed64/UInt32/UInt64? I simply took the Scalar mapping described here: https://developers.google.com/protocol-buffers/docs/proto#scalar

If you are talking about Timestamp... well I like to have this configurable. Having it default to DATETIME will be a helper for making it available in Beam SQL or in output IO's like BigQuery.

@alexvanboxel
Copy link
Contributor Author

Addition info, so I tried to get the logic using the FieldValueGetter with partial success, see experiment: alexvanboxel@d1a8e16

My main issue is with RowWithGetters private get(Map/List)Value values that assume they have enough information to construct the embedded complex data structures. I could get rid of my ProtoRow if I could let the handling of the List/Row/Map to it's dedicated FieldValueGetter

@alexvanboxel
Copy link
Contributor Author

alexvanboxel commented May 31, 2019

I got it to work with the RowWithGetters but not without modifying it. See alexvanboxel@1da5715

I keep on handling the List/Map/Row with the Getters as I need more context then the other implementations. Personally I find this better and more flexible then doing it in the RowWithGetters as you have more ways to give context (example: proto map is a repeatable message).

I looked for reference to the GenericRecord in Avro, but was disappointed that the GenericRecord gets materialised in a RowWithStorage. I'm quite sure with having the getters that handle collection this could be changed to RowWithGetters

*
* <p>Follow https://github.com/protocolbuffers/protobuf/issues/6101 for the state of the feature
* request in ProtoBuf.
*/
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 convert the Descriptor to a proto (using the Descriptor.toProto method) and store that? Then I think you won't need this class at all.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

So I removed this class as it was not super elegant, but .toProto() is not a solution as it only works in one way. A descriptor is always contained within a FileDescriptor.
I've introduced a Domain class that could group descriptors that are related with each other. This is an important component that a ProtoBuf schema registry should provision to make all Descriptors crosswired. See the Javadoc in the ProtoDomain class.

@alexvanboxel
Copy link
Contributor Author

alexvanboxel commented Jun 6, 2019 via email

@reuvenlax
Copy link
Contributor

Yes. I'd prefer to keep Row as a somewhat sealed class and not introduce extra subtypes. However there was enough other things to review in this PR that I figured I could review that first.

@alexvanboxel
Copy link
Contributor Author

alexvanboxel commented Jun 6, 2019 via email

@alexvanboxel alexvanboxel force-pushed the feature/BEAM-7274-proto-schema branch 3 times, most recently from 782b6b9 to cfadb9b Compare June 8, 2019 20:46
@alexvanboxel
Copy link
Contributor Author

Yes. I'd prefer to keep Row as a somewhat sealed class and not introduce extra subtypes. However there was enough other things to review in this PR that I figured I could review that first.

Answered (and fixed comments), I also rebased the experiment to remove the ProtoRow as a subtype: alexvanboxel@1629557

public Builder withFieldValueGettersHandleCollections(boolean collectionHandledByGetter) {
this.collectionHandledByGetter = collectionHandledByGetter;
return this;
}
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 having trouble understanding what this is for. Can you explain?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

OK, maybe the name could be better, but it means that the the FieldValueGetters also handle collections like ARRAY, ROW, MAP. If you go to the original implementation ( https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/values/RowWithGetters.java#L90 ) you see that RowWithGetters does naive handling of ARRAY, ROW and MAP, for protobuf you need more context (the descriptor) to handle them. That's why I need to disable the naive mapping and let the FieldValueGetters handle ARRAY, ROW and MAP.

Feel free to suggest a better name though.

Copy link
Member

Choose a reason for hiding this comment

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

@reuvenlax is this the last remaining concern you have for this PR?

#8690 (comment) is a good reference for the motivation of this as well.

I'm not crazy about the way this is implemented since it's adding state to RowWithGetter that will get checked every time a collection field is accessed. I can't think of a better way to do it without some non-trivial refactoring though. Some ideas:

  1. Rather than storing collectionHandledByGetter in RowWithGetters and changing behavior based on it, have two alternate RowWithGetters implementations, one with the special handling for collections and one without. I think this is still an improvement over the separate ProtoRow class that was rejected since it's not explicitly tied to a particular SchemaProvider, in fact it sounds like it could be re-used for Avro GenericRecord instances.
  2. Move the special logic for collection getters into those SchemaProvider/FieldValueGetter implementations that need it, and make RowWithGetters always behave as if collectionHandledByGetter is true. I think this could be a cleaner approach? But it's challenging because that special logic stores cached values that wouldn't be appropriate to move into the FieldValueGetter implementations. Maybe a SchemaProvider could have some way to indicate that a certain row is expensive to access and should be cached in RowWithGetters?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I've refactored from your input: I've created a RowWithGettersCachedCollection that inherits for RowWithGetters. This cached is the default.

Copy link
Contributor

Choose a reason for hiding this comment

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

Can you help me understand this a bit more? Why does it not work to cache lists for protocol buffers? We saw repeated array conversion to be a big problem (which is why we cache them). I'm wondering if we could instead cache a lazy array like we do with iterables.

I'll take a closer look at this code to figure it out.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Would we make a ticket out of this or is this blocking?

@alexvanboxel alexvanboxel force-pushed the feature/BEAM-7274-proto-schema branch 9 times, most recently from 868ec11 to 37edf7b Compare August 31, 2019 07:30
@alexvanboxel
Copy link
Contributor Author

@reuvenlax I think I tackled all your concerns you posted. The build is failing, but this is due to a warning in Jenkins that has nothing todo with this PR (I investigated). The PR is also rebased.

  • Reverted changes to Row so it's back to the original implementation, except for the need of collectionHandledByGetter see remark above. (better name always welcome)
  • ProtoDescriptorSerializer is gone. I defined a ProtoDomain class that the user needs to use themself if they want to use DynamicMessages. having one ProtoDomain for all messages that belong together will make sure object equality works.
  • ProtoSchemaProvider only handles compiled Proto classes, it will give an error on DynamicMessages.
  • ProtoSchema is the workhorse, it's a bit like AvroSchemaUtils.

It would be great having this in the next release. From then on we can do incremental changes like:

  • Documentation
  • Option handling
  • Logical types

I also backported the ProtoDomain on the DynamicMessage PR to align it with this.

Sorry it took so long, but I needed to prioritize something different, but related with Proto (a proto schema registry).

@alexvanboxel alexvanboxel force-pushed the feature/BEAM-7274-proto-schema branch from 37edf7b to 2988800 Compare September 3, 2019 09:52
@alexvanboxel
Copy link
Contributor Author

Run Java PreCommit

@alexvanboxel
Copy link
Contributor Author

Requesting @pabloem as a reviewer.

@pabloem pabloem self-requested a review September 19, 2019 17:02
Copy link
Member

@TheNeuralBit TheNeuralBit left a comment

Choose a reason for hiding this comment

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

Thanks @alexvanboxel I'm excited to add one more schema provider to Beam :)


Message messageCoded = schemaCoderCoded.getFromRowFunction().apply(rowCoded);
assertEquals(message, messageCoded);
}
Copy link
Member

Choose a reason for hiding this comment

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

I think CoderProperties.coderDecodeEncodeEqual actually implements a lot of the logic here. It even serializes/deserializes the coder before encoding and decoding the elements. What do you think about using that here?

There's also CoderProperties.coderSerializable which just clones the coder and checks that they're equal.

* external registry that maps those URN's with Descriptors.
*/
@Experimental(Experimental.Kind.SCHEMAS)
public class ProtoSchemaProvider implements SchemaProvider {
Copy link
Member

Choose a reason for hiding this comment

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

Is the need for withFieldValueGettersHandleCollections(true) the only reason this can't be made to implement GetterBasedSchemaProvider?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

No, each data type has it's own Provider (eg. AvroSchemaProvider). This class makes sure Proto can be auto mapped.

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 not sure I understand this comment.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Let me elaborate. The ProtoSchemaProvider delegates the functionality from to/fromRow to the ProtoSchema class that is, yes, optimized for dynamic types. The ProtoSchema class managed most of the descriptors. that's it doesn't the default GetterBasedProvider.

If we would optimize for generated classes this could switch (but I prefer this to do in a separate PR later on).

Comment on lines 64 to 66
* <li>Protobuf enums are mapped to non-nullable string values.
* <li>Enum map to their string representation
Copy link
Member

Choose a reason for hiding this comment

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

nit: Are these two lines saying the same thing?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Removed

Comment on lines +61 to +64
* <li>Protobuf maps are mapped to nullable maps, where empty maps are mapped to the null value.
* <li>Protobuf repeatables are mapped to nullable arrays, where empty arrays are mapped to the
* null value.
Copy link
Member

Choose a reason for hiding this comment

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

Why not just use non-nullable maps and arrays and produce empty ones when the proto ones are empty?

Copy link
Contributor Author

@alexvanboxel alexvanboxel Nov 11, 2019

Choose a reason for hiding this comment

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

Business wise this makes more sense. As proto doesn't distinguish between empty map/array and not set. It's also more performant as if the default would be an empty map/array it would result in a bigger Row. It would be an option to make it later configurable. Kept it as is.

@Override
public void set(Message.Builder message, Object value) {
// builder.setField(fieldDescriptor,
// convertSetObject(row.getString(fieldDescriptor.getName())));
Copy link
Member

Choose a reason for hiding this comment

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

nit: can you remove this?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

removed


@Test
public void testRowAndBack() {
setup();
Copy link
Member

Choose a reason for hiding this comment

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

I wonder if this could be re-structured a bit so that you could use @Before instead of calling the setup functions yourself. Maybe if you have two different Parameterized test classes that draw from the same data? One for message and one for dynamic message. You can have multiple suites in one outer class if you annotate it with @RunWith(Enclosed.class) (like I did here)

This is definitely a nitpicky thing... totally fine to leave it as is.

@alexvanboxel
Copy link
Contributor Author

This PR is now rebased against master.This removed ProtoDomain (and it's tests) from this PR as it was part of the DymamicMessage PR that is now part of master. I think I've incorporated most of the comment of the review in this PR as well.

@alexvanboxel alexvanboxel force-pushed the feature/BEAM-7274-proto-schema branch from 7e9c3b3 to 9f6d4f2 Compare November 11, 2019 16:36
The default implementation for Protobuf messages. Supports both
generated as dynamic messages.

Add the initial release of handling of options, converting it to
schema metadata. The implementation is not feature complete, be
shows the viability of the feature.
Split RowWithGetters in an implementation with a version that caches
collections, named RowWithGettersCachedCollection and one where all the
types are handled by the FieldGetters named RowWithGetters.
@alexvanboxel alexvanboxel force-pushed the feature/BEAM-7274-proto-schema branch from 9f6d4f2 to e5d5c77 Compare November 12, 2019 12:05
@alexvanboxel
Copy link
Contributor Author

@TheNeuralBit / @pabloem can you re-review. Thanks.

@reuvenlax
Copy link
Contributor

Another comment: this delegates everything to Proto's reflection API, which can be quite inefficient. Compare with AvroSchema where we delegate straight to generated classes when they exist. Reflection might be necessary for the case of DynamicMessage (though in that case I think we should use RowWithStorage instead of RowWithGetters), but shouldn't be necessary when we have generated classes.

@alexvanboxel
Copy link
Contributor Author

Another comment: this delegates everything to Proto's reflection API, which can be quite inefficient. Compare with AvroSchema where we delegate straight to generated classes when they exist.

Totally agree, but I wanted first a solution that works for every use-case. Optimization can happen afterward. I still have plans for iterative improvements, certainly related to protobuf options first, then optimize for performance (but I have to make performance tests for this to measure this)

Reflection might be necessary for the case of DynamicMessage (though in that case I think we should use RowWithStorage instead of RowWithGetters), but shouldn't be necessary when we have generated classes.

RowWithStorage could be a solution when the Row is materialized (with the getToRowFunction), but having the reflection API is still needed it you use the schema without materializing. Example just you the filter on a column function on a schema aware collection but keeping the original type, you don't want to materialize the a big proto just to filter on 1 column. Again, I like to measure this, but we need a base line.

@reuvenlax
Copy link
Contributor

BTW I've checked out the PR to play with it a bit more. My worry is that in its current form "optimizing later" will not be so simple, since it doesn't currently fit in with the framework used by AvroSchema, AutoValueSchema, JavaBeanSchema, etc.. If you don't mind, I'll try and play with it a bit to see if I can get it to fit into that framework a bit better.

@alexvanboxel
Copy link
Contributor Author

BTW I've checked out the PR to play with it a bit more. My worry is that in its current form "optimizing later" will not be so simple, since it doesn't currently fit in with the framework used by AvroSchema, AutoValueSchema, JavaBeanSchema, etc.. If you don't mind, I'll try and play with it a bit to see if I can get it to fit into that framework a bit better.

I don't mind, the thing that worries me is that it will miss the 2.18 as well (as it missed the 2.17 already). As we are depending in our business on this feature we are currently building our own version of Beam and that's a pain. For me the DynamicMessage is very important... I was thinking for the normal use-case going to a complete other codepath.

@alexvanboxel
Copy link
Contributor Author

Closing this PR as it's supersede by several other PR that are already on master: Last remaining PR for full proto support is: #10502

Option/Metadata support has been split of and can be followed:

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

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

3 participants