Skip to content

Support DatabaseID in Datastore beam connector#27987

Merged
johnjcasey merged 34 commits intoapache:masterfrom
pl04351820:multidb
Oct 31, 2023
Merged

Support DatabaseID in Datastore beam connector#27987
johnjcasey merged 34 commits intoapache:masterfrom
pl04351820:multidb

Conversation

@pl04351820
Copy link
Copy Markdown
Contributor

@pl04351820 pl04351820 commented Aug 14, 2023

[Feature] Support DatabaseID in Datastore beam connector

  • This allows users to explicit set DatabaseID in datastore beam connector for read and write.
  • Updated existing beam connector to explicitly set projectID and databaseID.
  • The Read option will access DatabaseID via option class defined in DatastoreV1.
  • The Write option will access DatabaseID via FirestoreOptions.

@github-actions
Copy link
Copy Markdown
Contributor

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

R: @robertwb for label java.
R: @johnjcasey for label io.

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).

@bvolpato
Copy link
Copy Markdown
Contributor

Run Java_GCP_IO_Direct PreCommit

@pl04351820
Copy link
Copy Markdown
Contributor Author

Looks like the failing tests in Direct IO are coming from BigQuery and unrelated to the change:

[org.apache.beam.sdk.io.gcp.bigquery.BigQueryIOWriteTest.testWriteAvro[0]](https://ci-beam.apache.org/job/beam_PreCommit_Java_GCP_IO_Direct_Commit/3301/testReport/junit/org.apache.beam.sdk.io.gcp.bigquery/BigQueryIOWriteTest/testWriteAvro_0_/)
[org.apache.beam.sdk.io.gcp.bigquery.BigQueryIOWriteTest.testUpsertAndDeleteBeamRows[4]](https://ci-beam.apache.org/job/beam_PreCommit_Java_GCP_IO_Direct_Commit/3301/testReport/junit/org.apache.beam.sdk.io.gcp.bigquery/BigQueryIOWriteTest/testUpsertAndDeleteBeamRows_4_/)

public void startBundle(StartBundleContext c) {
datastore = datastoreFactory.getDatastore(c.getPipelineOptions(), projectId.get(), localhost);
// Read Firestore DatabaseID from FirestoreOptions.
String firestoreDatabaseId =
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.

why are we passing the databaseId around, but querying for it from options here? I would think that it should be passed in from the constructor.

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 hope users can run the datastore beam with flag for write. StartBundleContext.

  "--firestoreDb=${firestoreDb}",

Similar example in Firestore:

String databaseId = c.getPipelineOptions().as(FirestoreOptions.class).getFirestoreDb();

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 doesn't track to for me. It looks like you pass the value around when constricting the IO, but then query from pipeline options here. That is very confusing for a user.

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.

A user could do .withDatabaseId(ID), and that doesn't appear to do anything

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.

Thanks for pointing this out! Now I see why this is confusing.

It's unexpected users can do DatabaseID in WriteFn. The only way that can pass DatabaseID is from FirestoreOptions. Removed withDatabaseId references for Write.

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.

I think this is an antipattern. I don't think we should be passing in options via options. What if the user wanted to query from two databases in a single pipeline?

@pl04351820 pl04351820 requested a review from johnjcasey August 14, 2023 15:56
Copy link
Copy Markdown
Contributor

@johnjcasey johnjcasey left a comment

Choose a reason for hiding this comment

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

We shouldn't have databaseID passed in as a pipeline option. It doesn't support reading from two databases in a single pipeline

Copy link
Copy Markdown
Contributor

@bvolpato bvolpato left a comment

Choose a reason for hiding this comment

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

In general it LGTM -- my only concern is on the ValueProvider usage, as it might break and throw IllegalStateException when provided in runtime.

@pl04351820 pl04351820 requested a review from johnjcasey October 16, 2023 18:29
@pl04351820 pl04351820 requested a review from bvolpato October 16, 2023 22:14
@pl04351820
Copy link
Copy Markdown
Contributor Author

I run the failed tests locally and looks like they passed.

./gradlew integrationTest -p sdks/java/io/google-cloud-platform -PgcpProject={project-id} --tests=org.apache.beam.sdk.io.gcp.datastore.V1WriteIT.testE2EV1Write  --info --stacktrace

@pl04351820
Copy link
Copy Markdown
Contributor Author

Run Java_GCP_IO_Direct PreCommit

@bvolpato
Copy link
Copy Markdown
Contributor

Run Java_GCP_IO_Direct PreCommit

Copy link
Copy Markdown
Contributor

@bvolpato bvolpato left a comment

Choose a reason for hiding this comment

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

LGTM, thanks!

@johnjcasey johnjcasey merged commit 181296d into apache:master Oct 31, 2023
damccorm added a commit that referenced this pull request Nov 6, 2023
damccorm added a commit that referenced this pull request Nov 6, 2023
damccorm added a commit that referenced this pull request Nov 6, 2023
damccorm added a commit that referenced this pull request Nov 6, 2023
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.

4 participants