Skip to content

[BEAM-5191] Support for BigQuery clustering#8945

Merged
chamikaramj merged 1 commit intoapache:masterfrom
jklukas:bq_clustering
Jul 22, 2019
Merged

[BEAM-5191] Support for BigQuery clustering#8945
chamikaramj merged 1 commit intoapache:masterfrom
jklukas:bq_clustering

Conversation

@jklukas
Copy link
Copy Markdown
Contributor

@jklukas jklukas commented Jun 25, 2019

This takes the commits from #7061, rebases on master, and adds an enableClustering method to allow users to opt in to the updated coder when using dynamic destinations.

This follows the pattern of #6914 where we added a new version of MetadataCoder, documenting how a user could opt in and why they might want to.

This should solve the coder compatibility concerns that were the blockers to merging #7061.


Thank you for your contribution! Follow this checklist to help us incorporate your contribution quickly and easily:

  • Choose reviewer(s) and mention them in a comment (R: @username).
  • Format the pull request title like [BEAM-XXX] Fixes bug in ApproximateQuantiles, where you replace BEAM-XXX with the appropriate JIRA issue, if applicable. This will automatically link the pull request to the issue.
  • If this contribution is large, please file an Apache Individual Contributor License Agreement.

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
Python Build Status
Build Status
--- Build Status
Build Status
Build Status --- --- Build Status

Pre-Commit Tests Status (on master branch)

--- Java Python Go Website
Non-portable 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.

@jklukas jklukas force-pushed the bq_clustering branch 2 times, most recently from eb18664 to 5acc4bf Compare June 26, 2019 13:52
@jklukas
Copy link
Copy Markdown
Contributor Author

jklukas commented Jun 26, 2019

R: @juancho088 @reuvenlax @alexvanboxel @chamikaramj who were reviewers on #7061

Also cc @wscheep who authored #7061 which is still the bulk of the code here.

@jklukas
Copy link
Copy Markdown
Contributor Author

jklukas commented Jun 26, 2019

Run JavaPortabilityApi PreCommit

.withTestServices(fakeBqServices)
.withMethod(BigQueryIO.Write.Method.FILE_LOADS)
.withSchema(schema)
.enableClustering()
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.

Notably, this test fails if enableClustering() is not called because the default TableDestinationCoderV2 is used and clustering information is dropped before the table is created. This is exactly the behavior we want for backwards compatibility.

@chamikaramj chamikaramj self-requested a review June 27, 2019 22:30
Copy link
Copy Markdown
Contributor

@chamikaramj chamikaramj left a comment

Choose a reason for hiding this comment

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

Thanks.

Coder<DestinationT> getDestinationCoderWithDefault(CoderRegistry registry)
throws CannotProvideCoderException {
return inner.getDestinationCoderWithDefault(registry);
Coder<DestinationT> destinationCoder = getDestinationCoder();
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

This seems to be a behavior change for non-clustering case ?
Previously we returned inner.getDestinationCoderWithDefault(registry) and now we return TableDestinationCoderV2.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.

DynamicDestinations#getDestinationCoderWithDefault is commented as:

  // Gets the destination coder. If the user does not provide one, try to find one in the coder
  // registry. If no coder can be found, throws CannotProvideCoderException.

This code is written with potentially multiple layers of delegation, and I think the correct behavior here is to return the first non-delegated implementation of getDestinationCoder() that appears as we move down the delegation chain.

I would argue that the existing behavior is incorrect. Currently, if an implementing class defines a custom return value for getDestinationCoder, that value is ignored when you call getDestinationCoderWithDefault. My expectation is that getDestinationCoderWithDefault would always return the same value as getDestinationCoder except in the null case, in which getDestinationCoderWithDefault would then attempt to look up a coder in the registry.

So the change here is intended to fix broken behavior.

It's possible that a user has written a custom class that extends DelegatingDynamicDestinations and relies on the incorrect behavior, but it feels unlikely to me.

For the scope of the coders provided here, I don't believe this change affects behavior (the method was already returning TableDestinationCoderV2 in all cases).

Copy link
Copy Markdown
Contributor Author

@jklukas jklukas Jul 1, 2019

Choose a reason for hiding this comment

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

It may actually be better to remove this override altogether. getDestinationCoderWithDefault is defined as package-private and probably shouldn't be overridden. Instead, we let getDestinationCoder handle delegation, and the base implementation of getDestinationCoderWithDefault will make the appropriate getDestinationCoder call before falling back to the registry.

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've removed the override getDestinationCoderWithDefault completely.

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've removed the override getDestinationCoderWithDefault completely.

Looks like that caused some regressions where destination coder couldn't be found from type. I'm looking into it.

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.

The override turns out to be necessary because the DynamicDestinations instance itself is eventually passed into extractFromTypeParameters and the inner class potentially has richer type information than the delegating class.

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've put the override method back in place, including the modification of first checking if we have a non-null return from getDestinationCoder().

.withCreateDisposition(BigQueryIO.Write.CreateDisposition.CREATE_IF_NEEDED)
.withWriteDisposition(BigQueryIO.Write.WriteDisposition.WRITE_TRUNCATE));

