-
Notifications
You must be signed in to change notification settings - Fork 4.6k
[yaml] - mongodb read normalization #38772
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
Draft
derrickaw
wants to merge
14
commits into
apache:master
Choose a base branch
from
derrickaw:20260602_readFromMongoDb
base: master
Could not load branches
Branch not found: {{ refName }}
Loading
Could not load tags
Nothing to show
Loading
Are you sure you want to change the base?
Some commits from the old base branch may be removed from the timeline,
and old review comments may become outdated.
Draft
Changes from all commits
Commits
Show all changes
14 commits
Select commit
Hold shift + click to select a range
43bc9f4
feat(yaml): Add MongoDB read connector draft
arnavarora2004 3a05926
feat(yaml): Enhance MongoDB Read SchemaTransform, schema/BSON mapping…
derrickaw eb55f98
test(yaml): Add error-handling coverage for MongoDB Write and Read tr…
derrickaw 205d95a
remove old edits to other transforms from original PR
derrickaw 8095180
add python support
derrickaw 73aad3a
revert changes to integration tests not needed from original PR and f…
derrickaw 0a09db3
fix more lint
derrickaw 5808923
fix generate external transforms
derrickaw 9983d7a
pin read to python provider
derrickaw 3bec200
comment out read part
derrickaw e288769
retry readfrommongodb
derrickaw 79f4ff1
fix gemini comments
derrickaw bb2b75b
address gemini
derrickaw 5060088
Merge remote-tracking branch 'upstream' into 20260602_readFromMongoDb
derrickaw File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
89 changes: 89 additions & 0 deletions
89
...src/main/java/org/apache/beam/sdk/io/mongodb/MongoDbReadSchemaTransformConfiguration.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,89 @@ | ||
| /* | ||
| * Licensed to the Apache Software Foundation (ASF) under one | ||
| * or more contributor license agreements. See the NOTICE file | ||
| * distributed with this work for additional information | ||
| * regarding copyright ownership. The ASF licenses this file | ||
| * to you under the Apache License, Version 2.0 (the | ||
| * "License"); you may not use this file except in compliance | ||
| * with the License. You may obtain a copy of the License at | ||
| * | ||
| * http://www.apache.org/licenses/LICENSE-2.0 | ||
| * | ||
| * Unless required by applicable law or agreed to in writing, software | ||
| * distributed under the License is distributed on an "AS IS" BASIS, | ||
| * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
| * See the License for the specific language governing permissions and | ||
| * limitations under the License. | ||
| */ | ||
| package org.apache.beam.sdk.io.mongodb; | ||
|
|
||
| import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; | ||
|
|
||
| import com.google.auto.value.AutoValue; | ||
| import java.io.Serializable; | ||
| import org.apache.beam.sdk.schemas.AutoValueSchema; | ||
| import org.apache.beam.sdk.schemas.annotations.DefaultSchema; | ||
| import org.apache.beam.sdk.schemas.annotations.SchemaFieldDescription; | ||
| import org.apache.beam.sdk.schemas.transforms.providers.ErrorHandling; | ||
| import org.checkerframework.checker.nullness.qual.Nullable; | ||
|
|
||
| /** Configuration class for the MongoDB Read transform. */ | ||
| @DefaultSchema(AutoValueSchema.class) | ||
| @AutoValue | ||
| public abstract class MongoDbReadSchemaTransformConfiguration implements Serializable { | ||
|
|
||
| @SchemaFieldDescription("The connection URI for the MongoDB server.") | ||
| public abstract String getUri(); | ||
|
|
||
| @SchemaFieldDescription("The MongoDB database to read from.") | ||
| public abstract String getDatabase(); | ||
|
|
||
| @SchemaFieldDescription("The MongoDB collection to read from.") | ||
| public abstract String getCollection(); | ||
|
|
||
| @SchemaFieldDescription( | ||
| "The schema in which the data is encoded, defined with JSON-schema syntax (https://json-schema.org/).") | ||
| public abstract String getSchema(); | ||
|
|
||
| @SchemaFieldDescription( | ||
| "An optional BSON filter to apply to the read. This should be a valid JSON string.") | ||
| @Nullable | ||
| public abstract String getFilter(); | ||
|
|
||
| @SchemaFieldDescription( | ||
| "This option specifies whether and where to output rows that failed to be read.") | ||
| @Nullable | ||
| public abstract ErrorHandling getErrorHandling(); | ||
|
|
||
| public void validate() { | ||
| checkArgument(getUri() != null && !getUri().isEmpty(), "MongoDB URI must be specified."); | ||
| checkArgument( | ||
| getDatabase() != null && !getDatabase().isEmpty(), "MongoDB database must be specified."); | ||
| checkArgument( | ||
| getCollection() != null && !getCollection().isEmpty(), | ||
| "MongoDB collection must be specified."); | ||
| checkArgument( | ||
| getSchema() != null && !getSchema().isEmpty(), "MongoDB schema must be specified."); | ||
| } | ||
|
|
||
| public static Builder builder() { | ||
| return new AutoValue_MongoDbReadSchemaTransformConfiguration.Builder(); | ||
| } | ||
|
|
||
| @AutoValue.Builder | ||
| public abstract static class Builder { | ||
| public abstract Builder setUri(String uri); | ||
|
|
||
| public abstract Builder setDatabase(String database); | ||
|
|
||
| public abstract Builder setCollection(String collection); | ||
|
|
||
| public abstract Builder setSchema(String schema); | ||
|
|
||
| public abstract Builder setFilter(String filter); | ||
|
|
||
| public abstract Builder setErrorHandling(ErrorHandling errorHandling); | ||
|
|
||
| public abstract MongoDbReadSchemaTransformConfiguration build(); | ||
| } | ||
| } |
152 changes: 152 additions & 0 deletions
152
...godb/src/main/java/org/apache/beam/sdk/io/mongodb/MongoDbReadSchemaTransformProvider.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,152 @@ | ||
| /* | ||
| * Licensed to the Apache Software Foundation (ASF) under one | ||
| * or more contributor license agreements. See the NOTICE file | ||
| * distributed with this work for additional information | ||
| * regarding copyright ownership. The ASF licenses this file | ||
| * to you under the Apache License, Version 2.0 (the | ||
| * "License"); you may not use this file except in compliance | ||
| * with the License. You may obtain a copy of the License at | ||
| * | ||
| * http://www.apache.org/licenses/LICENSE-2.0 | ||
| * | ||
| * Unless required by applicable law or agreed to in writing, software | ||
| * distributed under the License is distributed on an "AS IS" BASIS, | ||
| * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
| * See the License for the specific language governing permissions and | ||
| * limitations under the License. | ||
| */ | ||
| package org.apache.beam.sdk.io.mongodb; | ||
|
|
||
| import com.google.auto.service.AutoService; | ||
| import java.util.Collections; | ||
| import java.util.List; | ||
| import org.apache.beam.sdk.schemas.Schema; | ||
| import org.apache.beam.sdk.schemas.transforms.SchemaTransform; | ||
| import org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider; | ||
| import org.apache.beam.sdk.schemas.transforms.TypedSchemaTransformProvider; | ||
| import org.apache.beam.sdk.schemas.transforms.providers.ErrorHandling; | ||
| import org.apache.beam.sdk.schemas.utils.JsonUtils; | ||
| import org.apache.beam.sdk.transforms.DoFn; | ||
| import org.apache.beam.sdk.transforms.ParDo; | ||
| import org.apache.beam.sdk.values.PCollection; | ||
| import org.apache.beam.sdk.values.PCollectionRowTuple; | ||
| import org.apache.beam.sdk.values.PCollectionTuple; | ||
| import org.apache.beam.sdk.values.Row; | ||
| import org.apache.beam.sdk.values.TupleTag; | ||
| import org.apache.beam.sdk.values.TupleTagList; | ||
| import org.bson.Document; | ||
|
|
||
| /** An implementation of {@link TypedSchemaTransformProvider} for reading from MongoDB. */ | ||
| @AutoService(SchemaTransformProvider.class) | ||
| public class MongoDbReadSchemaTransformProvider | ||
| extends TypedSchemaTransformProvider<MongoDbReadSchemaTransformConfiguration> { | ||
|
|
||
| private static final String OUTPUT_TAG_NAME = "output"; | ||
| public static final TupleTag<Row> OUTPUT_TAG = new TupleTag<Row>() {}; | ||
| public static final TupleTag<Row> ERROR_TAG = new TupleTag<Row>() {}; | ||
|
|
||
| private static final org.apache.beam.sdk.metrics.Counter errorCounter = | ||
| org.apache.beam.sdk.metrics.Metrics.counter( | ||
| MongoDbReadSchemaTransformProvider.class, "MongoDB-read-error-counter"); | ||
|
|
||
| @Override | ||
| protected SchemaTransform from(MongoDbReadSchemaTransformConfiguration configuration) { | ||
| return new MongoDbReadSchemaTransform(configuration); | ||
| } | ||
|
|
||
| @Override | ||
| public String identifier() { | ||
| return "beam:schematransform:org.apache.beam:mongodb_read:v1"; | ||
| } | ||
|
|
||
| @Override | ||
| public List<String> inputCollectionNames() { | ||
| return Collections.emptyList(); | ||
| } | ||
|
|
||
| @Override | ||
| public List<String> outputCollectionNames() { | ||
| return Collections.singletonList(OUTPUT_TAG_NAME); | ||
| } | ||
|
|
||
| /** The {@link SchemaTransform} that performs the read operation. */ | ||
| private static class MongoDbReadSchemaTransform extends SchemaTransform { | ||
| private final MongoDbReadSchemaTransformConfiguration configuration; | ||
|
|
||
| MongoDbReadSchemaTransform(MongoDbReadSchemaTransformConfiguration configuration) { | ||
| configuration.validate(); | ||
| this.configuration = configuration; | ||
| } | ||
|
|
||
| @Override | ||
| public PCollectionRowTuple expand(PCollectionRowTuple input) { | ||
| Schema schema = JsonUtils.beamSchemaFromJsonSchema(configuration.getSchema()); | ||
|
|
||
| MongoDbIO.Read read = | ||
| MongoDbIO.read() | ||
| .withUri(configuration.getUri()) | ||
| .withDatabase(configuration.getDatabase()) | ||
| .withCollection(configuration.getCollection()); | ||
|
|
||
| final String filterStr = configuration.getFilter(); | ||
| if (filterStr != null) { | ||
| read = | ||
| read.withQueryFn(collection -> collection.find(Document.parse(filterStr))); | ||
| } | ||
|
|
||
| PCollection<Document> mongoDocs = input.getPipeline().apply("ReadFromMongoDb", read); | ||
|
|
||
| boolean handleErrors = ErrorHandling.hasOutput(configuration.getErrorHandling()); | ||
| Schema errorSchema = ErrorHandling.errorSchemaBytes(); | ||
|
|
||
| PCollectionTuple outputTuple = | ||
| mongoDocs.apply( | ||
| "ConvertToBeamRows", | ||
| ParDo.of(new DocumentToRowFn(schema, handleErrors, errorSchema)) | ||
| .withOutputTags(OUTPUT_TAG, TupleTagList.of(ERROR_TAG))); | ||
|
|
||
| PCollection<Row> beamRows = outputTuple.get(OUTPUT_TAG).setRowSchema(schema); | ||
| PCollection<Row> errorOutput = outputTuple.get(ERROR_TAG).setRowSchema(errorSchema); | ||
|
|
||
| PCollectionRowTuple output = PCollectionRowTuple.of(OUTPUT_TAG_NAME, beamRows); | ||
| ErrorHandling errorHandling = configuration.getErrorHandling(); | ||
| if (handleErrors && errorHandling != null) { | ||
| output = output.and(errorHandling.getOutput(), errorOutput); | ||
| } | ||
| return output; | ||
| } | ||
| } | ||
|
|
||
| /** Converts a MongoDB BSON {@link Document} to a Beam {@link Row}. */ | ||
| static class DocumentToRowFn extends DoFn<Document, Row> { | ||
| private final Schema schema; | ||
| private final boolean handleErrors; | ||
| private final Schema errorSchema; | ||
|
|
||
| DocumentToRowFn(Schema schema, boolean handleErrors, Schema errorSchema) { | ||
| this.schema = schema; | ||
| this.handleErrors = handleErrors; | ||
| this.errorSchema = errorSchema; | ||
| } | ||
|
|
||
| @ProcessElement | ||
| public void processElement(@Element Document doc, MultiOutputReceiver receiver) { | ||
| try { | ||
| receiver.get(OUTPUT_TAG).output(MongoDbUtils.toRow(doc, schema)); | ||
| } catch (Exception e) { | ||
| if (!handleErrors) { | ||
| throw new RuntimeException( | ||
| "Failed to convert BSON Document to Beam Row: " + doc.toJson(), e); | ||
| } | ||
| errorCounter.inc(); | ||
| byte[] docBytes; | ||
| try { | ||
| docBytes = doc.toJson().getBytes(java.nio.charset.StandardCharsets.UTF_8); | ||
| } catch (Exception jsonEx) { | ||
| docBytes = doc.toString().getBytes(java.nio.charset.StandardCharsets.UTF_8); | ||
| } | ||
| receiver.get(ERROR_TAG).output(ErrorHandling.errorRecord(errorSchema, docBytes, e)); | ||
| } | ||
| } | ||
| } | ||
| } | ||
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
Uh oh!
There was an error while loading. Please reload this page.