-
Notifications
You must be signed in to change notification settings - Fork 4.6k
[yaml] - mongodb write normalization #38376
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
Open
derrickaw
wants to merge
12
commits into
apache:master
Choose a base branch
from
derrickaw:20260505_mongodb_write
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.
Open
Changes from 11 commits
Commits
Show all changes
12 commits
Select commit
Hold shift + click to select a range
f9e27bd
MongoDB IO write connector for beam yaml
arnavarora2004 b1ec3d2
revert bigtable change
derrickaw 8fcfc33
add yaml and more transformer write support
derrickaw a7bba0b
more edits to support parity between java and python
derrickaw b8e9a48
remove read
derrickaw 4335cea
remove updateConfiguration
derrickaw a5512e2
remove updateField
derrickaw ac45b5b
update external transforms
derrickaw 0cfb387
remove unnecessary comments
derrickaw 9d38507
add clarifying comments
derrickaw e847410
fix gemini comments
derrickaw 6070b81
address gemini comments
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
85 changes: 85 additions & 0 deletions
85
...rc/main/java/org/apache/beam/sdk/io/mongodb/MongoDbWriteSchemaTransformConfiguration.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,85 @@ | ||
| /* | ||
| * 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 Write transform. */ | ||
| @DefaultSchema(AutoValueSchema.class) | ||
| @AutoValue | ||
| public abstract class MongoDbWriteSchemaTransformConfiguration implements Serializable { | ||
|
|
||
| @SchemaFieldDescription("The connection URI for the MongoDB server.") | ||
| public abstract String getUri(); | ||
|
|
||
| @SchemaFieldDescription("The MongoDB database to write to.") | ||
| public abstract String getDatabase(); | ||
|
|
||
| @SchemaFieldDescription("The MongoDB collection to write to.") | ||
| public abstract String getCollection(); | ||
|
|
||
| @SchemaFieldDescription("The number of documents to include in each batch write.") | ||
| @Nullable | ||
| public abstract Long getBatchSize(); | ||
|
|
||
| @SchemaFieldDescription( | ||
| "This option specifies whether and where to output unwritable rows. Note: Error handling is currently limited to data conversion failures before sending to the MongoDB driver, as the underlying MongoDbIO does not yet support dead-letter queues for write failures.") | ||
| @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."); | ||
|
|
||
| Long batchSize = getBatchSize(); | ||
| if (batchSize != null) { | ||
| checkArgument(batchSize > 0, "Batch size must be positive."); | ||
| } | ||
| } | ||
|
|
||
| public static Builder builder() { | ||
| return new AutoValue_MongoDbWriteSchemaTransformConfiguration.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 setBatchSize(Long batchSize); | ||
|
|
||
| public abstract Builder setErrorHandling(ErrorHandling errorHandling); | ||
|
|
||
| public abstract MongoDbWriteSchemaTransformConfiguration build(); | ||
| } | ||
| } |
201 changes: 201 additions & 0 deletions
201
...odb/src/main/java/org/apache/beam/sdk/io/mongodb/MongoDbWriteSchemaTransformProvider.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,201 @@ | ||
| /* | ||
| * 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.ArrayList; | ||
| import java.util.Collections; | ||
| import java.util.List; | ||
| import java.util.Map; | ||
| import org.apache.beam.sdk.schemas.Schema.Field; | ||
| 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.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; | ||
| import org.checkerframework.checker.nullness.qual.Nullable; | ||
|
|
||
| /** An implementation of {@link TypedSchemaTransformProvider} for writing to MongoDB. */ | ||
| @AutoService(SchemaTransformProvider.class) | ||
| public class MongoDbWriteSchemaTransformProvider | ||
| extends TypedSchemaTransformProvider<MongoDbWriteSchemaTransformConfiguration> { | ||
|
|
||
| private static final String INPUT_TAG = "input"; | ||
| public static final TupleTag<Document> OUTPUT_TAG = new TupleTag<Document>() {}; | ||
| 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( | ||
| MongoDbWriteSchemaTransformProvider.class, "MongoDB-write-error-counter"); | ||
|
|
||
| @Override | ||
| protected SchemaTransform from(MongoDbWriteSchemaTransformConfiguration configuration) { | ||
| return new MongoDbWriteSchemaTransform(configuration); | ||
| } | ||
|
|
||
| @Override | ||
| public String identifier() { | ||
| return "beam:schematransform:org.apache.beam:mongodb_write:v1"; | ||
| } | ||
|
|
||
| @Override | ||
| public List<String> inputCollectionNames() { | ||
| return Collections.singletonList(INPUT_TAG); | ||
| } | ||
|
|
||
| /** The {@link SchemaTransform} that performs the write operation. */ | ||
| private static class MongoDbWriteSchemaTransform extends SchemaTransform { | ||
| private final MongoDbWriteSchemaTransformConfiguration configuration; | ||
|
|
||
| MongoDbWriteSchemaTransform(MongoDbWriteSchemaTransformConfiguration configuration) { | ||
| configuration.validate(); | ||
| this.configuration = configuration; | ||
| } | ||
|
|
||
| @Override | ||
| public PCollectionRowTuple expand(PCollectionRowTuple input) { | ||
| // Retrieve the input PCollection of Rows and its schema. | ||
| PCollection<Row> rows = input.get(INPUT_TAG); | ||
| org.apache.beam.sdk.schemas.Schema inputSchema = rows.getSchema(); | ||
|
|
||
| // Determine if error handling is enabled and set up the error schema. | ||
| boolean handleErrors = ErrorHandling.hasOutput(configuration.getErrorHandling()); | ||
| org.apache.beam.sdk.schemas.Schema errorSchema = ErrorHandling.errorSchema(inputSchema); | ||
|
|
||
| // Convert Beam Rows to BSON Documents, emitting errors to a separate tag if enabled. | ||
| PCollectionTuple outputTuple = | ||
| rows.apply( | ||
| "ConvertToDocument", | ||
| ParDo.of(new RowToBsonDocumentFn(handleErrors, errorSchema)) | ||
| .withOutputTags(OUTPUT_TAG, TupleTagList.of(ERROR_TAG))); | ||
|
|
||
| PCollection<Document> documents = outputTuple.get(OUTPUT_TAG); | ||
|
|
||
| // Configure the MongoDB write operation. | ||
| MongoDbIO.Write write = | ||
| MongoDbIO.write() | ||
| .withUri(configuration.getUri()) | ||
| .withDatabase(configuration.getDatabase()) | ||
| .withCollection(configuration.getCollection()); | ||
|
|
||
| Long batchSize = configuration.getBatchSize(); | ||
| if (batchSize != null) { | ||
| write = write.withBatchSize(batchSize); | ||
| } | ||
|
|
||
| // Apply the MongoDB write transform. | ||
| documents.apply("WriteToMongo", write); | ||
|
|
||
| // Extract and format the error collection. | ||
| PCollection<Row> errorOutput = outputTuple.get(ERROR_TAG).setRowSchema(errorSchema); | ||
|
|
||
| // Return the error collection as specified by the configuration. | ||
| ErrorHandling errorHandling = configuration.getErrorHandling(); | ||
| return PCollectionRowTuple.of( | ||
| (handleErrors && errorHandling != null) ? errorHandling.getOutput() : "errors", | ||
| errorOutput); | ||
| } | ||
| } | ||
|
|
||
| /** Converts a Beam {@link Row} to a BSON {@link Document}. */ | ||
| static class RowToBsonDocumentFn extends DoFn<Row, Document> { | ||
| private final boolean handleErrors; | ||
| private final org.apache.beam.sdk.schemas.Schema errorSchema; | ||
|
|
||
| RowToBsonDocumentFn(boolean handleErrors, org.apache.beam.sdk.schemas.Schema errorSchema) { | ||
| this.handleErrors = handleErrors; | ||
| this.errorSchema = errorSchema; | ||
| } | ||
|
|
||
| @ProcessElement | ||
| public void processElement(@Element Row row, MultiOutputReceiver receiver) { | ||
| try { | ||
| Object converted = convertToBsonValue(row); | ||
| if (converted instanceof Document) { | ||
| receiver.get(OUTPUT_TAG).output((Document) converted); | ||
| } else { | ||
| throw new IllegalStateException( | ||
| "Expected Document but got " | ||
| + (converted != null ? converted.getClass().getName() : "null")); | ||
| } | ||
| } catch (Exception e) { | ||
| if (!handleErrors) { | ||
| throw new RuntimeException(e); | ||
| } | ||
| errorCounter.inc(); | ||
| receiver.get(ERROR_TAG).output(ErrorHandling.errorRecord(errorSchema, row, e)); | ||
| } | ||
| } | ||
| } | ||
|
|
||
| private static @Nullable Object convertToBsonValue(@Nullable Object value) { | ||
| if (value == null) { | ||
| return null; | ||
| } | ||
| if (value instanceof Row) { | ||
| Row row = (Row) value; | ||
| Document doc = new Document(); | ||
| for (Field field : row.getSchema().getFields()) { | ||
| Object fieldValue = row.getValue(field.getName()); | ||
| Object convertedValue = convertToBsonValue(fieldValue); | ||
| if (convertedValue != null) { | ||
| doc.append(field.getName(), convertedValue); | ||
| } | ||
| } | ||
| return doc; | ||
| } else if (value instanceof List) { | ||
| List<?> list = (List<?>) value; | ||
| List<Object> bsonList = new ArrayList<>(list.size()); | ||
| for (Object item : list) { | ||
| Object convertedItem = convertToBsonValue(item); | ||
| if (convertedItem != null) { | ||
| bsonList.add(convertedItem); | ||
| } | ||
| } | ||
| return bsonList; | ||
| } else if (value instanceof Iterable) { | ||
| List<Object> bsonList = new ArrayList<>(); | ||
| for (Object item : (Iterable<?>) value) { | ||
| Object convertedItem = convertToBsonValue(item); | ||
| if (convertedItem != null) { | ||
| bsonList.add(convertedItem); | ||
| } | ||
| } | ||
| return bsonList; | ||
|
derrickaw marked this conversation as resolved.
Outdated
|
||
| } else if (value instanceof Map) { | ||
| Map<?, ?> map = (Map<?, ?>) value; | ||
| Document doc = new Document(); | ||
| for (Map.Entry<?, ?> entry : map.entrySet()) { | ||
| Object convertedValue = convertToBsonValue(entry.getValue()); | ||
| if (convertedValue != null) { | ||
| doc.append(String.valueOf(entry.getKey()), convertedValue); | ||
| } | ||
|
derrickaw marked this conversation as resolved.
Outdated
|
||
| } | ||
| return doc; | ||
| } | ||
| return value; | ||
| } | ||
| } | ||
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.