Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Add schema to SpannerIO read #32008

Merged
merged 5 commits into from
Aug 27, 2024
Merged

Add schema to SpannerIO read #32008

merged 5 commits into from
Aug 27, 2024

Conversation

Polber
Copy link
Contributor

@Polber Polber commented Jul 26, 2024

Following change adds SpannerIO.readWithSchema() method for reading Spanner table schema. This performs a call to the source table to get schema metadata for constructing a RowCoder for the output PCollection without having to manually provide a schema.

Logic follows BigQuerySourceDef very closely.


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

  • Mention the appropriate issue in your description (for example: addresses #123), if applicable. This will automatically add a link to the pull request in the issue. If you would like the issue to automatically close on merging the pull request, comment fixes #<ISSUE NUMBER> instead.
  • Update CHANGES.md with noteworthy changes.
  • If this contribution is large, please file an Apache Individual Contributor License Agreement.

See the Contributor Guide for more tips on how to make review process smoother.

To check the build health, please visit https://github.com/apache/beam/blob/master/.test-infra/BUILD_STATUS.md

GitHub Actions Tests Status (on master branch)

Build python source distribution and wheels
Python tests
Java tests
Go tests

See CI.md for more information about GitHub Actions CI or the workflows README to see a list of phrases to trigger workflows.

Signed-off-by: Jeffrey Kinard <jeff@thekinards.com>
Copy link
Contributor

Checks are failing. Will not request review until checks are succeeding. If you'd like to override that behavior, comment assign set of reviewers

Signed-off-by: Jeffrey Kinard <jeff@thekinards.com>
@Abacn
Copy link
Contributor

Abacn commented Jul 31, 2024

Format check error message

 Name 'StructTypeWithStruct' must match pattern '^[a-z][a-zA-Z0-9]*$'. [LocalVariableName]

have you been able to find SpannerIO reviewer?

@Abacn
Copy link
Contributor

Abacn commented Jul 31, 2024

If this is meant for DataflowTemplates, we should push for GoogleCloudPlatform/DataflowTemplates#1429 again...

@Polber
Copy link
Contributor Author

Polber commented Jul 31, 2024

@Abacn

If this is meant for DataflowTemplates, we should push for GoogleCloudPlatform/DataflowTemplates#1429 again...

This is meant as a prerequisite to #31987 which adds ReadFromSpanner to Beam YAML.

It allows schema reads from Spanner without needing to specify a schema file which makes using in Beam YAML much more user-friendly and similar to how ReadFromBigQuery works.

Format check error message

 Name 'StructTypeWithStruct' must match pattern '^[a-z][a-zA-Z0-9]*$'. [LocalVariableName]

I can fix format error

have you been able to find SpannerIO reviewer?

No, I was waiting for checks to pass before finding one

Signed-off-by: Jeffrey Kinard <jeff@thekinards.com>
Copy link
Contributor

github-actions bot commented Aug 9, 2024

Assigning reviewers. If you would like to opt out of this review, comment assign to next reviewer:

R: @kennknowles for label java.
R: @shunping for label io.
R: @nielm for label spanner.

Available commands:

  • stop reviewer notifications - opt out of the automated review tooling
  • remind me after tests pass - tag the comment author after tests pass
  • waiting on author - shift the attention set back to the author (any comment or push by the author will return the attention set to the reviewers)

The PR bot will only process comments in the main thread (not review comments).

@damccorm
Copy link
Contributor

@Polber what are next steps here?

@Abacn
Copy link
Contributor

Abacn commented Aug 14, 2024

Pin a few potential @nielm @atask-g SpannerIO (owner) reviewer

@damccorm
Copy link
Contributor

Pin a few potential @nielm @atask-g SpannerIO (owner) reviewer

Looks like these folks are out - @manitgupta could you help with a review here (or recommend someone to help)?

@manitgupta
Copy link
Contributor

@bharadwaj-aditya @darshan-sj Can you have a look please?
I will review, but I have not reviewed many SpannerIO PRs in the past..

@nielm
Copy link
Contributor

nielm commented Aug 21, 2024

I am ooo until September but can review it then.

case FLOAT64:
return Schema.FieldType.DOUBLE;
case NUMERIC:
return Schema.FieldType.DECIMAL;
Copy link
Contributor

Choose a reason for hiding this comment

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

Is it expected that the datatype of the corresponding values in the rowStruct to be of certain datatype? Like what is the datatype expected for DECIMAL? Does Spanner client library give the data in the expected datatype for all these types?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Since data types are not 1:1 for many different sources in Beam (BigQuery has lots of data types that are exclusive to BigQuery and do not exist as Java types) we map to the closest Java type.

For example, FLOAT64 is not a Java type, but Double is implemented as a 64-bit float, so the cast will work when the data is deserialized.

As I said in my other comment, all the data types that were added in this PR were tested and all the data conversions worked successfully

Copy link
Contributor

@damccorm damccorm left a comment

Choose a reason for hiding this comment

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

Had 2 comments, (one just a nit), otherwise LGTM

Signed-off-by: Jeffrey Kinard <jeff@thekinards.com>
Comment on lines 916 to 924
private SpannerSourceDef createSourceDef() {
SpannerSourceDef sourceDef;
if (getReadOperation().getQuery() != null) {
return SpannerQuerySourceDef.create(getSpannerConfig(), getReadOperation().getQuery());
}

return SpannerTableSourceDef.create(
getSpannerConfig(), getReadOperation().getTable(), getReadOperation().getColumns());
}
Copy link
Contributor

Choose a reason for hiding this comment

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

My crappy markdown code didn't work

Suggested change
private SpannerSourceDef createSourceDef() {
SpannerSourceDef sourceDef;
if (getReadOperation().getQuery() != null) {
return SpannerQuerySourceDef.create(getSpannerConfig(), getReadOperation().getQuery());
}
return SpannerTableSourceDef.create(
getSpannerConfig(), getReadOperation().getTable(), getReadOperation().getColumns());
}
private SpannerSourceDef createSourceDef() {
if (getReadOperation().getQuery() != null) {
return SpannerQuerySourceDef.create(getSpannerConfig(), getReadOperation().getQuery());
}
return SpannerTableSourceDef.create(
getSpannerConfig(), getReadOperation().getTable(), getReadOperation().getColumns());

Copy link
Contributor

Choose a reason for hiding this comment

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

Probably worth doing this locally and running ./gradlew :sdks:java:io:google-cloud-platform:spotlessJava' as well to avoid more precommit failures

Signed-off-by: Jeffrey Kinard <jeff@thekinards.com>
Copy link
Contributor

@damccorm damccorm left a comment

Choose a reason for hiding this comment

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

Thanks!

@damccorm damccorm merged commit 24255ac into apache:master Aug 27, 2024
18 checks passed
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.

6 participants