Skip to content

[BEAM-3157] Generate BeamRecord types from Pojos#4204

Merged
iemejia merged 1 commit intoapache:masterfrom
akedin:generate-record-types
Feb 8, 2018
Merged

[BEAM-3157] Generate BeamRecord types from Pojos#4204
iemejia merged 1 commit intoapache:masterfrom
akedin:generate-record-types

Conversation

@akedin
Copy link
Copy Markdown
Contributor

@akedin akedin commented Dec 2, 2017

This implements automatic generation of BeamRecordTypes and BeamRecordSqlTypes from pojo types. Work is being done as part of BEAM-3157.

Main piece is RecordFactory which exposes a method to generate BeamRecords from pojos. See RecordFactoryTest for usage examples.

The plan is to integrate this into the Beam SQL framework. Integration into SQL will be done in the future PRs.

Records generation is a major step to simplify conversion of pojo model to BeamRecords. Immediate use case is implementation of Nexmark queries in Beam SQL using existing pojo models.
This can also be used as a starting point for code generation for schema-aware collections.

Follow this checklist to help us incorporate your contribution quickly and easily:

  • Make sure there is a JIRA issue filed for the change (usually before you start working on it). Trivial changes like typos do not require a JIRA issue. Your pull request should address just this issue, without pulling in other changes.
  • Each commit in the pull request should have a meaningful subject line and body.
  • Format the pull request title like [BEAM-XXX] Fixes bug in ApproximateQuantiles, where you replace BEAM-XXX with the appropriate JIRA issue.
  • Write a pull request description that is detailed enough to understand what the pull request does, how, and why.
  • Run mvn clean verify to make sure basic checks pass. A more thorough check will be performed on your pull request automatically.
  • If this contribution is large, please file an Apache Individual Contributor License Agreement.

@akedin
Copy link
Copy Markdown
Contributor Author

akedin commented Dec 2, 2017

R: @kennknowles @reuvenlax @iemejia
Please take a look

Copy link
Copy Markdown
Member

@iemejia iemejia left a comment

Choose a reason for hiding this comment

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

I just took a quick look and let some remarks but I feel a bit confused, I expected a simple function that from a POJO extracted the schema (aka BeamRecordType) + the machinery to ensure the order of the correct correspondance of types/coders, names, values (lists) to build a BeamRecord from it, but this seems way farther than this. So I am not sure if I am missing something or if this is more complex than I expected (or than it should be). I feel particularly uneasy about all the complexity of those injected FieldValueGetters when the object values have the same information than the Pojo.

