johannaojeling commented on code in PR #24663:
URL: https://github.com/apache/beam/pull/24663#discussion_r1049277665


##########
sdks/go/pkg/beam/io/mongodbio/write.go:
##########
@@ -0,0 +1,204 @@
+// 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 mongodbio
+
+import (
+       "context"
+       "fmt"
+       "reflect"
+
+       "github.com/apache/beam/sdks/v2/go/pkg/beam"
+       "github.com/apache/beam/sdks/v2/go/pkg/beam/register"
+       "github.com/apache/beam/sdks/v2/go/pkg/beam/util/structx"
+       "go.mongodb.org/mongo-driver/bson"
+       "go.mongodb.org/mongo-driver/bson/primitive"
+       "go.mongodb.org/mongo-driver/mongo"
+       "go.mongodb.org/mongo-driver/mongo/options"
+)
+
+const (
+       defaultWriteBatchSize = 1000
+       defaultWriteOrdered   = true
+)
+
+func init() {
+       register.Function1x2(createIDFn)
+       register.Emitter2[primitive.ObjectID, beam.Y]()
+
+       register.DoFn3x0[context.Context, beam.Y, func(beam.X, beam.Y)](
+               &extractIDFn{},
+       )
+       register.Emitter2[beam.X, beam.Y]()
+
+       register.DoFn4x1[context.Context, beam.X, beam.Y, func(beam.X), error](
+               &writeFn{},
+       )
+       register.Emitter1[primitive.ObjectID]()
+}
+
+// Write writes a PCollection<T> of a type T to MongoDB. T must be a struct 
with exported fields
+// that should have a "bson" tag. If the struct has a field with the bson tag 
"_id", the value of
+// that field will be used as the id of the document. Otherwise, a new id 
field of type
+// primitive.ObjectID will be generated for each document. Write returns a 
PCollection<K> of the
+// inserted id values with type K.
+//
+// The Write transform has the required parameters:
+//     - s: the scope of the pipeline
+//     - uri: the MongoDB connection string
+//     - database: the MongoDB database to write to
+//     - collection: the MongoDB collection to write to
+//     - col: the PCollection to write to MongoDB
+//
+// The Write transform takes a variadic number of WriteOptionFn which can set 
the WriteOption
+// fields:
+//     - BatchSize: the number of documents to write in a single batch. 
Defaults to 1000
+//     - Ordered: whether to execute the writes in order. Defaults to true
+func Write(
+       s beam.Scope,
+       uri string,
+       database string,
+       collection string,
+       col beam.PCollection,
+       opts ...WriteOptionFn,
+) beam.PCollection {
+       s = s.Scope("mongodbio.Write")
+
+       option := &WriteOption{
+               BatchSize: defaultWriteBatchSize,
+               Ordered:   defaultWriteOrdered,
+       }
+
+       for _, opt := range opts {
+               if err := opt(option); err != nil {
+                       panic(fmt.Sprintf("mongodbio.Write: invalid option: 
%v", err))
+               }
+       }
+
+       t := col.Type().Type()
+       idIndex := structx.FieldIndexByTag(t, bsonTag, "_id")
+
+       var keyed beam.PCollection
+
+       if idIndex == -1 {
+               pre := beam.ParDo(s, createIDFn, col)
+               keyed = beam.Reshuffle(s, pre)
+       } else {
+               keyed = beam.ParDo(
+                       s,
+                       newExtractIDFn(idIndex),
+                       col,
+                       beam.TypeDefinition{Var: beam.XType, T: 
t.Field(idIndex).Type},
+               )
+       }
+
+       return beam.ParDo(
+               s,
+               newWriteFn(uri, database, collection, option),
+               keyed,
+       )
+}
+
+func createIDFn(elem beam.Y) (primitive.ObjectID, beam.Y) {
+       id := primitive.NewObjectID()
+       return id, elem
+}
+
+type extractIDFn struct {
+       IDIndex int
+}
+
+func newExtractIDFn(idIndex int) *extractIDFn {
+       return &extractIDFn{
+               IDIndex: idIndex,
+       }
+}
+
+func (fn *extractIDFn) ProcessElement(
+       _ context.Context,
+       elem beam.Y,
+       emit func(beam.X, beam.Y),
+) {
+       id := reflect.ValueOf(elem).Field(fn.IDIndex).Interface()
+       emit(id, elem)
+}
+
+type writeFn struct {
+       mongoDBFn
+       BatchSize int64
+       Ordered   bool
+       models    []mongo.WriteModel
+}
+
+func newWriteFn(
+       uri string,
+       database string,
+       collection string,
+       option *WriteOption,
+) *writeFn {
+       return &writeFn{
+               mongoDBFn: mongoDBFn{
+                       URI:        uri,
+                       Database:   database,
+                       Collection: collection,
+               },
+               BatchSize: option.BatchSize,
+               Ordered:   option.Ordered,
+       }
+}
+
+func (fn *writeFn) ProcessElement(
+       ctx context.Context,
+       key beam.X,
+       value beam.Y,
+       emit func(beam.X),
+) error {
+       model := mongo.NewReplaceOneModel().
+               SetFilter(bson.M{"_id": key}).
+               SetUpsert(true).
+               SetReplacement(value)
+
+       fn.models = append(fn.models, model)
+
+       if len(fn.models) >= int(fn.BatchSize) {
+               if err := fn.flush(ctx); err != nil {
+                       return err
+               }
+       }
+
+       emit(key)

Review Comment:
   Having the same concern about exposing third party libraries here where ids 
are emitted. If the element struct did not have an id field, the id would have 
been generated with `createIDFn`, and with the current implementation that 
would be a `primitive.ObjectID`. Is it preferred to emit a `[12]byte` instead? 
(If the user themselves defined a struct with an id of type primitive.ObjectID, 
i.e. the id is extracted with `extractIDFn`, I think the use case is different 
and they should expect to get a primitive.ObjectID back)



-- 
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]

Reply via email to