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

[BEAM-991] Comply with byte limit for Datastore Commit. #2948

Closed
wants to merge 4 commits into from

Conversation

cph6
Copy link
Contributor

@cph6 cph6 commented May 8, 2017

This changes the Datastore connector to limit RPCs both by overall size and by the number of entities contained, to fit within the Datastore API limits https://cloud.google.com/datastore/docs/concepts/limits .

A similar change for the Python SDK is already written and will be submitted separately.

@coveralls
Copy link

Coverage Status

Coverage remained the same at 70.584% when pulling efec2ac on cph6:datastore_request_size_limit_java into 2b470ea on apache:master.

@dhalperi
Copy link
Contributor

dhalperi commented May 8, 2017

R: @vikkyrk

I thought this was deemed to be an incomplete fix because the underlying Cloud Datastore service might replicate the bytes N times depending on the number of indexes on the datastore.

@cph6
Copy link
Contributor Author

cph6 commented May 8, 2017

It is true that some requests complying with the limits might hit other internal limits in the stack. But this would be rare:

  • many Datastore users have only one entity per entity group, and so could not hit this limit (they would hit the entity size limit first).
  • typically, very large properties are (or can be) unindexed properties.

This fix helps most users that will see this problem.

Allowing for other cases — cases that the client cannot foresee — would need a more complex solution, but even if we implement that later, we would probably keep what this change adds: do not send requests exceeding the known limits.

Copy link
Contributor

@vikkyrk vikkyrk left a comment

Choose a reason for hiding this comment

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

I am fine with this as long it is well documented what cases work and what do not. Also it will be great if we can add an integration test here to test large entities https://github.com/apache/beam/blob/master/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1WriteIT.java.

@Test
public void testDatatoreWriterFnWithLargeEntities() throws Exception {
List<Mutation> mutations = new ArrayList<>();
for (int i = 0; i < 12; ++i) {
Copy link
Contributor

Choose a reason for hiding this comment

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

Can this be computed from DATASTORE_BATCH_UPDATE_BYTES_LIMIT, so that this test is robust against any changes in the value?

// requests, but we only need each CommitRequest to be less than 10MB in size.
int start = 0;
while (start < mutations.size()) {
int end = Math.min(mutations.size(), start + 4);
Copy link
Contributor

Choose a reason for hiding this comment

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

Can 4 be computed instead?

@cph6
Copy link
Contributor Author

cph6 commented May 9, 2017

Test changes done.

@vikkyrk
Copy link
Contributor

vikkyrk commented May 9, 2017

Run Java PostCommit

Copy link
Contributor

@vikkyrk vikkyrk left a comment

Choose a reason for hiding this comment

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

Nice, all tests are green. Mostly LGTM with minor comment.

@@ -208,6 +208,14 @@
static final int DATASTORE_BATCH_UPDATE_LIMIT = 500;

/**
* Cloud Datastore has a limit of 10MB per RPC, so we also flush if the total size of mutations
Copy link
Contributor

Choose a reason for hiding this comment

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

Can you update the doc here https://github.com/apache/beam/blob/master/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1.java#L183 the limitations of writes here (specifically about the possibility of large indexed entities failing).

@@ -1152,7 +1161,14 @@ public void startBundle(StartBundleContext c) {

@ProcessElement
public void processElement(ProcessContext c) throws Exception {
Copy link
Contributor

Choose a reason for hiding this comment

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

Just curious if this impacts deletes in any way, since this DoFn is used across writes and deletes.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Deletes are fine, I think, as they wouldn't send enough data per entity to trouble this limit.

Let's discuss docs separately.

@vikkyrk
Copy link
Contributor

vikkyrk commented May 17, 2017

LGTM, thanks! Please sync with master, and once tests pass I will ask one of the committers to merge it.

@coveralls
Copy link

Coverage Status

Coverage increased (+0.004%) to 70.675% when pulling 180bd86 on cph6:datastore_request_size_limit_java into 4924508 on apache:master.

@cph6
Copy link
Contributor Author

cph6 commented May 19, 2017

Thanks — updated.

@vikkyrk
Copy link
Contributor

vikkyrk commented May 19, 2017

Thanks. @jkff could you take a pass and merge this?

@asfgit asfgit closed this in 9a6baef May 19, 2017
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.

None yet

4 participants