[ https://issues.apache.org/jira/browse/BEAM-8427?focusedWorklogId=335177&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-335177 ]
ASF GitHub Bot logged work on BEAM-8427: ---------------------------------------- Author: ASF GitHub Bot Created on: 28/Oct/19 20:54 Start Date: 28/Oct/19 20:54 Worklog Time Spent: 10m Work Description: 11moon11 commented on pull request #9806: [BEAM-8427] Create a table and a table provider for MongoDB URL: https://github.com/apache/beam/pull/9806#discussion_r339787399 ########## File path: sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/mongodb/MongoDbTable.java ########## @@ -0,0 +1,118 @@ +/* + * 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.extensions.sql.meta.provider.mongodb; + +import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument; + +import java.io.Serializable; +import java.util.regex.Matcher; +import java.util.regex.Pattern; +import org.apache.beam.sdk.annotations.Experimental; +import org.apache.beam.sdk.extensions.sql.impl.BeamTableStatistics; +import org.apache.beam.sdk.extensions.sql.meta.SchemaBaseBeamTable; +import org.apache.beam.sdk.extensions.sql.meta.Table; +import org.apache.beam.sdk.io.mongodb.MongoDbIO; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.JsonToRow; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.ParDo.SingleOutput; +import org.apache.beam.sdk.values.PBegin; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollection.IsBounded; +import org.apache.beam.sdk.values.POutput; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.vendor.calcite.v1_20_0.com.google.common.annotations.VisibleForTesting; +import org.bson.Document; +import org.bson.json.JsonMode; +import org.bson.json.JsonWriterSettings; + +@Experimental +public class MongoDbTable extends SchemaBaseBeamTable implements Serializable { + // Should match: mongodb://username:password@localhost:27017/database/collection + @VisibleForTesting + final Pattern locationPattern = + Pattern.compile( + "(?<credsHostPort>mongodb://(?<usernamePassword>.*:.*@)?.+:\\d+)/(?<database>.+)/(?<collection>.+)"); + + @VisibleForTesting final String dbCollection; + @VisibleForTesting final String dbName; + @VisibleForTesting final String dbUri; + + MongoDbTable(Table table) { + super(table.getSchema()); + + String location = table.getLocation(); + Matcher matcher = locationPattern.matcher(location); + checkArgument( + matcher.matches(), + "MongoDb location must be in the following format: 'mongodb://(username:password@)?localhost:27017/database/collection'"); Review comment: I believe that `?` should be there, since `username:password@` are optional according to the JavaDoc here: https://github.com/apache/beam/blob/c2f0d282337f3ae0196a7717712396a5a41fdde1/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/MongoDbIO.java#L208-L225 It also states that `For some authentication mechanisms, only the username is specified and the password is not, in which case the ":" after the username is left off as well`, updated the pattern to take that into account as well (and added tests to make sure it works as intended). I decided to make a port number mandatory for now. I feel like it would be a good idea to break that regular expression down into multiple logical layers (ex: `layer1` -> extract `uri`, `database`, and `collection`; `layer2` -> extract `username:password@`, `port`). The main benefit is improved readability, especially if we plan to add support for all features mentioned in the JavaDoc: https://github.com/apache/beam/blob/c2f0d282337f3ae0196a7717712396a5a41fdde1/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/MongoDbIO.java#L202 ---------------------------------------------------------------- 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. For queries about this service, please contact Infrastructure at: us...@infra.apache.org Issue Time Tracking ------------------- Worklog Id: (was: 335177) Time Spent: 2h 20m (was: 2h 10m) > [SQL] Add support for MongoDB source > ------------------------------------ > > Key: BEAM-8427 > URL: https://issues.apache.org/jira/browse/BEAM-8427 > Project: Beam > Issue Type: New Feature > Components: dsl-sql > Reporter: Kirill Kozlov > Assignee: Kirill Kozlov > Priority: Major > Time Spent: 2h 20m > Remaining Estimate: 0h > > In progress: > * Create a MongoDB table and table provider. > * Implement buildIOReader > * Support primitive types > Still needs to be done: > * Implement buildIOWrite > * improve getTableStatistics -- This message was sent by Atlassian Jira (v8.3.4#803005)