Skip to content

[BEAM-4794] Move SQL and SQL Nexmark to the schema framework#5956

Merged
reuvenlax merged 10 commits intoapache:masterfrom
reuvenlax:sql_uses_schemas
Jul 31, 2018
Merged

[BEAM-4794] Move SQL and SQL Nexmark to the schema framework#5956
reuvenlax merged 10 commits intoapache:masterfrom
reuvenlax:sql_uses_schemas

Conversation

@reuvenlax
Copy link
Contributor

@reuvenlax reuvenlax commented Jul 16, 2018

This PR move SQL and Nexmark to the new schema framework.

Advantages:

  • SQL now works seamlessly on PCollection as long as UserType has a schema (which is simple for POJOs and Java Beans)
  • Row conversion for POJOs and Java Beans should be more efficient than the old version.
  • We now fully recursive and array fields in POJOs and Java Beans; these did not work previously in SQL.
  • A large amount of code is now deleted - InferredRowCoder, all the Nexmark ModelAdaptor code)

Note: this can not be merged until schemas are enabled for all major runners (BEAM-4793). A PR is currently out for review to make schemas work for all runners except for the Gearpump runner.

R: @apilloud

@echauchot
Copy link
Contributor

This is cool to see the first (AFAIK) real-pipeline use of the schema PCollection framework !

Copy link
Member

@apilloud apilloud left a comment

Choose a reason for hiding this comment

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

About half of the change is just reformatting so I might have missed something there, but LGTM.

}
}))
.setCoder(schema.getRowCoder());
.setSchema(schema, SerializableFunctions.identity(), SerializableFunctions.identity());
Copy link
Member

Choose a reason for hiding this comment

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

This pattern all over the place is kind of annoying. How about adding a setSchema(schema) function that uses the new of(Schema schema) so we don't have to repeat this pattern all over the place?

public class BeamPCollectionTable extends BaseBeamTable {
private transient PCollection<Row> upstream;
public class BeamPCollectionTable<InputT> extends BaseBeamTable {
private transient PCollection<InputT> upstream;
Copy link
Member

Choose a reason for hiding this comment

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

Not your bug, but this should be final instead of transient I believe.

LONG_CODER.encode(value.reserve, outStream);
LONG_CODER.encode(value.dateTime, outStream);
LONG_CODER.encode(value.expires, outStream);
INSTANT_CODER.encode(value.dateTime.toInstant(), outStream);
Copy link
Member

Choose a reason for hiding this comment

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

Thanks for cleaning this up!

@apilloud
Copy link
Member

Also, a good chuck of tests are failing. It would be good to run the nexmark postcommits before this goes in.

@reuvenlax
Copy link
Contributor Author

retest this please

}
}))
.setCoder(schema.getRowCoder());
.setSchema(schema, SerializableFunctions.identity(), SerializableFunctions.identity());
Copy link
Contributor

Choose a reason for hiding this comment

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

+1, some shorthand version would be helpful. Maybe default to identity() and have a couple of overloads of setSchema() to allow customization? Or wire it up to schema registry and default to identity() there?

And is setSchema the right place to specify these transforms? Isn't it just conflating .apply(ToRow.withSchema())... .apply(ParDo.of(fromRow()))? My thought is that if the transforms are non-trivial, then they are probably better be specified as ParDos

Copy link
Contributor Author

Choose a reason for hiding this comment

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

These functions are not being applied here. All schemas in Beam must be registered with a conversion function to/from Row. These conversions are applied by Beam at the appropriate places (e.g. in the DoFn state machine). They can't really be specified as part of ParDo, because they are needed to implement ParDo behind the scenes!

In this case, it's clearly not needed as the type is already Row. Will add a simpler function for PCollection

@reuvenlax
Copy link
Contributor Author

run Dataflow ValidatesRunner

@reuvenlax
Copy link
Contributor Author

run Flink ValidatesRunner

@reuvenlax
Copy link
Contributor Author

run Spark ValidatesRunner

@reuvenlax
Copy link
Contributor Author

run Java PostCommit

@reuvenlax
Copy link
Contributor Author

UnboundedEventSourceTest.resumeFromCheckpoint failed in Post Commit, but appears to simply be flaky. That same test ran successfully in PreCommit, and also passed locally.

@reuvenlax reuvenlax merged commit 06128f2 into apache:master Jul 31, 2018
@reuvenlax reuvenlax deleted the sql_uses_schemas branch December 9, 2018 23:07
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