Skip to content
1 change: 1 addition & 0 deletions sdks/java/io/expansion-service/build.gradle
Original file line number Diff line number Diff line change
Expand Up @@ -92,6 +92,7 @@ dependencies {
runtimeOnly project(path: ":sdks:java:io:iceberg:bqms", configuration: "shadow")
}

runtimeOnly project(":sdks:java:io:mongodb")
runtimeOnly library.java.kafka_clients
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.

How much does this add to I/O the expansion service jar ?

Also, are there any potential transitive dependencies that could bring in vulnerabilities ?

Copy link
Copy Markdown
Collaborator Author

Choose a reason for hiding this comment

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

I think it will be around an extra 3MB.

I think the risk is low for anything to come through.

runtimeOnly library.java.slf4j_jdk14

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,74 @@
/*
* 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 java.util.ArrayList;
import java.util.List;
import java.util.Map;
import org.apache.beam.sdk.schemas.Schema.Field;
import org.apache.beam.sdk.values.Row;
import org.bson.BsonNull;
import org.bson.Document;
import org.checkerframework.checker.nullness.qual.Nullable;

/** Utility methods for MongoDB IO. */
public class MongoDbUtils {

/** Converts a Beam {@link Row} to a BSON {@link Document}. */
public static Document toDocument(Row row) {
Object converted = convertToBsonValue(row);
if (converted instanceof Document) {
return (Document) converted;
}
throw new IllegalArgumentException(
"Expected Document but got "
+ (converted != null ? converted.getClass().getName() : "null"));
}

private static @Nullable Object convertToBsonValue(@Nullable Object value) {
if (value == null) {
return new BsonNull();
}
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 converted = convertToBsonValue(fieldValue);
doc.append(field.getName(), converted != null ? converted : new BsonNull());
}
return doc;
} else if (value instanceof Iterable) {
List<Object> bsonList = new ArrayList<>();
for (Object item : (Iterable<?>) value) {
Object converted = convertToBsonValue(item);
bsonList.add(converted != null ? converted : new BsonNull());
}
return bsonList;
} else if (value instanceof Map) {
Map<?, ?> map = (Map<?, ?>) value;
Document doc = new Document();
for (Map.Entry<?, ?> entry : map.entrySet()) {
Object converted = convertToBsonValue(entry.getValue());
doc.append(String.valueOf(entry.getKey()), converted != null ? converted : new BsonNull());
}
return doc;
}
return value;
}
}
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();
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,168 @@
/*
* 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.io.InputStream;
import java.io.OutputStream;
import java.io.Serializable;
import java.util.Collections;
import java.util.List;
import org.apache.beam.sdk.coders.AtomicCoder;
import org.apache.beam.sdk.coders.StringUtf8Coder;
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;

/** An implementation of {@link TypedSchemaTransformProvider} for writing to MongoDB. */
@AutoService(SchemaTransformProvider.class)
public class MongoDbWriteSchemaTransformProvider
extends TypedSchemaTransformProvider<MongoDbWriteSchemaTransformConfiguration> {

public static class DocumentCoder extends AtomicCoder<Document> implements Serializable {
private static final DocumentCoder INSTANCE = new DocumentCoder();

private DocumentCoder() {}

public static DocumentCoder of() {
return INSTANCE;
}

@Override
public void encode(Document value, OutputStream outStream) throws java.io.IOException {
StringUtf8Coder.of().encode(value.toJson(), outStream);
}

@Override
public Document decode(InputStream inStream) throws java.io.IOException {
String json = StringUtf8Coder.of().decode(inStream);
return Document.parse(json);
}
}

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).setCoder(DocumentCoder.of());

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

Seems like we are only reporting errors for the conversion part here. Are there are any errors from the MongoDB write transform that we might be missing here ?

Copy link
Copy Markdown
Collaborator Author

Choose a reason for hiding this comment

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

My understanding is that the underlying mongodbio doesn't support dlq so the only thing we can capture is conversion issues.


// 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 {
receiver.get(OUTPUT_TAG).output(MongoDbUtils.toDocument(row));
} catch (Exception e) {
if (!handleErrors) {
throw new RuntimeException(e);
}
errorCounter.inc();
receiver.get(ERROR_TAG).output(ErrorHandling.errorRecord(errorSchema, row, e));
}
}
}
}
Loading
Loading