p.run().waitUntilFinish();
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Thanks for adding these ITs :)

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 believe these ITs are the work of @wscheep in the previous PR, and I am going to borrow liberally from these examples to add the dynamic destinations ITs.

p.apply(BigQueryIO.readTableRows().from(options.getInput()))
.apply(ParDo.of(new KeepStationNumberAndConvertDate()))
.apply(
BigQueryIO.writeTableRows()
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Can we also add one with dynamic destinations and clustering ?

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.

Added. It doesn't look like the ITs are run by Jenkins by default and I also can't find that there's a command to kick off BigQueryIO integration tests. Do you have any advice about how we can get results for these?

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Post-commit test suite should capture these. Just triggered it.

* option, since {@link TableDestinationCoderV3} will not be able to read state written with a
* previous version.
*/
public Write<T> enableClustering() {
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Having these two methods seems to make the API pretty brittle.

How about just having one function withClustering() that optionally takes a clustering object ? In dynamic destinations case optional Clustering object can be skipped/null and the method will behave similar to enableClustering() ?

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.

Implemented. enableClustering() is now withClustering() and it sets clustering to a default Clustering instance with no fields set, which we use as a flag for enabling clustering on dynamic destinations.

@jklukas jklukas force-pushed the bq_clustering branch 2 times, most recently from bb77061 to 328d14b Compare July 2, 2019 12:22
@chamikaramj
Copy link
Copy Markdown
Contributor

Run Java PostCommit

Copy link
Copy Markdown
Contributor

@chamikaramj chamikaramj left a comment

Choose a reason for hiding this comment

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

Thanks. Looks great.

Just a couple of comments.

}

@Test
public void testClusteringStreamingInserts() throws Exception {
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Probably also add a test that confirms that we fail if withClustering() is set without withTimePartitioning().

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.

Added

  @Test(expected = IllegalArgumentException.class)
  public void testClusteringThrowsWithoutPartitioning() throws Exception {

@chamikaramj
Copy link
Copy Markdown
Contributor

chamikaramj commented Jul 3, 2019

FYI: I'm OOO till 7/9 so please add another reviewer if you need to get this merged early.

LGTM from my side other than the two comments I mentioned and integration tests passing.

@jklukas
Copy link
Copy Markdown
Contributor Author

jklukas commented Jul 3, 2019

Run Java PostCommit

@jklukas
Copy link
Copy Markdown
Contributor Author

jklukas commented Jul 3, 2019

LGTM from my side other than the two comments I mentioned and integration tests passing.

Addressed those comments, integration tests passed, and now I've squashed and rebased on master.

FYI: I'm OOO till 7/9 so please add another reviewer if you need to get this merged early.

I'm also OOO next week, so if there are any further changes that need to be made, I'll be back on 7/15. We have several weeks before the branch cut date for 2.15, so I don't see any point in rushing.

And thanks for the thoughtful review, @chamikaramj .

@jklukas
Copy link
Copy Markdown
Contributor Author

jklukas commented Jul 5, 2019

Run Java PostCommit

@jklukas
Copy link
Copy Markdown
Contributor Author

jklukas commented Jul 5, 2019

Run Java PostCommit

* A {@link Coder} for {@link TableDestination} that includes time partitioning and clustering
* information. Users must opt in to this version of the coder by setting one of the clustering
* options on {@link BigQueryIO.Write}, otherwise {@link TableDestinationCoderV2} will be used and
* clustering information will be discarded.
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

if a user forgets to set a clustering option but creates a TableDestination with clustering information, will we warn them or just silently discard?

@jklukas
Copy link
Copy Markdown
Contributor Author

jklukas commented Jul 8, 2019 via email

@reuvenlax
Copy link
Copy Markdown
Contributor

reuvenlax commented Jul 8, 2019 via email

@chamikaramj
Copy link
Copy Markdown
Contributor

Thanks. I agree that a compile time (or even run-time) failure is better but a warning should be OK if this is not possible for some reason.

I'll wait till this is addressed for the merge.

@jklukas
Copy link
Copy Markdown
Contributor Author

jklukas commented Jul 17, 2019

Run Java PostCommit

@jklukas
Copy link
Copy Markdown
Contributor Author

jklukas commented Jul 17, 2019

Run Java PreCommit

@jklukas
Copy link
Copy Markdown
Contributor Author

jklukas commented Jul 17, 2019

Run JavaPortabilityApi PreCommit

+ " to use TableDestinationCoderV2. Set withClustering() on BigQueryIO.write() and, "
+ " if you provided a custom DynamicDestinations instance, override"
+ " getDestinationCoder() to return TableDestinationCoderV3.",
dynamicDestinations);
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.

We cannot know at compile-time whether a custom table function or DynamicDestinations instance will produce any destination with clustering enabled, so we have to check the produced destinations at runtime.

This check will cause a runtime failure for StreamingInserts if a clustered destination is produced without the relevant configuration to use the newer destination coder.

+ " to use TableDestinationCoderV2. Set withClustering() on BigQueryIO.write() and, "
+ " if you provided a custom DynamicDestinations instance, override"
+ " getDestinationCoder() to return TableDestinationCoderV3.",
dynamicDestinations);
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.

This check will cause a runtime failure for BatchLoads if a clustered destination is produced without the relevant configuration to use the newer destination coder.

@jklukas
Copy link
Copy Markdown
Contributor Author

jklukas commented Jul 18, 2019

R: @chamikaramj

I've added in runtime checks for both the streaming insert and batch load cases where the transform will fail with an exception that includes instructions about how to configure clustering if a clustered destination is produced without the right coder in place.

Does this look ready to merge?

@chamikaramj
Copy link
Copy Markdown
Contributor

LGTM. Thanks Jeff.

@chamikaramj
Copy link
Copy Markdown
Contributor

Run Java PostCommit

@chamikaramj
Copy link
Copy Markdown
Contributor

Run Dataflow ValidatesRunner

@jklukas
Copy link
Copy Markdown
Contributor Author

jklukas commented Jul 18, 2019

Run Java PostCommit

2 similar comments
@jklukas
Copy link
Copy Markdown
Contributor Author

jklukas commented Jul 19, 2019

Run Java PostCommit

@jklukas
Copy link
Copy Markdown
Contributor Author

jklukas commented Jul 19, 2019

Run Java PostCommit

@jklukas
Copy link
Copy Markdown
Contributor Author

jklukas commented Jul 19, 2019

Run Java PostCommit

@jklukas
Copy link
Copy Markdown
Contributor Author

jklukas commented Jul 19, 2019

Run Dataflow ValidatesRunner

@jklukas
Copy link
Copy Markdown
Contributor Author

jklukas commented Jul 22, 2019

@chamikaramj All the tests here are finally passing again after transient issues.

@chamikaramj
Copy link
Copy Markdown
Contributor

Thanks. Merging.

@chamikaramj chamikaramj merged commit 94a3c85 into apache:master Jul 22, 2019
@jklukas jklukas deleted the bq_clustering branch July 22, 2019 16:24
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