johannaojeling commented on code in PR #24663: URL: https://github.com/apache/beam/pull/24663#discussion_r1058485073
########## sdks/go/pkg/beam/io/mongodbio/read.go: ########## @@ -0,0 +1,469 @@ +// 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" + "math" + "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/mongo" + "go.mongodb.org/mongo-driver/mongo/options" + "go.mongodb.org/mongo-driver/mongo/readpref" +) + +const ( + defaultReadBundleSize = 64 * 1024 * 1024 + + minSplitVectorChunkSize = 1024 * 1024 + maxSplitVectorChunkSize = 1024 * 1024 * 1024 + + maxBucketCount = math.MaxInt32 +) + +func init() { + register.DoFn3x1[context.Context, []byte, func(bson.M), error](&bucketAutoFn{}) + register.DoFn3x1[context.Context, []byte, func(bson.M), error](&splitVectorFn{}) + register.Emitter1[bson.M]() + + register.DoFn3x1[context.Context, bson.M, func(beam.Y), error](&readFn{}) + register.Emitter1[beam.Y]() +} + +// Read reads a MongoDB collection and returns a PCollection<T> for a given type T. T must be a +// struct with exported fields that should have a "bson" tag. By default, the transform uses the +// MongoDB internal splitVector command to split the collection into bundles. The transform can be +// configured to use the $bucketAuto aggregation instead to support reading from MongoDB Atlas +// where the splitVector command is not allowed. This is enabled by passing the ReadOptionFn +// WithReadBucketAuto(true). +// +// The Read transform has the required parameters: +// - s: the scope of the pipeline +// - uri: the MongoDB connection string +// - database: the MongoDB database to read from +// - collection: the MongoDB collection to read from +// - t: the type of the elements in the collection +// +// The Read transform takes a variadic number of ReadOptionFn which can set the ReadOption fields: +// - BucketAuto: whether to use the bucketAuto aggregation to split the collection into bundles. +// Defaults to false +// - Filter: a bson.M map that is used to filter the documents in the collection. Defaults to nil, +// which means no filter is applied +// - BundleSize: the size in bytes to bundle the documents into when reading. Defaults to +// 64 * 1024 * 1024 (64 MB) +func Read( + s beam.Scope, + uri string, + database string, + collection string, + t reflect.Type, + opts ...ReadOptionFn, +) beam.PCollection { + s = s.Scope("mongodbio.Read") + + option := &ReadOption{ + BundleSize: defaultReadBundleSize, + } + + for _, opt := range opts { + if err := opt(option); err != nil { + panic(fmt.Sprintf("mongodbio.Read: invalid option: %v", err)) + } + } + + imp := beam.Impulse(s) + + var bundled beam.PCollection + + if option.BucketAuto { + bundled = beam.ParDo(s, newBucketAutoFn(uri, database, collection, option), imp) + } else { + bundled = beam.ParDo(s, newSplitVectorFn(uri, database, collection, option), imp) Review Comment: Yes, correct. I would like to give the splittable implementation a try but will need to read up a bit more on how it works so I will open a separate issue for it. One question though: the splits in this case will be ID ranges/filters and to retrieve those from Mongo we need a context. The operation may also result in an error. Since the `SplitRestriction` function doesn't have a context parameter and error return type (until https://github.com/apache/beam/issues/20607 is fixed), some kind of workaround would be needed. I see `textio.Read` uses a pre DoFn `sizeFn` to resolve the context/error access for getting the file size used to create the initial restriction. However in `mongodbio.Read` the context is needed for getting the splits (what `bucketAutoFn` and `splitVectorFn` do now). Some less ideal approaches I can think of are attaching a context to the DoFn struct or using a context.Background. And to panic instead of returning an error if something goes wrong. Do you have a better approach in mind or what would be your recommendation? -- 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]