/**
* An interface to access a field of a class.
*
* <p>Implementations of this interface are generated at rutime by {@link RecordFactory}
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

s/rutime/runtime

import org.junit.Test;

/**
* Unit tests for {@link RecordFactoryTest}.
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

s/RecordFactoryTest/RecordFactory

*/
public class DefaultRecordTypeFactory implements RecordTypeFactory {

private static final ImmutableMap<Class, Coder> JAVA_TYPES_CODERS = ImmutableMap
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

Wonder if this could come somehow come from CoderRegistry#getCoder(java.lang.Class) to avoid the repeated mapping ?

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

+1 on using the registry to figure this out.

* Takes field names and types from the getters, maps them to coders, returns
* an instance of the {@link BeamRecordType}.
*
* <p>Coders not for all field types are implemented.
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

This line is hard to read, maybe remove.

*
* <p>Generated record types are cached in the instance of this factory.
*
* <p>Currently only 1-1 mapping is supported between pojo classes and record types.
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

Maybe add that only 'primitive' types are supported because it does not support more complex types, Lists/Sets/Maps, etc.

*
* <p>Currently only 1-1 mapping is supported between pojo classes and record types.
*/
public class RecordFactory {
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

BeamRecordFactory ? Probably to remind us in the future to rename this when we move BeamRecord -> Row (or Record).

* <p>Field names for getters are stripped of the 'get' prefix.
* For example record field 'name' will be generated for 'getName()' pojo method.
*/
public BeamRecord newRecordCopyOf(Object pojo) {
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

of(..) ?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

I wanted to make clear it's a copy, not a pass-through wrapper backed by the original pojo

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

create() then ?

* For example record field 'name' will be generated for 'getName()' pojo method.
*/
public BeamRecord newRecordCopyOf(Object pojo) {
RecordTypeGetters getters = getRowType(pojo.getClass());
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

We should probably not mix the terms Record and Row (we should stick to one and for the moment is Record), but in the discussion on schema-aware PCollection the future seems to be Row.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Ah, yes, these are typos, will double check to use Record everywhere. I don't have a strong opinion whether to use Row or Record, so will continue to use Record for now until there's a decision to rename to Rows.

* See {@link ByteBuddyUtils#makeNewGetterInstance(String, DynamicType.Builder)}
* and ByteBuddy documentation for details.
*/
public class DirectAccessGetterFactory {
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

This asumes a different contract of the Pojo contract I am not 100% that this is a good idea, maybe we should stick to the basic case (what is exposed via public getters, or is there a strong reason to do this ?.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Main reason is that current Nexmark model has data objects only with public fields. It's relatively easy to convert them to getters but I didn't see any harm in supporting this as well.

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

I see, in this case we will change Nexmark to have public getters, this implementation should stick to the POJO 'semantics' and avoid to move from it (to avoid possible issues).

* <p>Implementations of this interface are generated at rutime by {@link RecordFactory}
* to map pojo fields to BeamRecord fields.
*/
public interface FieldValueGetter<T> {
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

I am a bit puzzled by this Interface, is this really needed ? I mean with the Pojo object we have all that is needed, we have the name/type by reflection and the value just by invoking the method.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

If you use reflection, then you probably don't need this specific interface. But I don't think that getting rid of it, or switching to reflection will reduce complexity a lot without sacrificing something.

This is how I understand it:

Reflection or other implementation details aside, there are 2 problems:

  • map a pojo class to a record type;
  • given a pojo instance, create a record from it, populating the fields in the order they are specified in the record type;

Creating a record type is straightforward, you need to map pojo fields/getters to record field names and coders. You can do it either by convention or using an explicit spec. Record types can be either cached for pojo classes, or re-created for every pojo instance. Overall this feels straightforward.

Second problem creating a record from a pojo conceptually looks like this:

  • look up or create the record type for the pojo class;
  • get the fields in correct order (same as in record type);
  • for each field invoke the corresponding getter on the pojo;

This means that now we need a mechanism to map record field types to pojo getters (here I assume that even with reflection we will still need to cache Methods).

Given the above you seem to need few abstractions no matter whether you use reflection or not:

  • main component to create records based on pojos which will:
  • component to create record types, which will:
    • map getter names to record field names;
    • map getter return types to coders;
    • there will be 2 flavors of this (one, two) because SQL has its custom coders mapping;

So I feel like most of the components in this PR will be needed in some way or another, and the main question is how you map getters to fields and invoke them.

Using reflection:

  • store each getter in a List<Method>;
  • invoke a getter with reflection;

Using code generation:

  • generate a class to represent a Method call;
  • invoke getter the class directly;

There are pros and cons for both. How I see it:

Reflection:

  • simpler code because you don't have the code generation logic and custom class to represent the getter method;
  • the implementation is java runtime specific on multiple levels;
  • feels like it might require more serialization code if we want to pass the schema information around;
  • easy to shoot yourself in the foot if you start doing more reflection than needed;

Code generation:

  • more code;
  • potentially can be optimized further by JIT;
  • doesn't have to be at run-time;
  • similar interfaces / mechanisms can be implemented in other languages for schema support;

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

Thanks for specifying this, I understand now better the motivation.sorry I forgot to send some points after checking this, I think we agree in the core ideas/goal, my only complain is more API based. Some notes inline:

Reflection or other implementation details aside, there are 2 problems:

map a pojo class to a record type;
given a pojo instance, create a record from it, populating the fields in the order they are specified in the record type;

Sure !

You can quickly achieve this by mapping the POJO class into a BeamRecord type. This is just getting the public get* methods and create a LinkedHashMap<String, Coder> where the key is the name of the field and the coder is the given coder for the type.

Caching this mapping seems like an excellent idea because this is a time consuming step. So it should be cached in some sort of Map<Class, BeamRecordType> where the Key is the Pojo Class.

Second problem creating a record from a pojo conceptually looks like this:

look up or create the record type for the pojo class;
get the fields in correct order (same as in record type);
for each field invoke the corresponding getter on the pojo;

This means that now we need a mechanism to map record field types to pojo getters (here I assume that even with reflection we will still need to cache Methods).

Given the above you seem to need few abstractions no matter whether you use reflection or not:

I disagree, you don’t really need extra abstractions for this if you do it by reflection, but more important that needing or not these abstractions is the fact of exposing them in the API. You can do this by simply having a method with a signature like this:

List<Object> getValues(Object myPojo, List<String> attributes);

and then with that List of values and the BeamRecordType you can build the BeamRecord.

The way you implement this method to make it faster is another story and it is great if we can add some tricks to improve this, but we should try to avoid the optimization from leaking into the API. We are doing this in sdks/java/core, trying not to leak abstractions at this level is important because if we don’t take care here they will be used everywhere.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

I am not sure I understand your point about exposing the abstractions in the API. The only thing I am planning to expose is BeamRecordFactory.create(pojo). This currently creates the BeamRecords using all public getters from the pojo instances. This PR scope is to only implement this.
Probably eventually there will be another parameter added to specify a schema. Whether it's backed by reflection or code generation should not be leaked at this point. Actual schema support or even getting the BeamRecord for a subset of the getters of a pojo is out of the scope here.

Imagine that it is plugged into SQL to convert from pojos to records somewhat like this:

class PojoToRecord extends DoFn<Pojo, BeamRecord>() {
  @ProcessElement
  public void processElement(Context c) { 
     c.output(BeamRecordFactory.instance().create(c.element()));
  }

List<Object> getValues(Object myPojo, List<String> attributes);

Yes, I agree. Something like this is needed. But to actually create a record you also need to make sure that the attributes here and the BeamRecordType have the same attributes in the same order.

To achieve that, it would make sense to store that order somewhere, like Map<Class, List<String>>, and construct it at the same time you create the BeamRecordType because you're analyzing getters anyway. This way the list of attributes is available in the map when you are creating the beam records later. Then you use this list of attributes to get the values by calling getValues(myPojo, attributes).

I agree it's all pretty straightforward here, but I argue that it's roughly what's happening here anyway, minus the actual code generation bit.

For example, consider following:

  • you can combine the values from Map<Class, BeamRecordType> and Map<Class, List<String>> and put them in the utility class: RecordTypeGetters { beamRecordType, attributes }, and have a single map Map<Class, RecordTypeGetters>.
  • when calling getValues(myPojo, attributes) the most expensive part is actually looking up the methods on the class, so I would cache them:
    • instead of storing the attribute names, just store the getter methods in a List<Method>;
    • so now you can change getValues(Object myPojo, List<String> attributes) to getValues(Object myPojo, List<Method> getters);
    • now something like this is already in BeamRecordFactory: List<Object> getFieldValues(List<FieldValueGetter> fieldValueGetters, Object pojo);
  • you still need to wrap everything into some utility/factory class:
    • getting the getters from pojo;
    • mapping getter return types to coders for sql and non-sql versions;
    • caching;
    • getting the values from pojo;
  • now if you replace super low-level Method with our own class FieldValueGetter you have my implementation here. The only thing different is whether we get FieldValueGetter/Method by reflection or by codegen;

@akedin akedin force-pushed the generate-record-types branch from d2e4a41 to 4658d10 Compare December 4, 2017 21:31
*
* <p>Currently only 1-1 mapping is supported between pojo classes and record types.
*/
public class RecordFactory {
Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

iemejia wrote:
BeamRecordFactory ? Probably to remind us in the future to rename this when we move BeamRecord -> Row (or Record).

Sure, renamed this one as it is the main interface. Kept other classes without Beam prefix for now. Let me know if you think other ones would be also better with Beam

*/
public class DefaultRecordTypeFactory implements RecordTypeFactory {

private static final ImmutableMap<Class, Coder> JAVA_TYPES_CODERS = ImmutableMap
Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

lukecwik wrote:
+1 on using the registry to figure this out.

Changed to use CoderRegistry.createDefault(). Will update this to get the registry from the pipeline as soon as wire it up (future PRs).

* Takes field names and types from the getters, maps them to coders, returns
* an instance of the {@link BeamRecordType}.
*
* <p>Coders not for all field types are implemented.
Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

iemejia wrote:
This line is hard to read, maybe remove.

Updated

/**
* An interface to access a field of a class.
*
* <p>Implementations of this interface are generated at rutime by {@link RecordFactory}
Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

iemejia wrote:
s/rutime/runtime

Fixed

import org.junit.Test;

/**
* Unit tests for {@link RecordFactoryTest}.
Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

iemejia wrote:
s/RecordFactoryTest/RecordFactory

Fixed

*
* <p>Generated record types are cached in the instance of this factory.
*
* <p>Currently only 1-1 mapping is supported between pojo classes and record types.
Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

iemejia wrote:
Maybe add that only 'primitive' types are supported because it does not support more complex types, Lists/Sets/Maps, etc.

This class doesn't explicitly limit what types are supported. It uses RecordTypeFactory to create record types. After updating DefaultRecordTypeFactory to use CoderRegistry it should allow using a lot more types, e.g. byte arrays, sets, serializables. Not sure what would be the use of it at the moment, because it's not used in SQL. SqlRecordTypeFactory is the one used for SQL and it does not support complex types at the moment.

* See {@link ByteBuddyUtils#makeNewGetterInstance(String, DynamicType.Builder)}
* and ByteBuddy documentation for details.
*/
public class DirectAccessGetterFactory {
Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

iemejia wrote:
I see, in this case we will change Nexmark to have public getters, this implementation should stick to the POJO 'semantics' and avoid to move from it (to avoid possible issues).

Removed the direct field accessor implementation.

Although my understanding is that Pojo doesn't imply getter-only. Beans imply getters, but Pojos only imply independence, e.g. no interfaces implementation, custom annotations etc.

* <p>Field names for getters are stripped of the 'get' prefix.
* For example record field 'name' will be generated for 'getName()' pojo method.
*/
public BeamRecord newRecordCopyOf(Object pojo) {
Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

iemejia wrote:
create() then ?

done

@kennknowles
Copy link
Copy Markdown
Member

If I understand correctly, we may want to make different choices about generating bytecode (and how to do it) versus cached reflection (and how to do this) so the best thing to do would be to put the interface in a place that allows us to do each of these things and compare their performance. We can then change our mind over time.

Is there something in the API that would be a problem for this?

Copy link
Copy Markdown
Contributor Author

@akedin akedin left a comment

Choose a reason for hiding this comment

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

kennknowles wrote:
If I understand correctly, we may want to make different choices about generating bytecode (and how to do it) versus cached reflection (and how to do this) so the best thing to do would be to put the interface in a place that allows us to do each of these things and compare their performance. We can then change our mind over time.

Is there something in the API that would be a problem for this?

I don't believe there is currently anything that makes it hard to extend or make this more configurable in the future.

The only API I think should be used externally is BeamRecordFactory.create(pojo) which doesn't expose implementation details.

I did a brief exercise of implementing different ways of generating getters, including a different kind of code generation and reflection:

Public API BeamRecordFactory.create(pojo) internally calls GetterMethodGetterFactory. generateGetters(). This generateGetters() call is the main piece that generates wraps the calls to pojo fields getters into instances of FieldValueGetter.

At the moment, if you wanted to create instances of FieldValueGetter backed by reflection or different kind of code generation, you would need to implement a factory class similar to GetterMethodGetterFactory which would use reflection, and then swap the call to GetterMethodGetterFactory.generateGetters().

Right now there is no explicit java interface for the GetterMethodGetterFactory, and its generateGetters() call accepts a ByteBuddy instance. It is trivial to hide the ByteBuddy and introduce a general use interface though.

In one of the previous iterations of this PR there was already another implementation of the getters factory hooked up in addition to GetterMethodGetterFactory.generateGetters(). That implementation generated accessors for public fields instead of public getters. Both of these factories shared the same interface and it was a 1 line change in the top level BeamRecordFactory to remove it.

Similarly when I did a quick perf test I created a small ad-hoc reflection-based variant of GetterMethodGetterFactory, and hooking it up to BeamRecordFactory was also very simple.

If we go forward with some version of this code, then BeamRecordFactory can be modified to be configurable with instances of GetterMethodGetterFactory instead of hardcoding the calls. This change seems trivial to me.

@akedin akedin force-pushed the generate-record-types branch from a805d0a to 4596783 Compare January 5, 2018 00:37
Copy link
Copy Markdown
Contributor Author

@akedin akedin left a comment

Choose a reason for hiding this comment

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

@kennknowles

rebased, squashed commits, added interface for getter factory, reflective implementation. Quick perf

@akedin akedin force-pushed the generate-record-types branch from 4596783 to 40c4ff1 Compare January 5, 2018 01:09
@akedin
Copy link
Copy Markdown
Contributor Author

akedin commented Jan 5, 2018

retest this please

@lukecwik lukecwik requested review from kennknowles and lukecwik and removed request for lukecwik January 10, 2018 21:43
@kennknowles
Copy link
Copy Markdown
Member

Reviewed 12 of 18 files at r1, 12 of 12 files at r2.
Review status: all files reviewed at latest revision, 9 unresolved discussions, some commit checks failed.


sdks/java/core/src/main/java/org/apache/beam/sdk/values/reflect/BeamRecordFactory.java, line 53 at r2 (raw file):

   * <p>Use this to create instances of {@link BeamRecordType}.
   */
  public BeamRecordFactory() {

Recommended pattern for classes like this: A single private constructor that takes all the arguments. Then for each way of building one, have a public static method with a useful name. So this constructor would be default() and the one below would be forRecordTypeFactory(...), or something like that.


sdks/java/core/src/main/java/org/apache/beam/sdk/values/reflect/BeamRecordFactory.java, line 81 at r2 (raw file):

    RecordTypeGetters getters = getRecordType(pojo.getClass());
    List<Object> fieldValues = getFieldValues(getters.valueGetters(), pojo);
    return new BeamRecord(getters.recordType(), fieldValues);

Incidentally, BeamRecord should also have clear named factory methods instead of public constructor.


sdks/java/core/src/main/java/org/apache/beam/sdk/values/reflect/DefaultRecordTypeFactory.java, line 30 at r2 (raw file):

/**
 * A default factory to create a {@link BeamRecordType} based on pojo field getters.

Can you describe what its behavior is like? Why is it the default? Also - is it @Internal?


sdks/java/core/src/main/java/org/apache/beam/sdk/values/reflect/package-info.java, line 22 at r2 (raw file):

 * Classes to generate BeamRecords from pojos.
 */
package org.apache.beam.sdk.values.reflect;

Is there a special reason to make a new package? It does mean there might be things that are public but are not intended for users. So what we do in that case is mark them @Internal and also make a big bold comment that says "For internal use only. No backwards compatibility guarantees." Of if things that are public are what you want and you made stuff package private where appropriate, that's great.

And the other thing to do with the package is to set up static analysis with @DefaultAnnotation(NonNull.class)


sdks/java/core/src/main/java/org/apache/beam/sdk/values/reflect/field/FieldValueGetter.java, line 29 at r2 (raw file):

 * to map pojo fields to BeamRecord fields.
 */
public interface FieldValueGetter<T> {

While this is a pretty harmless interface, I think it should still be @Internal. I assume that SQL and the new DoFn code will perhaps interface with it so it will have to be public no matter what package, yea?


sdks/java/core/src/main/java/org/apache/beam/sdk/values/reflect/field/GeneratedGetterFactory.java, line 69 at r2 (raw file):

 * }</pre>
 *
 * <p>ByteBuddy is used to generate the code. Class naming is left to ByteBuddy's defaults.

With the DoFn stuff, we had issue where the stacktrace had pseudo-random bits in it so aggregated errors across workers didn't work. I think these bits are expected to be correct-by-construction and never allowed to throw exceptions, yes?


sdks/java/core/src/main/java/org/apache/beam/sdk/values/reflect/field/package-info.java, line 22 at r2 (raw file):

 * Classes to access fields of pojos to map to record fields.
 */
package org.apache.beam.sdk.values.reflect.field;

Is there a benefit to even more deeply nesting packages? I'd keep it a bit more flat just to make things easy to find and make it most flexible in terms of managing public/private.


sdks/java/core/src/test/java/org/apache/beam/sdk/values/reflect/BeamRecordFactoryTest.java, line 39 at r2 (raw file):

 * Unit tests for {@link BeamRecordFactory}.
 */
@RunWith(Parameterized.class)

Nice use


sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/BeamRecordSqlType.java, line 90 at r2 (raw file):

  public List<Integer> fieldTypes;

  public static BeamRecordSqlType create(List<String> fieldNames,

Is this class for internal use only or also do users interact with it? It should either be @Internal and with lots of caveats or should have javadoc on public factor methods like this one. Etc throughout (not actually related to this particular pull request)


Comments from Reviewable

@kennknowles kennknowles assigned akedin and unassigned kennknowles Jan 19, 2018
@akedin akedin force-pushed the generate-record-types branch from 692a1ae to a15ecb6 Compare January 19, 2018 23:56
@akedin
Copy link
Copy Markdown
Contributor Author

akedin commented Jan 19, 2018

Review status: all files reviewed at latest revision, 8 unresolved discussions, some commit checks failed.


sdks/java/core/src/main/java/org/apache/beam/sdk/values/reflect/BeamRecordFactory.java, line 53 at r2 (raw file):

Previously, kennknowles (Kenn Knowles) wrote…

Recommended pattern for classes like this: A single private constructor that takes all the arguments. Then for each way of building one, have a public static method with a useful name. So this constructor would be default() and the one below would be forRecordTypeFactory(...), or something like that.

Agree. Hidden the constructors, added BeamRecordFactory.createDefault().


sdks/java/core/src/main/java/org/apache/beam/sdk/values/reflect/BeamRecordFactory.java, line 81 at r2 (raw file):

Previously, kennknowles (Kenn Knowles) wrote…

Incidentally, BeamRecord should also have clear named factory methods instead of public constructor.

I agree. But there appears to be more than 70 usages of the constructors. I'd prefer to do this in another PR.


sdks/java/core/src/main/java/org/apache/beam/sdk/values/reflect/DefaultRecordTypeFactory.java, line 30 at r2 (raw file):

Previously, kennknowles (Kenn Knowles) wrote…

Can you describe what its behavior is like? Why is it the default? Also - is it @Internal?

Updated the javadoc with this.

Short version:

This factory creates record types from the list of getters.

Record types are represented by BeamRecordType which essentially is a List<Pair<FieldName, Coder>>. Getters are represented by a List<FieldValueGetter>. So the factory creates former from the latter.

Normally you can do this by mapping java types of the getters to coders using CoderRegistry or even manual mapping. This is what this factory does.

Problem with SQL is that it uses java.sq.Types to describe types of the record. It keeps this information in BeamRecordSqlType in addition to field names and coders. And it is referenced across a bunch of SQL classes. To map the java.sql.Types to coders we then need a custom mapping, which is done in a SQL-specific implementation of this interface.


sdks/java/core/src/main/java/org/apache/beam/sdk/values/reflect/package-info.java, line 22 at r2 (raw file):

Previously, kennknowles (Kenn Knowles) wrote…

Is there a special reason to make a new package? It does mean there might be things that are public but are not intended for users. So what we do in that case is mark them @Internal and also make a big bold comment that says "For internal use only. No backwards compatibility guarantees." Of if things that are public are what you want and you made stuff package private where appropriate, that's great.

And the other thing to do with the package is to set up static analysis with @DefaultAnnotation(NonNull.class)

The parent values package seems to contain mostly value types around PCollection with almost no other logic. Adding a ton of reflection and code-generation code there didn't seem right, they would become noise in the values package.

Haven't thought about @Internal until now, thanks for suggestion. Updated:

  • Combined org.apache.beam.sdk.values.reflect and org.apache.beam.sdk.values.reflect.field;
  • Made everything package-private except:
    • RecordTypeFactory interface. Used in SQL. Marked it as @Internal;
    • FieldValueGetter interface. Used by the above factory in SQL. Marked as @Internal;
    • BeamRecordFactory. Is the only functional class accessed outside of the package. Marked as @Internal;

sdks/java/core/src/main/java/org/apache/beam/sdk/values/reflect/field/FieldValueGetter.java, line 29 at r2 (raw file):

Previously, kennknowles (Kenn Knowles) wrote…

While this is a pretty harmless interface, I think it should still be @Internal. I assume that SQL and the new DoFn code will perhaps interface with it so it will have to be public no matter what package, yea?

I agree. Updated


sdks/java/core/src/main/java/org/apache/beam/sdk/values/reflect/field/GeneratedGetterFactory.java, line 69 at r2 (raw file):

Previously, kennknowles (Kenn Knowles) wrote…

With the DoFn stuff, we had issue where the stacktrace had pseudo-random bits in it so aggregated errors across workers didn't work. I think these bits are expected to be correct-by-construction and never allowed to throw exceptions, yes?

Hm, haven't really thought about this. Generated code itself doesn't throw but technically nothing stops the underlying getters from doing so. Do you have any ideas on how to handle this better?


sdks/java/core/src/main/java/org/apache/beam/sdk/values/reflect/field/package-info.java, line 22 at r2 (raw file):

Previously, kennknowles (Kenn Knowles) wrote…

Is there a benefit to even more deeply nesting packages? I'd keep it a bit more flat just to make things easy to find and make it most flexible in terms of managing public/private.

Combined with parent package.


sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/BeamRecordSqlType.java, line 90 at r2 (raw file):

Previously, kennknowles (Kenn Knowles) wrote…

Is this class for internal use only or also do users interact with it? It should either be @Internal and with lots of caveats or should have javadoc on public factor methods like this one. Etc throughout (not actually related to this particular pull request)

Added javadoc.

It's public. Until we have the code generation wired up the only way to create a record is to manually create a BeamRecordSqlType and then use it to create records.


Comments from Reviewable

@akedin akedin force-pushed the generate-record-types branch from a15ecb6 to 0763851 Compare January 20, 2018 00:08
@akedin
Copy link
Copy Markdown
Contributor Author

akedin commented Feb 1, 2018

This has been open for over 2 months now. @kennknowles , @iemejia do you see a path to get this merged? Otherwise I will close it

@kennknowles
Copy link
Copy Markdown
Member

Sorry about that. I think we should merge it. Technically, it has only been lying stale for 12 days... If you would just write less code, then I could get through the reviews! :-)

@kennknowles
Copy link
Copy Markdown
Member

:lgtm:

Any more comments from others? (don't let me LGTM dissuade further debate, please!)


Reviewed 1 of 22 files at r1, 3 of 20 files at r2, 20 of 31 files at r3.
Review status: 11 of 21 files reviewed at latest revision, 4 unresolved discussions, some commit checks failed.


sdks/java/core/src/main/java/org/apache/beam/sdk/values/reflect/BeamRecordFactory.java, line 81 at r2 (raw file):

Previously, akedin (Anton Kedin) wrote…

I agree. But there appears to be more than 70 usages of the constructors. I'd prefer to do this in another PR.

Yes, just a drive-by comment on past code.


sdks/java/core/src/main/java/org/apache/beam/sdk/values/reflect/DefaultRecordTypeFactory.java, line 30 at r2 (raw file):

Previously, akedin (Anton Kedin) wrote…

Updated the javadoc with this.

Short version:

This factory creates record types from the list of getters.

Record types are represented by BeamRecordType which essentially is a List<Pair<FieldName, Coder>>. Getters are represented by a List<FieldValueGetter>. So the factory creates former from the latter.

Normally you can do this by mapping java types of the getters to coders using CoderRegistry or even manual mapping. This is what this factory does.

Problem with SQL is that it uses java.sq.Types to describe types of the record. It keeps this information in BeamRecordSqlType in addition to field names and coders. And it is referenced across a bunch of SQL classes. To map the java.sql.Types to coders we then need a custom mapping, which is done in a SQL-specific implementation of this interface.

I see. LGTM at this juncture.


sdks/java/core/src/main/java/org/apache/beam/sdk/values/reflect/GeneratedGetterFactory.java, line 69 at r2 (raw file):

Previously, akedin (Anton Kedin) wrote…

Hm, haven't really thought about this. Generated code itself doesn't throw but technically nothing stops the underlying getters from doing so. Do you have any ideas on how to handle this better?

This was the fix for DoFnInvoker: https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/StableInvokerNamingStrategy.java

I think we can go with what you have for now anyhow and rethink later if it comes up. Not a big issue for POJOs, etc.


sdks/java/core/src/main/java/org/apache/beam/sdk/values/reflect/package-info.java, line 22 at r2 (raw file):

Previously, akedin (Anton Kedin) wrote…

The parent values package seems to contain mostly value types around PCollection with almost no other logic. Adding a ton of reflection and code-generation code there didn't seem right, they would become noise in the values package.

Haven't thought about @Internal until now, thanks for suggestion. Updated:

  • Combined org.apache.beam.sdk.values.reflect and org.apache.beam.sdk.values.reflect.field;
  • Made everything package-private except:
    • RecordTypeFactory interface. Used in SQL. Marked it as @Internal;
    • FieldValueGetter interface. Used by the above factory in SQL. Marked as @Internal;
    • BeamRecordFactory. Is the only functional class accessed outside of the package. Marked as @Internal;

OK, works for me!


Comments from Reviewable

@akedin akedin force-pushed the generate-record-types branch from 0763851 to 436d0ce Compare February 2, 2018 02:55
@akedin
Copy link
Copy Markdown
Contributor Author

akedin commented Feb 2, 2018

retest this please

@akedin
Copy link
Copy Markdown
Contributor Author

akedin commented Feb 2, 2018

tis green!

Copy link
Copy Markdown
Member

@iemejia iemejia left a comment

Choose a reason for hiding this comment

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

LGTM,

@iemejia
Copy link
Copy Markdown
Member

iemejia commented Feb 7, 2018

My excuses @akedin I have totally forgot about this one, and somehow I wasn't receiving any notifications. It is really nicer now. Thanks, Great work !

@iemejia
Copy link
Copy Markdown
Member

iemejia commented Feb 7, 2018

@akedin Can you please just resolve the existing conflict so I or @kennknowles can merge this immediately after.

@akedin akedin force-pushed the generate-record-types branch 2 times, most recently from a896c88 to d408c7e Compare February 7, 2018 19:21
@akedin
Copy link
Copy Markdown
Contributor Author

akedin commented Feb 8, 2018

found this with similar gradle failures: #4139

This is needed to avoid manual conversion from PCollection<Pojo> to PCollection<BeamRecord>.

Current usecase is Beam SQL and SQL Nexmark.

This can also be an exmple of how schema generation can be implemented internally for schema-aware PCollections.
@akedin akedin force-pushed the generate-record-types branch from d408c7e to f9f70e2 Compare February 8, 2018 03:03
@akedin
Copy link
Copy Markdown
Contributor Author

akedin commented Feb 8, 2018

yay it is green

@iemejia iemejia merged commit 6792846 into apache:master Feb 8, 2018
@iemejia
Copy link
Copy Markdown
Member

iemejia commented Feb 8, 2018

👍 Finally win the war against jenkins !

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.

4 participants