damccorm commented on code in PR #34127:
URL: https://github.com/apache/beam/pull/34127#discussion_r1976050396
##########
sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/MongoDbIO.java:
##########
@@ -312,7 +318,10 @@ public Read withBucketAuto(boolean bucketAuto) {
return builder().setBucketAuto(bucketAuto).build();
}
- /** Sets a queryFn. */
+ /**
+ * Sets a queryFn. The provided queryFn must be one of the predefined
classes in the MongoDbIO
+ * package such as FindQuery or AggregationQuery.
Review Comment:
You should be able to link to the actual objects like
https://github.com/apache/beam/blob/f4204fbeb9daeb6c9261c3b424ed84870eb702ed/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/MongoDbIO.java#L892
That will create links like
https://beam.apache.org/releases/javadoc/2.2.0/org/apache/beam/sdk/io/mongodb/MongoDbIO.Read.html#:~:text=%3Foptions%20are%20connection,withMaxConnectionIdleTime(int).
##########
sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/MongoDbIO.java:
##########
@@ -536,7 +545,7 @@ public List<BoundedSource<Document>> split(
LOG.debug("using filters: " + allFilters.toJson());
sources.add(new
BoundedMongoDbSource(spec.withQueryFn(queryWithFilter)));
}
- } else {
+ } else if (spec.queryFn().getClass() ==
AutoValue_AggregationQuery.class) {
Review Comment:
Instead of throwing in split (which will fail the pipeline at runtime), it
would be better to fail the pipeline during pipeline construction so that a
user doesn't need to wait until the pipeline submits. This actually won't fail
until the first time we try to split (which could take a bit), and I _think_ it
won't fail the pipeline we'll just fail the split attempt (not 100% sure about
that offhand).
We can do that by adding a check in expand like
https://github.com/apache/beam/blob/f4204fbeb9daeb6c9261c3b424ed84870eb702ed/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/MongoDbIO.java#L334
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]